From a4f4fbc8fa5886a8c6ee58ee614de0cc6e67dcd7 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Mon, 24 Feb 2014 21:13:38 -0800 Subject: [PATCH 001/397] Include reference to twitter/chill in tuning docs Author: Andrew Ash Closes #647 from ash211/doc-tuning and squashes the following commits: b87de0a [Andrew Ash] Include reference to twitter/chill in tuning docs --- docs/tuning.md | 9 ++++++--- 1 file changed, 6 insertions(+), 3 deletions(-) diff --git a/docs/tuning.md b/docs/tuning.md index 6b010aed618a3..704778681cb8f 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -44,7 +44,10 @@ This setting configures the serializer used for not only shuffling data between nodes but also when serializing RDDs to disk. The only reason Kryo is not the default is because of the custom registration requirement, but we recommend trying it in any network-intensive application. -Finally, to register your classes with Kryo, create a public class that extends +Spark automatically includes Kryo serializers for the many commonly-used core Scala classes covered +in the AllScalaRegistrar from the [Twitter chill](https://github.com/twitter/chill) library. + +To register your own custom classes with Kryo, create a public class that extends [`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the `spark.kryo.registrator` config property to point to it, as follows: @@ -72,8 +75,8 @@ If your objects are large, you may also need to increase the `spark.kryoserializ config property. The default is 2, but this value needs to be large enough to hold the *largest* object you will serialize. -Finally, if you don't register your classes, Kryo will still work, but it will have to store the -full class name with each object, which is wasteful. +Finally, if you don't register your custom classes, Kryo will still work, but it will have to store +the full class name with each object, which is wasteful. # Memory Tuning From 1f4c7f7ecc9d2393663fc4d059e71fe4c70bad84 Mon Sep 17 00:00:00 2001 From: Semih Salihoglu Date: Mon, 24 Feb 2014 22:42:30 -0800 Subject: [PATCH 002/397] Graph primitives2 Hi guys, I'm following Joey and Ankur's suggestions to add collectEdges and pickRandomVertex. I'm also adding the tests for collectEdges and refactoring one method getCycleGraph in GraphOpsSuite.scala. Thank you, semih Author: Semih Salihoglu Closes #580 from semihsalihoglu/GraphPrimitives2 and squashes the following commits: 937d3ec [Semih Salihoglu] - Fixed the scalastyle errors. a69a152 [Semih Salihoglu] - Adding collectEdges and pickRandomVertices. - Adding tests for collectEdges. - Refactoring a getCycle utility function for GraphOpsSuite.scala. 41265a6 [Semih Salihoglu] - Adding collectEdges and pickRandomVertex. - Adding tests for collectEdges. - Recycling a getCycle utility test file. --- .../org/apache/spark/graphx/GraphOps.scala | 59 +++++++- .../apache/spark/graphx/GraphOpsSuite.scala | 134 ++++++++++++++++-- 2 files changed, 183 insertions(+), 10 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 0fc1e4df6813c..377d9d6bd5e72 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -18,11 +18,11 @@ package org.apache.spark.graphx import scala.reflect.ClassTag - import org.apache.spark.SparkContext._ import org.apache.spark.SparkException import org.apache.spark.graphx.lib._ import org.apache.spark.rdd.RDD +import scala.util.Random /** * Contains additional functionality for [[Graph]]. All operations are expressed in terms of the @@ -137,6 +137,42 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali } } // end of collectNeighbor + /** + * Returns an RDD that contains for each vertex v its local edges, + * i.e., the edges that are incident on v, in the user-specified direction. + * Warning: note that singleton vertices, those with no edges in the given + * direction will not be part of the return value. + * + * @note This function could be highly inefficient on power-law + * graphs where high degree vertices may force a large amount of + * information to be collected to a single location. + * + * @param edgeDirection the direction along which to collect + * the local edges of vertices + * + * @return the local edges for each vertex + */ + def collectEdges(edgeDirection: EdgeDirection): VertexRDD[Array[Edge[ED]]] = { + edgeDirection match { + case EdgeDirection.Either => + graph.mapReduceTriplets[Array[Edge[ED]]]( + edge => Iterator((edge.srcId, Array(new Edge(edge.srcId, edge.dstId, edge.attr))), + (edge.dstId, Array(new Edge(edge.srcId, edge.dstId, edge.attr)))), + (a, b) => a ++ b) + case EdgeDirection.In => + graph.mapReduceTriplets[Array[Edge[ED]]]( + edge => Iterator((edge.dstId, Array(new Edge(edge.srcId, edge.dstId, edge.attr)))), + (a, b) => a ++ b) + case EdgeDirection.Out => + graph.mapReduceTriplets[Array[Edge[ED]]]( + edge => Iterator((edge.srcId, Array(new Edge(edge.srcId, edge.dstId, edge.attr)))), + (a, b) => a ++ b) + case EdgeDirection.Both => + throw new SparkException("collectEdges does not support EdgeDirection.Both. Use" + + "EdgeDirection.Either instead.") + } + } + /** * Join the vertices with an RDD and then apply a function from the * the vertex and RDD entry to a new vertex value. The input table @@ -209,6 +245,27 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali graph.mask(preprocess(graph).subgraph(epred, vpred)) } + /** + * Picks a random vertex from the graph and returns its ID. + */ + def pickRandomVertex(): VertexId = { + val probability = 50 / graph.numVertices + var found = false + var retVal: VertexId = null.asInstanceOf[VertexId] + while (!found) { + val selectedVertices = graph.vertices.flatMap { vidVvals => + if (Random.nextDouble() < probability) { Some(vidVvals._1) } + else { None } + } + if (selectedVertices.count > 1) { + found = true + val collectedVertices = selectedVertices.collect() + retVal = collectedVertices(Random.nextInt(collectedVertices.size)) + } + } + retVal + } + /** * Execute a Pregel-like iterative vertex-parallel abstraction. The * user-defined vertex-program `vprog` is executed in parallel on diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index bc2ad5677f806..6386306c048fc 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -42,21 +42,20 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { test("collectNeighborIds") { withSpark { sc => - val chain = (0 until 100).map(x => (x, (x+1)%100) ) - val rawEdges = sc.parallelize(chain, 3).map { case (s,d) => (s.toLong, d.toLong) } - val graph = Graph.fromEdgeTuples(rawEdges, 1.0).cache() + val graph = getCycleGraph(sc, 100) val nbrs = graph.collectNeighborIds(EdgeDirection.Either).cache() - assert(nbrs.count === chain.size) + assert(nbrs.count === 100) assert(graph.numVertices === nbrs.count) nbrs.collect.foreach { case (vid, nbrs) => assert(nbrs.size === 2) } - nbrs.collect.foreach { case (vid, nbrs) => - val s = nbrs.toSet - assert(s.contains((vid + 1) % 100)) - assert(s.contains(if (vid > 0) vid - 1 else 99 )) + nbrs.collect.foreach { + case (vid, nbrs) => + val s = nbrs.toSet + assert(s.contains((vid + 1) % 100)) + assert(s.contains(if (vid > 0) vid - 1 else 99)) } } } - + test ("filter") { withSpark { sc => val n = 5 @@ -80,4 +79,121 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { } } + test("collectEdgesCycleDirectionOut") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val edges = graph.collectEdges(EdgeDirection.Out).cache() + assert(edges.count == 100) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeDstIds = s.map(e => e.dstId) + assert(edgeDstIds.contains((vid + 1) % 100)) + } + } + } + + test("collectEdgesCycleDirectionIn") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val edges = graph.collectEdges(EdgeDirection.In).cache() + assert(edges.count == 100) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeSrcIds = s.map(e => e.srcId) + assert(edgeSrcIds.contains(if (vid > 0) vid - 1 else 99)) + } + } + } + + test("collectEdgesCycleDirectionEither") { + withSpark { sc => + val graph = getCycleGraph(sc, 100) + val edges = graph.collectEdges(EdgeDirection.Either).cache() + assert(edges.count == 100) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 2) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeIds = s.map(e => if (vid != e.srcId) e.srcId else e.dstId) + assert(edgeIds.contains((vid + 1) % 100)) + assert(edgeIds.contains(if (vid > 0) vid - 1 else 99)) + } + } + } + + test("collectEdgesChainDirectionOut") { + withSpark { sc => + val graph = getChainGraph(sc, 50) + val edges = graph.collectEdges(EdgeDirection.Out).cache() + assert(edges.count == 49) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeDstIds = s.map(e => e.dstId) + assert(edgeDstIds.contains(vid + 1)) + } + } + } + + test("collectEdgesChainDirectionIn") { + withSpark { sc => + val graph = getChainGraph(sc, 50) + val edges = graph.collectEdges(EdgeDirection.In).cache() + // We expect only 49 because collectEdges does not return vertices that do + // not have any edges in the specified direction. + assert(edges.count == 49) + edges.collect.foreach { case (vid, edges) => assert(edges.size == 1) } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeDstIds = s.map(e => e.srcId) + assert(edgeDstIds.contains((vid - 1) % 100)) + } + } + } + + test("collectEdgesChainDirectionEither") { + withSpark { sc => + val graph = getChainGraph(sc, 50) + val edges = graph.collectEdges(EdgeDirection.Either).cache() + // We expect only 49 because collectEdges does not return vertices that do + // not have any edges in the specified direction. + assert(edges.count === 50) + edges.collect.foreach { + case (vid, edges) => if (vid > 0 && vid < 49) assert(edges.size == 2) + else assert(edges.size == 1) + } + edges.collect.foreach { + case (vid, edges) => + val s = edges.toSet + val edgeIds = s.map(e => if (vid != e.srcId) e.srcId else e.dstId) + if (vid == 0) { assert(edgeIds.contains(1)) } + else if (vid == 49) { assert(edgeIds.contains(48)) } + else { + assert(edgeIds.contains(vid + 1)) + assert(edgeIds.contains(vid - 1)) + } + } + } + } + + private def getCycleGraph(sc: SparkContext, numVertices: Int): Graph[Double, Int] = { + val cycle = (0 until numVertices).map(x => (x, (x + 1) % numVertices)) + getGraphFromSeq(sc, cycle) + } + + private def getChainGraph(sc: SparkContext, numVertices: Int): Graph[Double, Int] = { + val chain = (0 until numVertices - 1).map(x => (x, (x + 1))) + getGraphFromSeq(sc, chain) + } + + private def getGraphFromSeq(sc: SparkContext, seq: IndexedSeq[(Int, Int)]): Graph[Double, Int] = { + val rawEdges = sc.parallelize(seq, 3).map { case (s, d) => (s.toLong, d.toLong) } + Graph.fromEdgeTuples(rawEdges, 1.0).cache() + } } From c852201ce95c7c982ff3794c114427eb33e92922 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Mon, 24 Feb 2014 23:20:38 -0800 Subject: [PATCH 003/397] For SPARK-1082, Use Curator for ZK interaction in standalone cluster Author: Raymond Liu Closes #611 from colorant/curator and squashes the following commits: 7556aa1 [Raymond Liu] Address review comments af92e1f [Raymond Liu] Fix coding style 964f3c2 [Raymond Liu] Ignore NodeExists exception 6df2966 [Raymond Liu] Rewrite zookeeper client code with curator --- core/pom.xml | 4 +- .../deploy/master/LeaderElectionAgent.scala | 1 + .../spark/deploy/master/MasterMessages.scala | 4 - .../deploy/master/SparkCuratorUtil.scala | 53 +++++ .../deploy/master/SparkZooKeeperSession.scala | 205 ------------------ .../master/ZooKeeperLeaderElectionAgent.scala | 94 +++----- .../master/ZooKeeperPersistenceEngine.scala | 30 +-- pom.xml | 6 +- project/SparkBuild.scala | 2 +- 9 files changed, 99 insertions(+), 300 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala delete mode 100644 core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala diff --git a/core/pom.xml b/core/pom.xml index 5576b0c3b4795..f209704f319de 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -55,8 +55,8 @@ avro-ipc - org.apache.zookeeper - zookeeper + org.apache.curator + curator-recipes org.eclipse.jetty diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala index f25a1ad3bf92a..a730fe1f599af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -30,6 +30,7 @@ import org.apache.spark.deploy.master.MasterMessages.ElectedLeader * [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]] */ private[spark] trait LeaderElectionAgent extends Actor { + //TODO: LeaderElectionAgent does not necessary to be an Actor anymore, need refactoring. val masterActor: ActorRef } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala index 74a9f8cd824fb..db72d8ae9bdaf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/MasterMessages.scala @@ -28,10 +28,6 @@ private[master] object MasterMessages { case object RevokedLeadership - // Actor System to LeaderElectionAgent - - case object CheckLeader - // Actor System to Master case object CheckForWorkerTimeOut diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala new file mode 100644 index 0000000000000..2d35397035a03 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala @@ -0,0 +1,53 @@ +/* + * 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.deploy.master + +import org.apache.spark.{SparkConf, Logging} +import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} +import org.apache.curator.retry.ExponentialBackoffRetry +import org.apache.zookeeper.KeeperException + + +object SparkCuratorUtil extends Logging { + + val ZK_CONNECTION_TIMEOUT_MILLIS = 15000 + val ZK_SESSION_TIMEOUT_MILLIS = 60000 + val RETRY_WAIT_MILLIS = 5000 + val MAX_RECONNECT_ATTEMPTS = 3 + + def newClient(conf: SparkConf): CuratorFramework = { + val ZK_URL = conf.get("spark.deploy.zookeeper.url") + val zk = CuratorFrameworkFactory.newClient(ZK_URL, + ZK_SESSION_TIMEOUT_MILLIS, ZK_CONNECTION_TIMEOUT_MILLIS, + new ExponentialBackoffRetry(RETRY_WAIT_MILLIS, MAX_RECONNECT_ATTEMPTS)) + zk.start() + zk + } + + def mkdir(zk: CuratorFramework, path: String) { + if (zk.checkExists().forPath(path) == null) { + try { + zk.create().creatingParentsIfNeeded().forPath(path) + } catch { + case nodeExist: KeeperException.NodeExistsException => + // do nothing, ignore node existing exception. + case e: Exception => throw e + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala deleted file mode 100644 index 57758055b19c0..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkZooKeeperSession.scala +++ /dev/null @@ -1,205 +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.deploy.master - -import scala.collection.JavaConversions._ - -import org.apache.zookeeper._ -import org.apache.zookeeper.Watcher.Event.KeeperState -import org.apache.zookeeper.data.Stat - -import org.apache.spark.{Logging, SparkConf} - -/** - * Provides a Scala-side interface to the standard ZooKeeper client, with the addition of retry - * logic. If the ZooKeeper session expires or otherwise dies, a new ZooKeeper session will be - * created. If ZooKeeper remains down after several retries, the given - * [[org.apache.spark.deploy.master.SparkZooKeeperWatcher SparkZooKeeperWatcher]] will be - * informed via zkDown(). - * - * Additionally, all commands sent to ZooKeeper will be retried until they either fail too many - * times or a semantic exception is thrown (e.g., "node already exists"). - */ -private[spark] class SparkZooKeeperSession(zkWatcher: SparkZooKeeperWatcher, - conf: SparkConf) extends Logging { - val ZK_URL = conf.get("spark.deploy.zookeeper.url", "") - - val ZK_ACL = ZooDefs.Ids.OPEN_ACL_UNSAFE - val ZK_TIMEOUT_MILLIS = 30000 - val RETRY_WAIT_MILLIS = 5000 - val ZK_CHECK_PERIOD_MILLIS = 10000 - val MAX_RECONNECT_ATTEMPTS = 3 - - private var zk: ZooKeeper = _ - - private val watcher = new ZooKeeperWatcher() - private var reconnectAttempts = 0 - private var closed = false - - /** Connect to ZooKeeper to start the session. Must be called before anything else. */ - def connect() { - connectToZooKeeper() - - new Thread() { - override def run() = sessionMonitorThread() - }.start() - } - - def sessionMonitorThread(): Unit = { - while (!closed) { - Thread.sleep(ZK_CHECK_PERIOD_MILLIS) - if (zk.getState != ZooKeeper.States.CONNECTED) { - reconnectAttempts += 1 - val attemptsLeft = MAX_RECONNECT_ATTEMPTS - reconnectAttempts - if (attemptsLeft <= 0) { - logError("Could not connect to ZooKeeper: system failure") - zkWatcher.zkDown() - close() - } else { - logWarning("ZooKeeper connection failed, retrying " + attemptsLeft + " more times...") - connectToZooKeeper() - } - } - } - } - - def close() { - if (!closed && zk != null) { zk.close() } - closed = true - } - - private def connectToZooKeeper() { - if (zk != null) zk.close() - zk = new ZooKeeper(ZK_URL, ZK_TIMEOUT_MILLIS, watcher) - } - - /** - * Attempts to maintain a live ZooKeeper exception despite (very) transient failures. - * Mainly useful for handling the natural ZooKeeper session expiration. - */ - private class ZooKeeperWatcher extends Watcher { - def process(event: WatchedEvent) { - if (closed) { return } - - event.getState match { - case KeeperState.SyncConnected => - reconnectAttempts = 0 - zkWatcher.zkSessionCreated() - case KeeperState.Expired => - connectToZooKeeper() - case KeeperState.Disconnected => - logWarning("ZooKeeper disconnected, will retry...") - case s => // Do nothing - } - } - } - - def create(path: String, bytes: Array[Byte], createMode: CreateMode): String = { - retry { - zk.create(path, bytes, ZK_ACL, createMode) - } - } - - def exists(path: String, watcher: Watcher = null): Stat = { - retry { - zk.exists(path, watcher) - } - } - - def getChildren(path: String, watcher: Watcher = null): List[String] = { - retry { - zk.getChildren(path, watcher).toList - } - } - - def getData(path: String): Array[Byte] = { - retry { - zk.getData(path, false, null) - } - } - - def delete(path: String, version: Int = -1): Unit = { - retry { - zk.delete(path, version) - } - } - - /** - * Creates the given directory (non-recursively) if it doesn't exist. - * All znodes are created in PERSISTENT mode with no data. - */ - def mkdir(path: String) { - if (exists(path) == null) { - try { - create(path, "".getBytes, CreateMode.PERSISTENT) - } catch { - case e: Exception => - // If the exception caused the directory not to be created, bubble it up, - // otherwise ignore it. - if (exists(path) == null) { throw e } - } - } - } - - /** - * Recursively creates all directories up to the given one. - * All znodes are created in PERSISTENT mode with no data. - */ - def mkdirRecursive(path: String) { - var fullDir = "" - for (dentry <- path.split("/").tail) { - fullDir += "/" + dentry - mkdir(fullDir) - } - } - - /** - * Retries the given function up to 3 times. The assumption is that failure is transient, - * UNLESS it is a semantic exception (i.e., trying to get data from a node that doesn't exist), - * in which case the exception will be thrown without retries. - * - * @param fn Block to execute, possibly multiple times. - */ - def retry[T](fn: => T, n: Int = MAX_RECONNECT_ATTEMPTS): T = { - try { - fn - } catch { - case e: KeeperException.NoNodeException => throw e - case e: KeeperException.NodeExistsException => throw e - case e: Exception if n > 0 => - logError("ZooKeeper exception, " + n + " more retries...", e) - Thread.sleep(RETRY_WAIT_MILLIS) - retry(fn, n-1) - } - } -} - -trait SparkZooKeeperWatcher { - /** - * Called whenever a ZK session is created -- - * this will occur when we create our first session as well as each time - * the session expires or errors out. - */ - def zkSessionCreated() - - /** - * Called if ZK appears to be completely down (i.e., not just a transient error). - * We will no longer attempt to reconnect to ZK, and the SparkZooKeeperSession is considered dead. - */ - def zkDown() -} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala index 47b8f67f8a45b..285f9b014e291 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperLeaderElectionAgent.scala @@ -18,105 +18,67 @@ package org.apache.spark.deploy.master import akka.actor.ActorRef -import org.apache.zookeeper._ -import org.apache.zookeeper.Watcher.Event.EventType import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.master.MasterMessages._ +import org.apache.curator.framework.CuratorFramework +import org.apache.curator.framework.recipes.leader.{LeaderLatchListener, LeaderLatch} private[spark] class ZooKeeperLeaderElectionAgent(val masterActor: ActorRef, masterUrl: String, conf: SparkConf) - extends LeaderElectionAgent with SparkZooKeeperWatcher with Logging { + extends LeaderElectionAgent with LeaderLatchListener with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/leader_election" - private val watcher = new ZooKeeperWatcher() - private val zk = new SparkZooKeeperSession(this, conf) + private var zk: CuratorFramework = _ + private var leaderLatch: LeaderLatch = _ private var status = LeadershipStatus.NOT_LEADER - private var myLeaderFile: String = _ - private var leaderUrl: String = _ override def preStart() { + logInfo("Starting ZooKeeper LeaderElection agent") - zk.connect() - } + zk = SparkCuratorUtil.newClient(conf) + leaderLatch = new LeaderLatch(zk, WORKING_DIR) + leaderLatch.addListener(this) - override def zkSessionCreated() { - synchronized { - zk.mkdirRecursive(WORKING_DIR) - myLeaderFile = - zk.create(WORKING_DIR + "/master_", masterUrl.getBytes, CreateMode.EPHEMERAL_SEQUENTIAL) - self ! CheckLeader - } + leaderLatch.start() } override def preRestart(reason: scala.Throwable, message: scala.Option[scala.Any]) { - logError("LeaderElectionAgent failed, waiting " + zk.ZK_TIMEOUT_MILLIS + "...", reason) - Thread.sleep(zk.ZK_TIMEOUT_MILLIS) + logError("LeaderElectionAgent failed...", reason) super.preRestart(reason, message) } - override def zkDown() { - logError("ZooKeeper down! LeaderElectionAgent shutting down Master.") - System.exit(1) - } - override def postStop() { + leaderLatch.close() zk.close() } override def receive = { - case CheckLeader => checkLeader() + case _ => } - private class ZooKeeperWatcher extends Watcher { - def process(event: WatchedEvent) { - if (event.getType == EventType.NodeDeleted) { - logInfo("Leader file disappeared, a master is down!") - self ! CheckLeader + override def isLeader() { + synchronized { + // could have lost leadership by now. + if (!leaderLatch.hasLeadership) { + return } - } - } - /** Uses ZK leader election. Navigates several ZK potholes along the way. */ - def checkLeader() { - val masters = zk.getChildren(WORKING_DIR).toList - val leader = masters.sorted.head - val leaderFile = WORKING_DIR + "/" + leader - - // Setup a watch for the current leader. - zk.exists(leaderFile, watcher) - - try { - leaderUrl = new String(zk.getData(leaderFile)) - } catch { - // A NoNodeException may be thrown if old leader died since the start of this method call. - // This is fine -- just check again, since we're guaranteed to see the new values. - case e: KeeperException.NoNodeException => - logInfo("Leader disappeared while reading it -- finding next leader") - checkLeader() - return + logInfo("We have gained leadership") + updateLeadershipStatus(true) } + } - // Synchronization used to ensure no interleaving between the creation of a new session and the - // checking of a leader, which could cause us to delete our real leader file erroneously. + override def notLeader() { synchronized { - val isLeader = myLeaderFile == leaderFile - if (!isLeader && leaderUrl == masterUrl) { - // We found a different master file pointing to this process. - // This can happen in the following two cases: - // (1) The master process was restarted on the same node. - // (2) The ZK server died between creating the file and returning the name of the file. - // For this case, we will end up creating a second file, and MUST explicitly delete the - // first one, since our ZK session is still open. - // Note that this deletion will cause a NodeDeleted event to be fired so we check again for - // leader changes. - assert(leaderFile < myLeaderFile) - logWarning("Cleaning up old ZK master election file that points to this master.") - zk.delete(leaderFile) - } else { - updateLeadershipStatus(isLeader) + // could have gained leadership by now. + if (leaderLatch.hasLeadership) { + return } + + logInfo("We have lost leadership") + updateLeadershipStatus(false) } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 48b2fc06a9d70..939006239d2b1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -17,36 +17,28 @@ package org.apache.spark.deploy.master +import scala.collection.JavaConversions._ + import akka.serialization.Serialization -import org.apache.zookeeper._ +import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) extends PersistenceEngine - with SparkZooKeeperWatcher with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" + val zk = SparkCuratorUtil.newClient(conf) - val zk = new SparkZooKeeperSession(this, conf) - - zk.connect() - - override def zkSessionCreated() { - zk.mkdirRecursive(WORKING_DIR) - } - - override def zkDown() { - logError("PersistenceEngine disconnected from ZooKeeper -- ZK looks down.") - } + SparkCuratorUtil.mkdir(zk, WORKING_DIR) override def addApplication(app: ApplicationInfo) { serializeIntoFile(WORKING_DIR + "/app_" + app.id, app) } override def removeApplication(app: ApplicationInfo) { - zk.delete(WORKING_DIR + "/app_" + app.id) + zk.delete().forPath(WORKING_DIR + "/app_" + app.id) } override def addDriver(driver: DriverInfo) { @@ -54,7 +46,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) } override def removeDriver(driver: DriverInfo) { - zk.delete(WORKING_DIR + "/driver_" + driver.id) + zk.delete().forPath(WORKING_DIR + "/driver_" + driver.id) } override def addWorker(worker: WorkerInfo) { @@ -62,7 +54,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) } override def removeWorker(worker: WorkerInfo) { - zk.delete(WORKING_DIR + "/worker_" + worker.id) + zk.delete().forPath(WORKING_DIR + "/worker_" + worker.id) } override def close() { @@ -70,7 +62,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) } override def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = { - val sortedFiles = zk.getChildren(WORKING_DIR).toList.sorted + val sortedFiles = zk.getChildren().forPath(WORKING_DIR).toList.sorted val appFiles = sortedFiles.filter(_.startsWith("app_")) val apps = appFiles.map(deserializeFromFile[ApplicationInfo]) val driverFiles = sortedFiles.filter(_.startsWith("driver_")) @@ -83,11 +75,11 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) private def serializeIntoFile(path: String, value: AnyRef) { val serializer = serialization.findSerializerFor(value) val serialized = serializer.toBinary(value) - zk.create(path, serialized, CreateMode.PERSISTENT) + zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) } def deserializeFromFile[T](filename: String)(implicit m: Manifest[T]): T = { - val fileData = zk.getData(WORKING_DIR + "/" + filename) + val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) val clazz = m.runtimeClass.asInstanceOf[Class[T]] val serializer = serialization.serializerFor(clazz) serializer.fromBinary(fileData).asInstanceOf[T] diff --git a/pom.xml b/pom.xml index 3a530685b8e5a..4f1e8398d9b8d 100644 --- a/pom.xml +++ b/pom.xml @@ -393,9 +393,9 @@ test - org.apache.zookeeper - zookeeper - 3.4.5 + org.apache.curator + curator-recipes + 2.4.0 org.jboss.netty diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f0d2e741484f9..220894affb4da 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -277,7 +277,7 @@ object SparkBuild extends Build { "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib, excludeCommonsLogging, excludeSLF4J), "org.apache.avro" % "avro" % "1.7.4", "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty), - "org.apache.zookeeper" % "zookeeper" % "3.4.5" excludeAll(excludeNetty), + "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", From b8a1871953058c67b49b7f8455cbb417d5b50ab6 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Wed, 26 Feb 2014 10:00:02 -0600 Subject: [PATCH 004/397] SPARK-1053. Don't require SPARK_YARN_APP_JAR It looks this just requires taking out the checks. I verified that, with the patch, I was able to run spark-shell through yarn without setting the environment variable. Author: Sandy Ryza Closes #553 from sryza/sandy-spark-1053 and squashes the following commits: b037676 [Sandy Ryza] SPARK-1053. Don't require SPARK_YARN_APP_JAR --- docs/running-on-yarn.md | 6 ++---- .../org/apache/spark/deploy/yarn/ClientArguments.scala | 4 ++-- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 3 ++- .../scheduler/cluster/YarnClientSchedulerBackend.scala | 6 +----- 4 files changed, 7 insertions(+), 12 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index cd4509ede735a..ee1d892a3b630 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -99,13 +99,12 @@ With this mode, your application is actually run on the remote machine where the ## Launch spark application with yarn-client mode. -With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR and SPARK_YARN_APP_JAR +With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR. Configuration in yarn-client mode: In order to tune worker core/number/memory etc. You need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. -* `SPARK_YARN_APP_JAR`, Path to your application's JAR file (required) * `SPARK_WORKER_INSTANCES`, Number of workers to start (Default: 2) * `SPARK_WORKER_CORES`, Number of cores for the workers (Default: 1). * `SPARK_WORKER_MEMORY`, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) @@ -118,12 +117,11 @@ In order to tune worker core/number/memory etc. You need to export environment v For example: SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ ./bin/run-example org.apache.spark.examples.SparkPi yarn-client +or SPARK_JAR=./assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop2.0.5-alpha.jar \ - SPARK_YARN_APP_JAR=examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ MASTER=yarn-client ./bin/spark-shell diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 1419f215c78e5..fe37168e5a7ba 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -108,7 +108,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { args = tail case Nil => - if (userJar == null || userClass == null) { + if (userClass == null) { printUsageAndExit(1) } @@ -129,7 +129,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { System.err.println( "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required)\n" + + " --jar JAR_PATH Path to your application's JAR file (required in yarn-standalone mode)\n" + " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 2db5744be1a70..24520bd21ba98 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -68,7 +68,8 @@ trait ClientBase extends Logging { def validateArgs() = { Map( (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", - (args.userJar == null) -> "Error: You must specify a user jar!", + ((args.userJar == null && args.amClass == classOf[ApplicationMaster].getName) -> + "Error: You must specify a user jar when running in standalone mode!"), (args.userClass == null) -> "Error: You must specify a user class!", (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!", (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" + diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 22e55e0c60647..e7130d24072ca 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -44,10 +44,6 @@ private[spark] class YarnClientSchedulerBackend( override def start() { super.start() - val userJar = System.getenv("SPARK_YARN_APP_JAR") - if (userJar == null) - throw new SparkException("env SPARK_YARN_APP_JAR is not set") - val driverHost = conf.get("spark.driver.host") val driverPort = conf.get("spark.driver.port") val hostport = driverHost + ":" + driverPort @@ -55,7 +51,7 @@ private[spark] class YarnClientSchedulerBackend( val argsArrayBuf = new ArrayBuffer[String]() argsArrayBuf += ( "--class", "notused", - "--jar", userJar, + "--jar", null, "--args", hostport, "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" ) From fbedc8eff2573b31320ad1b11a1826a2e530c16c Mon Sep 17 00:00:00 2001 From: William Benton Date: Wed, 26 Feb 2014 10:09:50 -0800 Subject: [PATCH 005/397] SPARK-1078: Replace lift-json with json4s-jackson. The aim of the Json4s project is to provide a common API for Scala JSON libraries. It is Apache-licensed, easier for downstream distributions to package, and mostly API-compatible with lift-json. Furthermore, the Jackson-backed implementation parses faster than lift-json on all but the smallest inputs. Author: William Benton Closes #582 from willb/json4s and squashes the following commits: 7ca62c4 [William Benton] Replace lift-json with json4s-jackson. --- core/pom.xml | 5 ++-- .../spark/deploy/FaultToleranceTest.scala | 9 ++++--- .../apache/spark/deploy/JsonProtocol.scala | 2 +- .../deploy/master/ui/ApplicationPage.scala | 3 ++- .../spark/deploy/master/ui/IndexPage.scala | 3 ++- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- .../org/apache/spark/ui/JettyUtils.scala | 3 ++- .../spark/deploy/JsonProtocolSuite.scala | 27 ++++++++++--------- project/SparkBuild.scala | 2 +- 9 files changed, 32 insertions(+), 24 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index f209704f319de..ebc178a10541a 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -130,8 +130,9 @@ scala-library - net.liftweb - lift-json_${scala.binary.version} + org.json4s + json4s-jackson_${scala.binary.version} + 3.2.6 it.unimi.dsi diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index 190b331cfe7d8..d48c1892aea9c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -27,7 +27,8 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ import scala.sys.process._ -import net.liftweb.json.JsonParser +import org.json4s._ +import org.json4s.jackson.JsonMethods import org.apache.spark.{Logging, SparkContext} import org.apache.spark.deploy.master.RecoveryState @@ -311,7 +312,7 @@ private[spark] object FaultToleranceTest extends App with Logging { private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { - implicit val formats = net.liftweb.json.DefaultFormats + implicit val formats = org.json4s.DefaultFormats var state: RecoveryState.Value = _ var liveWorkerIPs: List[String] = _ var numLiveApps = 0 @@ -321,7 +322,7 @@ private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val def readState() { try { val masterStream = new InputStreamReader(new URL("http://%s:8080/json".format(ip)).openStream) - val json = JsonParser.parse(masterStream, closeAutomatically = true) + val json = JsonMethods.parse(masterStream) val workers = json \ "workers" val liveWorkers = workers.children.filter(w => (w \ "state").extract[String] == "ALIVE") @@ -349,7 +350,7 @@ private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val private[spark] class TestWorkerInfo(val ip: String, val dockerId: DockerId, val logFile: File) extends Logging { - implicit val formats = net.liftweb.json.DefaultFormats + implicit val formats = org.json4s.DefaultFormats logDebug("Created worker: " + this) diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index 318beb5db5214..cefb1ff97e83c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import net.liftweb.json.JsonDSL._ +import org.json4s.JsonDSL._ import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 5cc4adbe448b7..90cad3c37fda6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -23,7 +23,8 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import net.liftweb.json.JsonAST.JValue +import javax.servlet.http.HttpServletRequest +import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 01c8f9065e50a..bac922bcd3d98 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -23,7 +23,8 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import net.liftweb.json.JsonAST.JValue +import javax.servlet.http.HttpServletRequest +import org.json4s.JValue import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 3089acffb8d98..85200ab0e102d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -22,7 +22,7 @@ import scala.xml.Node import akka.pattern.ask import javax.servlet.http.HttpServletRequest -import net.liftweb.json.JsonAST.JValue +import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 1f048a84cdfb6..1b78c52ff6077 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -24,7 +24,8 @@ import scala.annotation.tailrec import scala.util.{Failure, Success, Try} import scala.xml.Node -import net.liftweb.json.{JValue, pretty, render} +import org.json4s.JValue +import org.json4s.jackson.JsonMethods.{pretty, render} import org.eclipse.jetty.server.{Handler, Request, Server} import org.eclipse.jetty.server.handler.{AbstractHandler, ContextHandler, HandlerList, ResourceHandler} import org.eclipse.jetty.util.thread.QueuedThreadPool diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index de866ed7ffed8..bae3b37e267d5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -20,9 +20,12 @@ package org.apache.spark.deploy import java.io.File import java.util.Date -import net.liftweb.json.Diff -import net.liftweb.json.{JsonAST, JsonParser} -import net.liftweb.json.JsonAST.{JNothing, JValue} +import org.json4s._ + +import org.json4s.JValue +import org.json4s.jackson.JsonMethods +import com.fasterxml.jackson.core.JsonParseException + import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} @@ -34,31 +37,31 @@ class JsonProtocolSuite extends FunSuite { test("writeApplicationInfo") { val output = JsonProtocol.writeApplicationInfo(createAppInfo()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.appInfoJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.appInfoJsonStr)) } test("writeWorkerInfo") { val output = JsonProtocol.writeWorkerInfo(createWorkerInfo()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.workerInfoJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.workerInfoJsonStr)) } test("writeApplicationDescription") { val output = JsonProtocol.writeApplicationDescription(createAppDesc()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.appDescJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.appDescJsonStr)) } test("writeExecutorRunner") { val output = JsonProtocol.writeExecutorRunner(createExecutorRunner()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.executorRunnerJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.executorRunnerJsonStr)) } test("writeDriverInfo") { val output = JsonProtocol.writeDriverInfo(createDriverInfo()) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.driverInfoJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.driverInfoJsonStr)) } test("writeMasterState") { @@ -71,7 +74,7 @@ class JsonProtocolSuite extends FunSuite { activeDrivers, completedDrivers, RecoveryState.ALIVE) val output = JsonProtocol.writeMasterState(stateResponse) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.masterStateJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.masterStateJsonStr)) } test("writeWorkerState") { @@ -83,7 +86,7 @@ class JsonProtocolSuite extends FunSuite { finishedExecutors, drivers, finishedDrivers, "masterUrl", 4, 1234, 4, 1234, "masterWebUiUrl") val output = JsonProtocol.writeWorkerState(stateResponse) assertValidJson(output) - assertValidDataInJson(output, JsonParser.parse(JsonConstants.workerStateJsonStr)) + assertValidDataInJson(output, JsonMethods.parse(JsonConstants.workerStateJsonStr)) } def createAppDesc(): ApplicationDescription = { @@ -125,9 +128,9 @@ class JsonProtocolSuite extends FunSuite { def assertValidJson(json: JValue) { try { - JsonParser.parse(JsonAST.compactRender(json)) + JsonMethods.parse(JsonMethods.compact(json)) } catch { - case e: JsonParser.ParseException => fail("Invalid Json detected", e) + case e: JsonParseException => fail("Invalid Json detected", e) } } diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 220894affb4da..7d157160d164b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -268,7 +268,7 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", - "net.liftweb" %% "lift-json" % "2.5.1" excludeAll(excludeNetty), + "org.json4s" %% "json4s-jackson" % "3.2.6", "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", From c86eec584312072e73754a4f1cbe34d2e1968c77 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 26 Feb 2014 11:20:16 -0800 Subject: [PATCH 006/397] SPARK-1135: fix broken anchors in docs A recent PR that added Java vs Scala tabs for streaming also inadvertently added some bad code to a document.ready handler, breaking our other handler that manages scrolling to anchors correctly with the floating top bar. As a result the section title ended up always being hidden below the top bar. This removes the unnecessary JavaScript code. Author: Matei Zaharia Closes #3 from mateiz/doc-links and squashes the following commits: e2a3488 [Matei Zaharia] SPARK-1135: fix broken anchors in docs --- docs/js/main.js | 29 +---------------------------- 1 file changed, 1 insertion(+), 28 deletions(-) diff --git a/docs/js/main.js b/docs/js/main.js index 102699789a71a..0bd2286cced19 100755 --- a/docs/js/main.js +++ b/docs/js/main.js @@ -1,26 +1,3 @@ - -// From docs.scala-lang.org -function styleCode() { - if (typeof disableStyleCode != "undefined") { - return; - } - $(".codetabs pre code").parent().each(function() { - if (!$(this).hasClass("prettyprint")) { - var lang = $(this).parent().data("lang"); - if (lang == "python") { - lang = "py" - } - if (lang == "bash") { - lang = "bsh" - } - $(this).addClass("prettyprint lang-"+lang+" linenums"); - } - }); - console.log("runningPrettyPrint()") - prettyPrint(); -} - - function codeTabs() { var counter = 0; var langImages = { @@ -97,11 +74,7 @@ function viewSolution() { } -$(document).ready(function() { +$(function() { codeTabs(); viewSolution(); - $('#chapter-toc').toc({exclude: '', context: '.container'}); - $('#chapter-toc').prepend('

In This Chapter

'); - makeCollapsable($('#global-toc'), "", "global-toc", "Show Table of Contents"); - //styleCode(); }); From 12738c1aec136acd7f2e3e2f8f2b541db0890630 Mon Sep 17 00:00:00 2001 From: Bouke van der Bijl Date: Wed, 26 Feb 2014 14:50:37 -0800 Subject: [PATCH 007/397] SPARK-1115: Catch depickling errors This surroungs the complete worker code in a try/except block so we catch any error that arrives. An example would be the depickling failing for some reason @JoshRosen Author: Bouke van der Bijl Closes #644 from bouk/catch-depickling-errors and squashes the following commits: f0f67cc [Bouke van der Bijl] Lol indentation 0e4d504 [Bouke van der Bijl] Surround the complete python worker with the try block --- python/pyspark/worker.py | 48 ++++++++++++++++++++-------------------- 1 file changed, 24 insertions(+), 24 deletions(-) diff --git a/python/pyspark/worker.py b/python/pyspark/worker.py index 158646352039f..4c214ef359685 100644 --- a/python/pyspark/worker.py +++ b/python/pyspark/worker.py @@ -45,34 +45,34 @@ def report_times(outfile, boot, init, finish): def main(infile, outfile): - boot_time = time.time() - split_index = read_int(infile) - if split_index == -1: # for unit tests - return + try: + boot_time = time.time() + split_index = read_int(infile) + if split_index == -1: # for unit tests + return - # fetch name of workdir - spark_files_dir = utf8_deserializer.loads(infile) - SparkFiles._root_directory = spark_files_dir - SparkFiles._is_running_on_worker = True + # fetch name of workdir + spark_files_dir = utf8_deserializer.loads(infile) + SparkFiles._root_directory = spark_files_dir + SparkFiles._is_running_on_worker = True - # fetch names and values of broadcast variables - num_broadcast_variables = read_int(infile) - for _ in range(num_broadcast_variables): - bid = read_long(infile) - value = pickleSer._read_with_length(infile) - _broadcastRegistry[bid] = Broadcast(bid, value) + # fetch names and values of broadcast variables + num_broadcast_variables = read_int(infile) + for _ in range(num_broadcast_variables): + bid = read_long(infile) + value = pickleSer._read_with_length(infile) + _broadcastRegistry[bid] = Broadcast(bid, value) - # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH - sys.path.append(spark_files_dir) # *.py files that were added will be copied here - num_python_includes = read_int(infile) - for _ in range(num_python_includes): - filename = utf8_deserializer.loads(infile) - sys.path.append(os.path.join(spark_files_dir, filename)) + # fetch names of includes (*.zip and *.egg files) and construct PYTHONPATH + sys.path.append(spark_files_dir) # *.py files that were added will be copied here + num_python_includes = read_int(infile) + for _ in range(num_python_includes): + filename = utf8_deserializer.loads(infile) + sys.path.append(os.path.join(spark_files_dir, filename)) - command = pickleSer._read_with_length(infile) - (func, deserializer, serializer) = command - init_time = time.time() - try: + command = pickleSer._read_with_length(infile) + (func, deserializer, serializer) = command + init_time = time.time() iterator = deserializer.load_stream(infile) serializer.dump_stream(func(split_index, iterator), outfile) except Exception as e: From 84f7ca138165ca413897dada35c602676b0a614f Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 26 Feb 2014 16:52:26 -0800 Subject: [PATCH 008/397] Removed reference to incubation in README.md. Author: Reynold Xin Closes #1 from rxin/readme and squashes the following commits: b3a77cd [Reynold Xin] Removed reference to incubation in README.md. --- README.md | 17 +++-------------- 1 file changed, 3 insertions(+), 14 deletions(-) diff --git a/README.md b/README.md index c840a68f76b17..dc8135b9b8b51 100644 --- a/README.md +++ b/README.md @@ -1,12 +1,12 @@ # Apache Spark -Lightning-Fast Cluster Computing - +Lightning-Fast Cluster Computing - ## Online Documentation You can find the latest Spark documentation, including a programming -guide, on the project webpage at . +guide, on the project webpage at . This README file only contains basic setup instructions. @@ -92,21 +92,10 @@ If your project is built with Maven, add this to your POM file's ` ## Configuration -Please refer to the [Configuration guide](http://spark.incubator.apache.org/docs/latest/configuration.html) +Please refer to the [Configuration guide](http://spark.apache.org/docs/latest/configuration.html) in the online documentation for an overview on how to configure Spark. -## Apache Incubator Notice - -Apache Spark is an effort undergoing incubation at The Apache Software -Foundation (ASF), sponsored by the Apache Incubator. Incubation is required of -all newly accepted projects until a further review indicates that the -infrastructure, communications, and decision making process have stabilized in -a manner consistent with other successful ASF projects. While incubation status -is not necessarily a reflection of the completeness or stability of the code, -it does indicate that the project has yet to be fully endorsed by the ASF. - - ## Contributing to Spark Contributions via GitHub pull requests are gladly accepted from their original From 0e40e2b12692d37e35d75e8b495ecd181fbce7de Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 26 Feb 2014 21:17:44 -0800 Subject: [PATCH 009/397] Deprecated and added a few java api methods for corresponding scala api. PR [402](https://github.com/apache/incubator-spark/pull/402) from incubator repo. Author: Prashant Sharma Closes #19 from ScrapCodes/java-api-completeness and squashes the following commits: 11d0c2b [Prashant Sharma] Integer -> java.lang.Integer 737819a [Prashant Sharma] SPARK-1095 add explicit return types to APIs. 3ddc8bb [Prashant Sharma] Deprected *With functions in scala and added a few missing Java APIs --- .../scala/org/apache/spark/SparkContext.scala | 3 ++- .../org/apache/spark/api/java/JavaRDD.scala | 2 ++ .../apache/spark/api/java/JavaRDDLike.scala | 2 +- .../spark/api/java/JavaSparkContext.scala | 19 +++++++++++++++++++ .../main/scala/org/apache/spark/rdd/RDD.scala | 12 ++++++++---- 5 files changed, 32 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a24f07e9a6e9a..1f5334f3dbb40 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -240,6 +240,7 @@ class SparkContext( localProperties.set(props) } + @deprecated("Properties no longer need to be explicitly initialized.", "1.0.0") def initLocalProperties() { localProperties.set(new Properties()) } @@ -308,7 +309,7 @@ class SparkContext( private val dagSchedulerSource = new DAGSchedulerSource(this.dagScheduler, this) private val blockManagerSource = new BlockManagerSource(SparkEnv.get.blockManager, this) - def initDriverMetrics() { + private def initDriverMetrics() { SparkEnv.get.metricsSystem.registerSource(dagSchedulerSource) SparkEnv.get.metricsSystem.registerSource(blockManagerSource) } diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 0055c98844ded..d7ce8fdfc23f4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -126,6 +126,8 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] = wrapRDD(rdd.subtract(other, p)) + def generator: String = rdd.generator + override def toString = rdd.toString /** Assign a name to this RDD */ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 24a9925dbd22c..729668fb679b4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -74,7 +74,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * of the original partition. */ def mapPartitionsWithIndex[R: ClassTag]( - f: JFunction2[Int, java.util.Iterator[T], java.util.Iterator[R]], + f: JFunction2[java.lang.Integer, java.util.Iterator[T], java.util.Iterator[R]], preservesPartitioning: Boolean = false): JavaRDD[R] = new JavaRDD(rdd.mapPartitionsWithIndex(((a,b) => f(a,asJavaIterator(b))), preservesPartitioning)) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index dc26b7f621fee..8e0eab56a3dcf 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -17,6 +17,7 @@ package org.apache.spark.api.java +import java.util import java.util.{Map => JMap} import scala.collection.JavaConversions @@ -92,6 +93,24 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork private[spark] val env = sc.env + def isLocal: java.lang.Boolean = sc.isLocal + + def sparkUser: String = sc.sparkUser + + def master: String = sc.master + + def appName: String = sc.appName + + def jars: util.List[String] = sc.jars + + def startTime: java.lang.Long = sc.startTime + + /** Default level of parallelism to use when not given by user (e.g. parallelize and makeRDD). */ + def defaultParallelism: java.lang.Integer = sc.defaultParallelism + + /** Default min number of partitions for Hadoop RDDs when not given by user */ + def defaultMinSplits: java.lang.Integer = sc.defaultMinSplits + /** Distribute a local Scala collection to form an RDD. */ def parallelize[T](list: java.util.List[T], numSlices: Int): JavaRDD[T] = { implicit val ctag: ClassTag[T] = fakeClassTag diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 50320f40350cd..3fe56963e0008 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -543,7 +543,8 @@ abstract class RDD[T: ClassTag]( * additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def mapWith[A: ClassTag, U: ClassTag] + @deprecated("use mapPartitionsWithIndex", "1.0.0") + def mapWith[A, U: ClassTag] (constructA: Int => A, preservesPartitioning: Boolean = false) (f: (T, A) => U): RDD[U] = { mapPartitionsWithIndex((index, iter) => { @@ -557,7 +558,8 @@ abstract class RDD[T: ClassTag]( * additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def flatMapWith[A: ClassTag, U: ClassTag] + @deprecated("use mapPartitionsWithIndex and flatMap", "1.0.0") + def flatMapWith[A, U: ClassTag] (constructA: Int => A, preservesPartitioning: Boolean = false) (f: (T, A) => Seq[U]): RDD[U] = { mapPartitionsWithIndex((index, iter) => { @@ -571,7 +573,8 @@ abstract class RDD[T: ClassTag]( * This additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def foreachWith[A: ClassTag](constructA: Int => A)(f: (T, A) => Unit) { + @deprecated("use mapPartitionsWithIndex and foreach", "1.0.0") + def foreachWith[A](constructA: Int => A)(f: (T, A) => Unit) { mapPartitionsWithIndex { (index, iter) => val a = constructA(index) iter.map(t => {f(t, a); t}) @@ -583,7 +586,8 @@ abstract class RDD[T: ClassTag]( * additional parameter is produced by constructA, which is called in each * partition with the index of that partition. */ - def filterWith[A: ClassTag](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = { + @deprecated("use mapPartitionsWithIndex and filter", "1.0.0") + def filterWith[A](constructA: Int => A)(p: (T, A) => Boolean): RDD[T] = { mapPartitionsWithIndex((index, iter) => { val a = constructA(index) iter.filter(t => p(t, a)) From 26450351af8406704c6823ce88426bd3e579d4cb Mon Sep 17 00:00:00 2001 From: Jyotiska NK Date: Wed, 26 Feb 2014 21:37:04 -0800 Subject: [PATCH 010/397] Updated link for pyspark examples in docs Author: Jyotiska NK Closes #22 from jyotiska/pyspark_docs and squashes the following commits: 426136c [Jyotiska NK] Updated link for pyspark examples --- docs/python-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 7c5283fb0b6fb..57ed54c9cf4c0 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -157,7 +157,7 @@ some example applications. # Where to Go from Here -PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/incubator-spark/tree/master/python/examples). +PySpark also includes several sample programs in the [`python/examples` folder](https://github.com/apache/spark/tree/master/python/examples). You can run them by passing the files to `pyspark`; e.g.: ./bin/pyspark python/examples/wordcount.py From 71f69d66ce50991e99408791ade25a670598d32a Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 26 Feb 2014 22:52:42 -0800 Subject: [PATCH 011/397] Remove references to ClusterScheduler (SPARK-1140) ClusterScheduler was renamed to TaskSchedulerImpl; this commit updates comments and tests accordingly. Author: Kay Ousterhout Closes #9 from kayousterhout/cluster_scheduler_death and squashes the following commits: d6fd119 [Kay Ousterhout] Remove references to ClusterScheduler. --- .../spark/scheduler/SchedulerBackend.scala | 2 +- .../spark/scheduler/TaskScheduler.scala | 2 +- .../spark/scheduler/TaskSetManager.scala | 7 +-- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../spark/scheduler/local/LocalBackend.scala | 4 +- .../scheduler/TaskResultGetterSuite.scala | 6 +-- ...ite.scala => TaskSchedulerImplSuite.scala} | 50 +++++++++---------- .../spark/scheduler/TaskSetManagerSuite.scala | 20 ++++---- 8 files changed, 47 insertions(+), 46 deletions(-) rename core/src/test/scala/org/apache/spark/scheduler/{ClusterSchedulerSuite.scala => TaskSchedulerImplSuite.scala} (79%) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index eefc8c232b564..f1924a4573b21 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler /** * A backend interface for scheduling systems that allows plugging in different ones under - * ClusterScheduler. We assume a Mesos-like model where the application gets resource offers as + * TaskSchedulerImpl. We assume a Mesos-like model where the application gets resource offers as * machines become available and can launch tasks on them. */ private[spark] trait SchedulerBackend { diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 1cdfed1d7005e..92616c997e20c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -20,7 +20,7 @@ package org.apache.spark.scheduler import org.apache.spark.scheduler.SchedulingMode.SchedulingMode /** - * Low-level task scheduler interface, currently implemented exclusively by the ClusterScheduler. + * Low-level task scheduler interface, currently implemented exclusively by TaskSchedulerImpl. * This interface allows plugging in different task schedulers. Each TaskScheduler schedulers tasks * for a single SparkContext. These schedulers get sets of tasks submitted to them from the * DAGScheduler for each stage, and are responsible for sending the tasks to the cluster, running diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 1a4b7e599c01e..5ea4557bbf56a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -26,13 +26,14 @@ import scala.collection.mutable.HashSet import scala.math.max import scala.math.min -import org.apache.spark.{ExceptionFailure, ExecutorLostFailure, FetchFailed, Logging, Resubmitted, SparkEnv, Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} +import org.apache.spark.{ExceptionFailure, ExecutorLostFailure, FetchFailed, Logging, Resubmitted, + SparkEnv, Success, TaskEndReason, TaskKilled, TaskResultLost, TaskState} import org.apache.spark.TaskState.TaskState import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.{Clock, SystemClock} /** - * Schedules the tasks within a single TaskSet in the ClusterScheduler. This class keeps track of + * Schedules the tasks within a single TaskSet in the TaskSchedulerImpl. This class keeps track of * each task, retries tasks if they fail (up to a limited number of times), and * handles locality-aware scheduling for this TaskSet via delay scheduling. The main interfaces * to it are resourceOffer, which asks the TaskSet whether it wants to run a task on one node, @@ -41,7 +42,7 @@ import org.apache.spark.util.{Clock, SystemClock} * THREADING: This class is designed to only be called from code with a lock on the * TaskScheduler (e.g. its event handlers). It should not be called from other threads. * - * @param sched the ClusterScheduler associated with the TaskSetManager + * @param sched the TaskSchedulerImpl associated with the TaskSetManager * @param taskSet the TaskSet to manage scheduling for * @param maxTaskFailures if any particular task fails more than this number of times, the entire * task set will be aborted diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index c576beb0c0d38..bcf0ce19a54cd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -203,7 +203,7 @@ private[spark] class MesosSchedulerBackend( getResource(offer.getResourcesList, "cpus").toInt) } - // Call into the ClusterScheduler + // Call into the TaskSchedulerImpl val taskLists = scheduler.resourceOffers(offerableWorkers) // Build a list of Mesos tasks for each slave diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 50f7e79e97dd8..16e2f5cf3076d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -35,7 +35,7 @@ private case class KillTask(taskId: Long) /** * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on * LocalBackend asynchronous, which is necessary to prevent deadlock between LocalBackend - * and the ClusterScheduler. + * and the TaskSchedulerImpl. */ private[spark] class LocalActor( scheduler: TaskSchedulerImpl, @@ -76,7 +76,7 @@ private[spark] class LocalActor( /** * LocalBackend is used when running a local version of Spark where the executor, backend, and - * master all run in the same JVM. It sits behind a ClusterScheduler and handles launching tasks + * master all run in the same JVM. It sits behind a TaskSchedulerImpl and handles launching tasks * on a single Executor (created by the LocalBackend) running locally. */ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: Int) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala index ac07f60e284bb..c4e7a4bb7d385 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskResultGetterSuite.scala @@ -93,10 +93,10 @@ class TaskResultGetterSuite extends FunSuite with BeforeAndAfter with BeforeAndA // If this test hangs, it's probably because no resource offers were made after the task // failed. val scheduler: TaskSchedulerImpl = sc.taskScheduler match { - case clusterScheduler: TaskSchedulerImpl => - clusterScheduler + case taskScheduler: TaskSchedulerImpl => + taskScheduler case _ => - assert(false, "Expect local cluster to use ClusterScheduler") + assert(false, "Expect local cluster to use TaskSchedulerImpl") throw new ClassCastException } scheduler.taskResultGetter = new ResultDeletingTaskResultGetter(sc.env, scheduler) diff --git a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala similarity index 79% rename from core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala rename to core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 85e929925e3b5..f4e62c64daf12 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/ClusterSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -29,9 +29,9 @@ class FakeTaskSetManager( initPriority: Int, initStageId: Int, initNumTasks: Int, - clusterScheduler: TaskSchedulerImpl, + taskScheduler: TaskSchedulerImpl, taskSet: TaskSet) - extends TaskSetManager(clusterScheduler, taskSet, 0) { + extends TaskSetManager(taskScheduler, taskSet, 0) { parent = null weight = 1 @@ -105,7 +105,7 @@ class FakeTaskSetManager( } } -class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging { +class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Logging { def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskSchedulerImpl, taskSet: TaskSet): FakeTaskSetManager = { new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet) @@ -133,8 +133,8 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging } test("FIFO Scheduler Test") { - sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new TaskSchedulerImpl(sc) + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) var tasks = ArrayBuffer[Task[_]]() val task = new FakeTask(0) tasks += task @@ -144,9 +144,9 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) schedulableBuilder.buildPools() - val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, clusterScheduler, taskSet) - val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, clusterScheduler, taskSet) - val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, clusterScheduler, taskSet) + val taskSetManager0 = createDummyTaskSetManager(0, 0, 2, taskScheduler, taskSet) + val taskSetManager1 = createDummyTaskSetManager(0, 1, 2, taskScheduler, taskSet) + val taskSetManager2 = createDummyTaskSetManager(0, 2, 2, taskScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager0, null) schedulableBuilder.addTaskSetManager(taskSetManager1, null) schedulableBuilder.addTaskSetManager(taskSetManager2, null) @@ -160,8 +160,8 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging } test("Fair Scheduler Test") { - sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new TaskSchedulerImpl(sc) + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) var tasks = ArrayBuffer[Task[_]]() val task = new FakeTask(0) tasks += task @@ -189,15 +189,15 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging val properties2 = new Properties() properties2.setProperty("spark.scheduler.pool","2") - val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, clusterScheduler, taskSet) - val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, clusterScheduler, taskSet) - val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, clusterScheduler, taskSet) + val taskSetManager10 = createDummyTaskSetManager(1, 0, 1, taskScheduler, taskSet) + val taskSetManager11 = createDummyTaskSetManager(1, 1, 1, taskScheduler, taskSet) + val taskSetManager12 = createDummyTaskSetManager(1, 2, 2, taskScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager10, properties1) schedulableBuilder.addTaskSetManager(taskSetManager11, properties1) schedulableBuilder.addTaskSetManager(taskSetManager12, properties1) - val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, clusterScheduler, taskSet) - val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, clusterScheduler, taskSet) + val taskSetManager23 = createDummyTaskSetManager(2, 3, 2, taskScheduler, taskSet) + val taskSetManager24 = createDummyTaskSetManager(2, 4, 2, taskScheduler, taskSet) schedulableBuilder.addTaskSetManager(taskSetManager23, properties2) schedulableBuilder.addTaskSetManager(taskSetManager24, properties2) @@ -217,8 +217,8 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging } test("Nested Pool Test") { - sc = new SparkContext("local", "ClusterSchedulerSuite") - val clusterScheduler = new TaskSchedulerImpl(sc) + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) var tasks = ArrayBuffer[Task[_]]() val task = new FakeTask(0) tasks += task @@ -240,23 +240,23 @@ class ClusterSchedulerSuite extends FunSuite with LocalSparkContext with Logging pool1.addSchedulable(pool10) pool1.addSchedulable(pool11) - val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, clusterScheduler, taskSet) - val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, clusterScheduler, taskSet) + val taskSetManager000 = createDummyTaskSetManager(0, 0, 5, taskScheduler, taskSet) + val taskSetManager001 = createDummyTaskSetManager(0, 1, 5, taskScheduler, taskSet) pool00.addSchedulable(taskSetManager000) pool00.addSchedulable(taskSetManager001) - val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, clusterScheduler, taskSet) - val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, clusterScheduler, taskSet) + val taskSetManager010 = createDummyTaskSetManager(1, 2, 5, taskScheduler, taskSet) + val taskSetManager011 = createDummyTaskSetManager(1, 3, 5, taskScheduler, taskSet) pool01.addSchedulable(taskSetManager010) pool01.addSchedulable(taskSetManager011) - val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, clusterScheduler, taskSet) - val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, clusterScheduler, taskSet) + val taskSetManager100 = createDummyTaskSetManager(2, 4, 5, taskScheduler, taskSet) + val taskSetManager101 = createDummyTaskSetManager(2, 5, 5, taskScheduler, taskSet) pool10.addSchedulable(taskSetManager100) pool10.addSchedulable(taskSetManager101) - val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, clusterScheduler, taskSet) - val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, clusterScheduler, taskSet) + val taskSetManager110 = createDummyTaskSetManager(3, 6, 5, taskScheduler, taskSet) + val taskSetManager111 = createDummyTaskSetManager(3, 7, 5, taskScheduler, taskSet) pool11.addSchedulable(taskSetManager110) pool11.addSchedulable(taskSetManager111) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 34a7d8cefeea2..20f6e503872ac 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.FakeClock -class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler(taskScheduler) { +class FakeDAGScheduler(taskScheduler: FakeTaskScheduler) extends DAGScheduler(taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo) { taskScheduler.startedTasks += taskInfo.index } @@ -51,12 +51,12 @@ class FakeDAGScheduler(taskScheduler: FakeClusterScheduler) extends DAGScheduler } /** - * A mock ClusterScheduler implementation that just remembers information about tasks started and + * A mock TaskSchedulerImpl implementation that just remembers information about tasks started and * feedback received from the TaskSetManagers. Note that it's important to initialize this with * a list of "live" executors and their hostnames for isExecutorAlive and hasExecutorsAliveOnHost * to work, and these are required for locality in TaskSetManager. */ -class FakeClusterScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */) +class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* execId, host */) extends TaskSchedulerImpl(sc) { val startedTasks = new ArrayBuffer[Long] @@ -87,7 +87,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("TaskSet with no preferences") { sc = new SparkContext("local", "test") - val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) + val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) @@ -113,7 +113,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("multiple offers with no preferences") { sc = new SparkContext("local", "test") - val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) + val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) @@ -144,7 +144,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("basic delay scheduling") { sc = new SparkContext("local", "test") - val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = createTaskSet(4, Seq(TaskLocation("host1", "exec1")), Seq(TaskLocation("host2", "exec2")), @@ -188,7 +188,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("delay scheduling with fallback") { sc = new SparkContext("local", "test") - val sched = new FakeClusterScheduler(sc, + val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3")) val taskSet = createTaskSet(5, Seq(TaskLocation("host1")), @@ -228,7 +228,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("delay scheduling with failed hosts") { sc = new SparkContext("local", "test") - val sched = new FakeClusterScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) + val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) val taskSet = createTaskSet(3, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), @@ -260,7 +260,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("task result lost") { sc = new SparkContext("local", "test") - val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) + val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(1) val clock = new FakeClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) @@ -277,7 +277,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("repeated failures lead to task set abortion") { sc = new SparkContext("local", "test") - val sched = new FakeClusterScheduler(sc, ("exec1", "host1")) + val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) val taskSet = createTaskSet(1) val clock = new FakeClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) From 5a3ad107c01dbc09cd144376ca808125ee821093 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Feb 2014 23:22:30 -0800 Subject: [PATCH 012/397] SPARK-1129: use a predefined seed when seed is zero in XORShiftRandom If the seed is zero, XORShift generates all zeros, which would create unexpected result. JIRA: https://spark-project.atlassian.net/browse/SPARK-1129 Author: Xiangrui Meng Closes #645 from mengxr/xor and squashes the following commits: 1b086ab [Xiangrui Meng] use MurmurHash3 to set seed in XORShiftRandom 45c6f16 [Xiangrui Meng] minor style change 51f4050 [Xiangrui Meng] use a predefined seed when seed is zero in XORShiftRandom --- .../apache/spark/util/random/XORShiftRandom.scala | 15 ++++++++++++--- .../spark/util/random/XORShiftRandomSuite.scala | 4 ++++ 2 files changed, 16 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index ca611b67ed91d..8a4cdea2fa7b1 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -17,8 +17,11 @@ package org.apache.spark.util.random +import java.nio.ByteBuffer import java.util.{Random => JavaRandom} +import scala.util.hashing.MurmurHash3 + import org.apache.spark.util.Utils.timeIt /** @@ -36,8 +39,8 @@ private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) { def this() = this(System.nanoTime) - private var seed = init - + private var seed = XORShiftRandom.hashSeed(init) + // we need to just override next - this will be called by nextInt, nextDouble, // nextGaussian, nextLong, etc. override protected def next(bits: Int): Int = { @@ -49,13 +52,19 @@ private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) { } override def setSeed(s: Long) { - seed = s + seed = XORShiftRandom.hashSeed(s) } } /** Contains benchmark method and main method to run benchmark of the RNG */ private[spark] object XORShiftRandom { + /** Hash seeds to have 0/1 bits throughout. */ + private def hashSeed(seed: Long): Long = { + val bytes = ByteBuffer.allocate(java.lang.Long.SIZE).putLong(seed).array() + MurmurHash3.bytesHash(bytes) + } + /** * Main method for running benchmark * @param args takes one argument - the number of random numbers to generate diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index c51d12bfe0bc6..757476efdb789 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -72,4 +72,8 @@ class XORShiftRandomSuite extends FunSuite with ShouldMatchers { } + test ("XORShift with zero seed") { + val random = new XORShiftRandom(0L) + assert(random.nextInt() != 0) + } } From 6ccd6c55bdfcf1a4f8f8fd1a9715d4a45ec1703a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 26 Feb 2014 23:40:49 -0800 Subject: [PATCH 013/397] SPARK-1121 Only add avro if the build is for Hadoop 0.23.X and SPARK_YARN is set Author: Prashant Sharma Closes #6 from ScrapCodes/SPARK-1121/avro-dep-fix and squashes the following commits: 9b29e34 [Prashant Sharma] Review feedback on PR 46ed2ad [Prashant Sharma] SPARK-1121-Only add avro if the build is for Hadoop 0.23.X and SPARK_YARN is set --- docs/building-with-maven.md | 4 +++ pom.xml | 21 ----------- project/SparkBuild.scala | 69 +++++++++++++++++++------------------ 3 files changed, 39 insertions(+), 55 deletions(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index ded12926885b9..40cac8eb4f0db 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -76,3 +76,7 @@ The maven build includes support for building a Debian package containing the as $ mvn -Pdeb -DskipTests clean package The debian package can then be found under assembly/target. We added the short commit hash to the file name so that we can distinguish individual packages built for SNAPSHOT versions. + +## A note about Hadoop version 0.23.x + +For building spark with hadoop 0.23.x and also yarn, you will have to manually add a dependency on avro (org.apache.avro, avro, 1.7.4). diff --git a/pom.xml b/pom.xml index 4f1e8398d9b8d..6adc670462b25 100644 --- a/pom.xml +++ b/pom.xml @@ -505,27 +505,6 @@
- - - org.apache.avro - avro - 1.7.4 - - - org.apache.avro - avro-ipc - 1.7.4 - - - org.jboss.netty - netty - - - io.netty - netty - - - diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7d157160d164b..c402415742b5e 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -87,7 +87,7 @@ object SparkBuild extends Build { case Some(v) => v.toBoolean } lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" - + val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() // Conditionally include the yarn sub-project lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core) lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core) @@ -130,6 +130,8 @@ object SparkBuild extends Build { javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, + // This is to add convenience of enabling sbt -Dsbt.offline=true for making the build offline. + offline := "true".equalsIgnoreCase(sys.props("sbt.offline")), retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", transitiveClassifiers in Scope.GlobalScope := Seq("sources"), testListeners <<= target.map(t => Seq(new eu.henkelmann.sbt.JUnitXmlTestsListener(t.getAbsolutePath))), @@ -254,39 +256,38 @@ object SparkBuild extends Build { ), libraryDependencies ++= Seq( - "com.google.guava" % "guava" % "14.0.1", - "com.google.code.findbugs" % "jsr305" % "1.3.9", - "log4j" % "log4j" % "1.2.17", - "org.slf4j" % "slf4j-api" % slf4jVersion, - "org.slf4j" % "slf4j-log4j12" % slf4jVersion, - "org.slf4j" % "jul-to-slf4j" % slf4jVersion, - "org.slf4j" % "jcl-over-slf4j" % slf4jVersion, - "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 - "com.ning" % "compress-lzf" % "1.0.0", - "org.xerial.snappy" % "snappy-java" % "1.0.5", - "org.ow2.asm" % "asm" % "4.0", - "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", - "org.json4s" %% "json4s-jackson" % "3.2.6", - "it.unimi.dsi" % "fastutil" % "6.4.4", - "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.13.0", - "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), - "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib, excludeCommonsLogging, excludeSLF4J), - "org.apache.avro" % "avro" % "1.7.4", - "org.apache.avro" % "avro-ipc" % "1.7.4" excludeAll(excludeNetty), - "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), - "com.codahale.metrics" % "metrics-core" % "3.0.0", - "com.codahale.metrics" % "metrics-jvm" % "3.0.0", - "com.codahale.metrics" % "metrics-json" % "3.0.0", - "com.codahale.metrics" % "metrics-ganglia" % "3.0.0", - "com.codahale.metrics" % "metrics-graphite" % "3.0.0", - "com.twitter" %% "chill" % "0.3.1", - "com.twitter" % "chill-java" % "0.3.1", - "com.clearspring.analytics" % "stream" % "2.5.1" - ) + "com.google.guava" % "guava" % "14.0.1", + "com.google.code.findbugs" % "jsr305" % "1.3.9", + "log4j" % "log4j" % "1.2.17", + "org.slf4j" % "slf4j-api" % slf4jVersion, + "org.slf4j" % "slf4j-log4j12" % slf4jVersion, + "org.slf4j" % "jul-to-slf4j" % slf4jVersion, + "org.slf4j" % "jcl-over-slf4j" % slf4jVersion, + "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 + "com.ning" % "compress-lzf" % "1.0.0", + "org.xerial.snappy" % "snappy-java" % "1.0.5", + "org.ow2.asm" % "asm" % "4.0", + "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", + "org.json4s" %% "json4s-jackson" % "3.2.6", + "it.unimi.dsi" % "fastutil" % "6.4.4", + "colt" % "colt" % "1.2.0", + "org.apache.mesos" % "mesos" % "0.13.0", + "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), + "org.apache.derby" % "derby" % "10.4.2.0" % "test", + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib, excludeCommonsLogging, excludeSLF4J), + "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), + "com.codahale.metrics" % "metrics-core" % "3.0.0", + "com.codahale.metrics" % "metrics-jvm" % "3.0.0", + "com.codahale.metrics" % "metrics-json" % "3.0.0", + "com.codahale.metrics" % "metrics-ganglia" % "3.0.0", + "com.codahale.metrics" % "metrics-graphite" % "3.0.0", + "com.twitter" %% "chill" % "0.3.1", + "com.twitter" % "chill-java" % "0.3.1", + "com.clearspring.analytics" % "stream" % "2.5.1" + ), + libraryDependencies ++= maybeAvro ) def rootSettings = sharedSettings ++ Seq( From 345df5f4a9c16a6a87440afa2b09082fc3d224bd Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 26 Feb 2014 23:42:15 -0800 Subject: [PATCH 014/397] [SPARK-1089] fix the regression problem on ADD_JARS in 0.9 https://spark-project.atlassian.net/browse/SPARK-1089 copied from JIRA, reported by @ash211 "Using the ADD_JARS environment variable with spark-shell used to add the jar to both the shell and the various workers. Now it only adds to the workers and importing a custom class in the shell is broken. The workaround is to add custom jars to both ADD_JARS and SPARK_CLASSPATH. We should fix ADD_JARS so it works properly again. See various threads on the user list: https://mail-archives.apache.org/mod_mbox/incubator-spark-user/201402.mbox/%3CCAJbo4neMLiTrnm1XbyqomWmp0m+EUcg4yE-txuRGSVKOb5KLeA@mail.gmail.com%3E (another one that doesn't appear in the archives yet titled "ADD_JARS not working on 0.9")" The reason of this bug is two-folds in the current implementation of SparkILoop.scala, the settings.classpath is not set properly when the process() method is invoked the weird behaviour of Scala 2.10, (I personally thought it is a bug) if we simply set value of a PathSettings object (like settings.classpath), the isDefault is not set to true (this is a flag showing if the variable is modified), so it makes the PathResolver loads the default CLASSPATH environment variable value to calculated the path (see https://github.com/scala/scala/blob/2.10.x/src/compiler/scala/tools/util/PathResolver.scala#L215) what we have to do is to manually make this flag set, (https://github.com/CodingCat/incubator-spark/blob/e3991d97ddc33e77645e4559b13bf78b9e68239a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala#L884) Author: CodingCat Closes #13 from CodingCat/SPARK-1089 and squashes the following commits: 8af81e7 [CodingCat] impose non-null settings 9aa2125 [CodingCat] code cleaning ce36676 [CodingCat] code cleaning e045582 [CodingCat] fix the regression problem on ADD_JARS in 0.9 --- .../main/scala/org/apache/spark/repl/SparkILoop.scala | 9 +++++++-- 1 file changed, 7 insertions(+), 2 deletions(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 013cea07d48fd..f52ebe4a159f1 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -182,8 +182,13 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, /** Create a new interpreter. */ def createInterpreter() { - if (addedClasspath != "") - settings.classpath append addedClasspath + require(settings != null) + + if (addedClasspath != "") settings.classpath.append(addedClasspath) + // work around for Scala bug + val totalClassPath = SparkILoop.getAddedJars.foldLeft( + settings.classpath.value)((l, r) => ClassPath.join(l, r)) + this.settings.classpath.value = totalClassPath intp = new SparkILoopInterpreter } From aace2c097ed2ca8bca33a3a3f07fb8bf772b3c50 Mon Sep 17 00:00:00 2001 From: Raymond Liu Date: Wed, 26 Feb 2014 23:51:32 -0800 Subject: [PATCH 015/397] Show Master status on UI page For standalone HA mode, A status is useful to identify the current master, already in json format too. Author: Raymond Liu Closes #24 from colorant/status and squashes the following commits: df630b3 [Raymond Liu] Show Master status on UI page --- .../main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index bac922bcd3d98..3233cd97f7bd0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -86,6 +86,7 @@ private[spark] class IndexPage(parent: MasterWebUI) {
  • Drivers: {state.activeDrivers.size} Running, {state.completedDrivers.size} Completed
  • +
  • Status: {state.status}
  • From 12bbca20657c17d5ebfceaacb37dddc851772675 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 27 Feb 2014 11:12:21 -0800 Subject: [PATCH 016/397] SPARK 1084.1 (resubmitted) (Ported from https://github.com/apache/incubator-spark/pull/637 ) Author: Sean Owen Closes #31 from srowen/SPARK-1084.1 and squashes the following commits: 6c4a32c [Sean Owen] Suppress warnings about legitimate unchecked array creations, or change code to avoid it f35b833 [Sean Owen] Fix two misc javadoc problems 254e8ef [Sean Owen] Fix one new style error introduced in scaladoc warning commit 5b2fce2 [Sean Owen] Fix scaladoc invocation warning, and enable javac warnings properly, with plugin config updates 007762b [Sean Owen] Remove dead scaladoc links b8ff8cb [Sean Owen] Replace deprecated Ant with --- .../scala/org/apache/spark/bagel/Bagel.scala | 14 +- core/pom.xml | 4 +- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../apache/spark/scheduler/JobLogger.scala | 2 +- .../util/IndestructibleActorSystem.scala | 4 +- .../org/apache/spark/util/StatCounter.scala | 6 +- .../scala/org/apache/spark/util/Vector.scala | 2 +- .../java/org/apache/spark/JavaAPISuite.java | 35 +++-- pom.xml | 5 +- repl/pom.xml | 4 +- .../streaming/api/java/JavaPairDStream.scala | 16 +-- .../api/java/JavaStreamingContext.scala | 4 +- .../dstream/PairDStreamFunctions.scala | 16 +-- .../apache/spark/streaming/JavaAPISuite.java | 124 ++++++++++++------ yarn/pom.xml | 4 +- 15 files changed, 154 insertions(+), 88 deletions(-) diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index dd3eed8affe39..70c7474a936dc 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -27,7 +27,7 @@ object Bagel extends Logging { /** * Runs a Bagel program. - * @param sc [[org.apache.spark.SparkContext]] to use for the program. + * @param sc org.apache.spark.SparkContext to use for the program. * @param vertices vertices of the graph represented as an RDD of (Key, Vertex) pairs. Often the * Key will be the vertex id. * @param messages initial set of messages represented as an RDD of (Key, Message) pairs. Often @@ -38,10 +38,10 @@ object Bagel extends Logging { * @param aggregator [[org.apache.spark.bagel.Aggregator]] performs a reduce across all vertices * after each superstep and provides the result to each vertex in the next * superstep. - * @param partitioner [[org.apache.spark.Partitioner]] partitions values by key + * @param partitioner org.apache.spark.Partitioner partitions values by key * @param numPartitions number of partitions across which to split the graph. * Default is the default parallelism of the SparkContext - * @param storageLevel [[org.apache.spark.storage.StorageLevel]] to use for caching of + * @param storageLevel org.apache.spark.storage.StorageLevel to use for caching of * intermediate RDDs in each superstep. Defaults to caching in memory. * @param compute function that takes a Vertex, optional set of (possibly combined) messages to * the Vertex, optional Aggregator and the current superstep, @@ -131,7 +131,7 @@ object Bagel extends Logging { /** * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], default - * [[org.apache.spark.HashPartitioner]] and default storage level + * org.apache.spark.HashPartitioner and default storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, @@ -146,7 +146,7 @@ object Bagel extends Logging { /** * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]] and the - * default [[org.apache.spark.HashPartitioner]] + * default org.apache.spark.HashPartitioner */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest, C: Manifest]( sc: SparkContext, @@ -166,7 +166,7 @@ object Bagel extends Logging { /** * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], - * default [[org.apache.spark.HashPartitioner]], + * default org.apache.spark.HashPartitioner, * [[org.apache.spark.bagel.DefaultCombiner]] and the default storage level */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( @@ -180,7 +180,7 @@ object Bagel extends Logging { /** * Runs a Bagel program with no [[org.apache.spark.bagel.Aggregator]], - * the default [[org.apache.spark.HashPartitioner]] + * the default org.apache.spark.HashPartitioner * and [[org.apache.spark.bagel.DefaultCombiner]] */ def run[K: Manifest, V <: Vertex : Manifest, M <: Message[K] : Manifest]( diff --git a/core/pom.xml b/core/pom.xml index ebc178a10541a..a333bff28c246 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -225,7 +225,7 @@ true - + @@ -238,7 +238,7 @@ - + diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 1f5334f3dbb40..da778aa851cd2 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -351,7 +351,7 @@ class SparkContext( * using the older MapReduce API (`org.apache.hadoop.mapred`). * * @param conf JobConf for setting up the dataset - * @param inputFormatClass Class of the [[InputFormat]] + * @param inputFormatClass Class of the InputFormat * @param keyClass Class of the keys * @param valueClass Class of the values * @param minSplits Minimum number of Hadoop Splits to generate. diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 9d75d7c4ad69a..006e2a3335428 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -81,7 +81,7 @@ class JobLogger(val user: String, val logDirName: String) /** * Create a log file for one job * @param jobID ID of the job - * @exception FileNotFoundException Fail to create log file + * @throws FileNotFoundException Fail to create log file */ protected def createLogWriter(jobID: Int) { try { diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala index bf71882ef770a..c539d2f708f95 100644 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala @@ -23,9 +23,9 @@ import scala.util.control.{ControlThrowable, NonFatal} import com.typesafe.config.Config /** - * An [[akka.actor.ActorSystem]] which refuses to shut down in the event of a fatal exception. + * An akka.actor.ActorSystem which refuses to shut down in the event of a fatal exception * This is necessary as Spark Executors are allowed to recover from fatal exceptions - * (see [[org.apache.spark.executor.Executor]]). + * (see org.apache.spark.executor.Executor) */ object IndestructibleActorSystem { def apply(name: String, config: Config): ActorSystem = diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala index 5b0d2c36510b8..f837dc7ccc860 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -19,9 +19,9 @@ package org.apache.spark.util /** * A class for tracking the statistics of a set of numbers (count, mean and variance) in a - * numerically robust way. Includes support for merging two StatCounters. Based on - * [[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance - * Welford and Chan's algorithms for running variance]]. + * numerically robust way. Includes support for merging two StatCounters. Based on Welford + * and Chan's [[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance algorithms]] + * for running variance. * * @constructor Initialize the StatCounter with the given values. */ diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index d437c055f33d4..dc4b8f253f259 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -136,7 +136,7 @@ object Vector { /** * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers - * between 0.0 and 1.0. Optional [[scala.util.Random]] number generator can be provided. + * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. */ def random(length: Int, random: Random = new XORShiftRandom()) = Vector(length, _ => random.nextDouble()) diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 20232e9fbb8d0..aa5079c159830 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -75,8 +75,9 @@ public int compare(Integer a, Integer b) { else if (a < b) return 1; else return 0; } - }; + } + @SuppressWarnings("unchecked") @Test public void sparkContextUnion() { // Union of non-specialized JavaRDDs @@ -148,6 +149,7 @@ public void call(String s) { Assert.assertEquals(2, foreachCalls); } + @SuppressWarnings("unchecked") @Test public void lookup() { JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( @@ -179,6 +181,7 @@ public Boolean call(Integer x) { Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds } + @SuppressWarnings("unchecked") @Test public void cogroup() { JavaPairRDD categories = sc.parallelizePairs(Arrays.asList( @@ -197,6 +200,7 @@ public void cogroup() { cogrouped.collect(); } + @SuppressWarnings("unchecked") @Test public void leftOuterJoin() { JavaPairRDD rdd1 = sc.parallelizePairs(Arrays.asList( @@ -243,6 +247,7 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(33, sum); } + @SuppressWarnings("unchecked") @Test public void foldByKey() { List> pairs = Arrays.asList( @@ -265,6 +270,7 @@ public Integer call(Integer a, Integer b) { Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); } + @SuppressWarnings("unchecked") @Test public void reduceByKey() { List> pairs = Arrays.asList( @@ -320,8 +326,8 @@ public void approximateResults() { public void take() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); Assert.assertEquals(1, rdd.first().intValue()); - List firstTwo = rdd.take(2); - List sample = rdd.takeSample(false, 2, 42); + rdd.take(2); + rdd.takeSample(false, 2, 42); } @Test @@ -359,8 +365,8 @@ public Boolean call(Double x) { Assert.assertEquals(2.49444, rdd.stdev(), 0.01); Assert.assertEquals(2.73252, rdd.sampleStdev(), 0.01); - Double first = rdd.first(); - List take = rdd.take(5); + rdd.first(); + rdd.take(5); } @Test @@ -438,11 +444,11 @@ public Iterable call(String s) { return lengths; } }); - Double x = doubles.first(); - Assert.assertEquals(5.0, doubles.first().doubleValue(), 0.01); + Assert.assertEquals(5.0, doubles.first(), 0.01); Assert.assertEquals(11, pairs.count()); } + @SuppressWarnings("unchecked") @Test public void mapsFromPairsToPairs() { List> pairs = Arrays.asList( @@ -509,6 +515,7 @@ public void repartition() { } } + @SuppressWarnings("unchecked") @Test public void persist() { JavaDoubleRDD doubleRDD = sc.parallelizeDoubles(Arrays.asList(1.0, 1.0, 2.0, 3.0, 5.0, 8.0)); @@ -573,6 +580,7 @@ public void textFilesCompressed() throws IOException { Assert.assertEquals(expected, readRDD.collect()); } + @SuppressWarnings("unchecked") @Test public void sequenceFile() { File tempDir = Files.createTempDir(); @@ -602,6 +610,7 @@ public Tuple2 call(Tuple2 pair) { Assert.assertEquals(pairs, readRDD.collect()); } + @SuppressWarnings("unchecked") @Test public void writeWithNewAPIHadoopFile() { File tempDir = Files.createTempDir(); @@ -632,6 +641,7 @@ public String call(Tuple2 x) { }).collect().toString()); } + @SuppressWarnings("unchecked") @Test public void readWithNewAPIHadoopFile() throws IOException { File tempDir = Files.createTempDir(); @@ -674,6 +684,7 @@ public void objectFilesOfInts() { Assert.assertEquals(expected, readRDD.collect()); } + @SuppressWarnings("unchecked") @Test public void objectFilesOfComplexTypes() { File tempDir = Files.createTempDir(); @@ -690,6 +701,7 @@ public void objectFilesOfComplexTypes() { Assert.assertEquals(pairs, readRDD.collect()); } + @SuppressWarnings("unchecked") @Test public void hadoopFile() { File tempDir = Files.createTempDir(); @@ -719,6 +731,7 @@ public String call(Tuple2 x) { }).collect().toString()); } + @SuppressWarnings("unchecked") @Test public void hadoopFileCompressed() { File tempDir = Files.createTempDir(); @@ -824,7 +837,7 @@ public Float zero(Float initialValue) { } }; - final Accumulator floatAccum = sc.accumulator((Float) 10.0f, floatAccumulatorParam); + final Accumulator floatAccum = sc.accumulator(10.0f, floatAccumulatorParam); rdd.foreach(new VoidFunction() { public void call(Integer x) { floatAccum.add((float) x); @@ -876,6 +889,7 @@ public void checkpointAndRestore() { Assert.assertEquals(Arrays.asList(1, 2, 3, 4, 5), recovered.collect()); } + @SuppressWarnings("unchecked") @Test public void mapOnPairRDD() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1,2,3,4)); @@ -900,6 +914,7 @@ public Tuple2 call(Tuple2 in) throws Excepti } + @SuppressWarnings("unchecked") @Test public void collectPartitions() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); @@ -968,7 +983,7 @@ public void countApproxDistinctByKey() { @Test public void collectAsMapWithIntArrayValues() { // Regression test for SPARK-1040 - JavaRDD rdd = sc.parallelize(Arrays.asList(new Integer[] { 1 })); + JavaRDD rdd = sc.parallelize(Arrays.asList(1)); JavaPairRDD pairRDD = rdd.map(new PairFunction() { @Override public Tuple2 call(Integer x) throws Exception { @@ -976,6 +991,6 @@ public Tuple2 call(Integer x) throws Exception { } }); pairRDD.collect(); // Works fine - Map map = pairRDD.collectAsMap(); // Used to crash with ClassCastException + pairRDD.collectAsMap(); // Used to crash with ClassCastException } } diff --git a/pom.xml b/pom.xml index 6adc670462b25..21060ee69c041 100644 --- a/pom.xml +++ b/pom.xml @@ -592,12 +592,13 @@ org.apache.maven.plugins maven-compiler-plugin - 2.5.1 + 3.1 ${java.version} ${java.version} UTF-8 1024m + true @@ -612,7 +613,7 @@ org.scalatest scalatest-maven-plugin - 1.0-M2 + 1.0-RC2 ${project.build.directory}/surefire-reports . diff --git a/repl/pom.xml b/repl/pom.xml index 73597f635b9e0..4c5f9720c802a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -98,7 +98,7 @@ true - + @@ -111,7 +111,7 @@ - + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 4dcd0e4c51ec3..2c7ff87744d7a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -127,7 +127,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. * Therefore, the values for each key in `this` DStream's RDDs are grouped into a - * single sequence to generate the RDDs of the new DStream. [[org.apache.spark.Partitioner]] + * single sequence to generate the RDDs of the new DStream. org.apache.spark.Partitioner * is used to control the partitioning of each RDD. */ def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] = @@ -151,7 +151,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. [[org.apache.spark.Partitioner]] is used to control + * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control * thepartitioning of each RDD. */ def reduceByKey(func: JFunction2[V, V, V], partitioner: Partitioner): JavaPairDStream[K, V] = { @@ -161,7 +161,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Combine elements of each key in DStream's RDDs using custom function. This is similar to the * combineByKey for RDDs. Please refer to combineByKey in - * [[org.apache.spark.rdd.PairRDDFunctions]] for more information. + * org.apache.spark.rdd.PairRDDFunctions for more information. */ def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], @@ -176,7 +176,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Combine elements of each key in DStream's RDDs using custom function. This is similar to the * combineByKey for RDDs. Please refer to combineByKey in - * [[org.apache.spark.rdd.PairRDDFunctions]] for more information. + * org.apache.spark.rdd.PairRDDFunctions for more information. */ def combineByKey[C](createCombiner: JFunction[V, C], mergeValue: JFunction2[C, V, C], @@ -479,7 +479,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. - * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new @@ -579,7 +579,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. - * The supplied [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ def join[W]( other: JavaPairDStream[K, W], @@ -619,7 +619,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. - * The supplied [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ def leftOuterJoin[W]( other: JavaPairDStream[K, W], @@ -660,7 +660,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and - * `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ def rightOuterJoin[W]( diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index 2268160dccc1f..b082bb058529b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -406,7 +406,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * JavaPairDStream in the list of JavaDStreams, convert it to a JavaDStream using * [[org.apache.spark.streaming.api.java.JavaPairDStream]].toJavaDStream(). * In the transform function, convert the JavaRDD corresponding to that JavaDStream to - * a JavaPairRDD using [[org.apache.spark.api.java.JavaPairRDD]].fromJavaRDD(). + * a JavaPairRDD using org.apache.spark.api.java.JavaPairRDD.fromJavaRDD(). */ def transform[T]( dstreams: JList[JavaDStream[_]], @@ -429,7 +429,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * JavaPairDStream in the list of JavaDStreams, convert it to a JavaDStream using * [[org.apache.spark.streaming.api.java.JavaPairDStream]].toJavaDStream(). * In the transform function, convert the JavaRDD corresponding to that JavaDStream to - * a JavaPairRDD using [[org.apache.spark.api.java.JavaPairRDD]].fromJavaRDD(). + * a JavaPairRDD using org.apache.spark.api.java.JavaPairRDD.fromJavaRDD(). */ def transform[K, V]( dstreams: JList[JavaDStream[_]], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index f3c58aede092a..2473496949360 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -65,7 +65,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) /** * Return a new DStream by applying `groupByKey` on each RDD. The supplied - * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = { val createCombiner = (v: V) => ArrayBuffer[V](v) @@ -95,7 +95,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are - * merged using the supplied reduce function. [[org.apache.spark.Partitioner]] is used to control + * merged using the supplied reduce function. org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ def reduceByKey(reduceFunc: (V, V) => V, partitioner: Partitioner): DStream[(K, V)] = { @@ -376,7 +376,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of the key. - * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. * @param partitioner Partitioner for controlling the partitioning of each RDD in the new @@ -396,7 +396,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) /** * Return a new "state" DStream where the state for each key is updated by applying * the given function on the previous state of the key and the new values of each key. - * [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * org.apache.spark.Partitioner is used to control the partitioning of each RDD. * @param updateFunc State update function. If `this` function returns None, then * corresponding state key-value pair will be eliminated. Note, that * this function may generate a different a tuple with a different key @@ -453,7 +453,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) /** * Return a new DStream by applying 'cogroup' between RDDs of `this` DStream and `other` DStream. - * The supplied [[org.apache.spark.Partitioner]] is used to partition the generated RDDs. + * The supplied org.apache.spark.Partitioner is used to partition the generated RDDs. */ def cogroup[W: ClassTag]( other: DStream[(K, W)], @@ -483,7 +483,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) /** * Return a new DStream by applying 'join' between RDDs of `this` DStream and `other` DStream. - * The supplied [[org.apache.spark.Partitioner]] is used to control the partitioning of each RDD. + * The supplied org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ def join[W: ClassTag]( other: DStream[(K, W)], @@ -518,7 +518,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) /** * Return a new DStream by applying 'left outer join' between RDDs of `this` DStream and - * `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ def leftOuterJoin[W: ClassTag]( @@ -554,7 +554,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) /** * Return a new DStream by applying 'right outer join' between RDDs of `this` DStream and - * `other` DStream. The supplied [[org.apache.spark.Partitioner]] is used to control + * `other` DStream. The supplied org.apache.spark.Partitioner is used to control * the partitioning of each RDD. */ def rightOuterJoin[W: ClassTag]( diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 4fbbce9b8b90e..54a0791d04ea4 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -19,7 +19,6 @@ import scala.Tuple2; -import org.junit.After; import org.junit.Assert; import org.junit.Test; import java.io.*; @@ -30,7 +29,6 @@ import com.google.common.io.Files; import com.google.common.collect.Sets; -import org.apache.spark.SparkConf; import org.apache.spark.HashPartitioner; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; @@ -38,6 +36,7 @@ import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaDStreamLike; import org.apache.spark.streaming.api.java.JavaPairDStream; import org.apache.spark.streaming.api.java.JavaStreamingContext; @@ -45,6 +44,8 @@ // serialized, as an alternative to converting these anonymous classes to static inner classes; // see http://stackoverflow.com/questions/758570/. public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable { + + @SuppressWarnings("unchecked") @Test public void testCount() { List> inputData = Arrays.asList( @@ -64,6 +65,7 @@ public void testCount() { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testMap() { List> inputData = Arrays.asList( @@ -87,6 +89,7 @@ public Integer call(String s) throws Exception { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testWindow() { List> inputData = Arrays.asList( @@ -108,6 +111,7 @@ public void testWindow() { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testWindowWithSlideDuration() { List> inputData = Arrays.asList( @@ -132,6 +136,7 @@ public void testWindowWithSlideDuration() { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testFilter() { List> inputData = Arrays.asList( @@ -155,13 +160,16 @@ public Boolean call(String s) throws Exception { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testRepartitionMorePartitions() { List> inputData = Arrays.asList( Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 2); - JavaDStream repartitioned = stream.repartition(4); + JavaDStream stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 2); + JavaDStreamLike,JavaRDD> repartitioned = + stream.repartition(4); JavaTestUtils.attachTestOutputStream(repartitioned); List>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2); Assert.assertEquals(2, result.size()); @@ -172,13 +180,16 @@ public void testRepartitionMorePartitions() { } } + @SuppressWarnings("unchecked") @Test public void testRepartitionFewerPartitions() { List> inputData = Arrays.asList( Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10), Arrays.asList(1, 2, 3, 4, 5, 6, 7, 8, 9, 10)); - JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 4); - JavaDStream repartitioned = stream.repartition(2); + JavaDStream stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 4); + JavaDStreamLike,JavaRDD> repartitioned = + stream.repartition(2); JavaTestUtils.attachTestOutputStream(repartitioned); List>> result = JavaTestUtils.runStreamsWithPartitions(ssc, 2, 2); Assert.assertEquals(2, result.size()); @@ -188,6 +199,7 @@ public void testRepartitionFewerPartitions() { } } + @SuppressWarnings("unchecked") @Test public void testGlom() { List> inputData = Arrays.asList( @@ -206,6 +218,7 @@ public void testGlom() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testMapPartitions() { List> inputData = Arrays.asList( @@ -217,16 +230,17 @@ public void testMapPartitions() { Arrays.asList("YANKEESRED SOCKS")); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaDStream mapped = stream.mapPartitions(new FlatMapFunction, String>() { - @Override - public Iterable call(Iterator in) { - String out = ""; - while (in.hasNext()) { - out = out + in.next().toUpperCase(); - } - return Lists.newArrayList(out); - } - }); + JavaDStream mapped = stream.mapPartitions( + new FlatMapFunction, String>() { + @Override + public Iterable call(Iterator in) { + String out = ""; + while (in.hasNext()) { + out = out + in.next().toUpperCase(); + } + return Lists.newArrayList(out); + } + }); JavaTestUtils.attachTestOutputStream(mapped); List> result = JavaTestUtils.runStreams(ssc, 2, 2); @@ -247,6 +261,7 @@ public Integer call(Integer i1, Integer i2) throws Exception { } } + @SuppressWarnings("unchecked") @Test public void testReduce() { List> inputData = Arrays.asList( @@ -267,6 +282,7 @@ public void testReduce() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testReduceByWindow() { List> inputData = Arrays.asList( @@ -289,6 +305,7 @@ public void testReduceByWindow() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testQueueStream() { List> expected = Arrays.asList( @@ -312,6 +329,7 @@ public void testQueueStream() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testTransform() { List> inputData = Arrays.asList( @@ -344,6 +362,7 @@ public Integer call(Integer i) throws Exception { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testVariousTransform() { // tests whether all variations of transform can be called from Java @@ -423,6 +442,7 @@ public JavaRDD call(JavaRDD in) throws Exception { } + @SuppressWarnings("unchecked") @Test public void testTransformWith() { List>> stringStringKVStream1 = Arrays.asList( @@ -492,6 +512,7 @@ public JavaPairRDD> call( } + @SuppressWarnings("unchecked") @Test public void testVariousTransformWith() { // tests whether all variations of transformWith can be called from Java @@ -591,6 +612,7 @@ public JavaPairRDD call(JavaPairRDD rdd1, JavaP ); } + @SuppressWarnings("unchecked") @Test public void testStreamingContextTransform(){ List> stream1input = Arrays.asList( @@ -658,6 +680,7 @@ public Tuple2 call(Integer i) throws Exception { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testFlatMap() { List> inputData = Arrays.asList( @@ -683,6 +706,7 @@ public Iterable call(String x) { assertOrderInvariantEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairFlatMap() { List> inputData = Arrays.asList( @@ -718,22 +742,24 @@ public void testPairFlatMap() { new Tuple2(9, "s"))); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream flatMapped = stream.flatMap(new PairFlatMapFunction() { - @Override - public Iterable> call(String in) throws Exception { - List> out = Lists.newArrayList(); - for (String letter: in.split("(?!^)")) { - out.add(new Tuple2(in.length(), letter)); - } - return out; - } - }); + JavaPairDStream flatMapped = stream.flatMap( + new PairFlatMapFunction() { + @Override + public Iterable> call(String in) throws Exception { + List> out = Lists.newArrayList(); + for (String letter: in.split("(?!^)")) { + out.add(new Tuple2(in.length(), letter)); + } + return out; + } + }); JavaTestUtils.attachTestOutputStream(flatMapped); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testUnion() { List> inputData1 = Arrays.asList( @@ -778,6 +804,7 @@ public static > void assertOrderInvariantEquals( // PairDStream Functions + @SuppressWarnings("unchecked") @Test public void testPairFilter() { List> inputData = Arrays.asList( @@ -810,7 +837,8 @@ public Boolean call(Tuple2 in) throws Exception { Assert.assertEquals(expected, result); } - List>> stringStringKVStream = Arrays.asList( + @SuppressWarnings("unchecked") + private List>> stringStringKVStream = Arrays.asList( Arrays.asList(new Tuple2("california", "dodgers"), new Tuple2("california", "giants"), new Tuple2("new york", "yankees"), @@ -820,7 +848,8 @@ public Boolean call(Tuple2 in) throws Exception { new Tuple2("new york", "rangers"), new Tuple2("new york", "islanders"))); - List>> stringIntKVStream = Arrays.asList( + @SuppressWarnings("unchecked") + private List>> stringIntKVStream = Arrays.asList( Arrays.asList( new Tuple2("california", 1), new Tuple2("california", 3), @@ -832,6 +861,7 @@ public Boolean call(Tuple2 in) throws Exception { new Tuple2("new york", 3), new Tuple2("new york", 1))); + @SuppressWarnings("unchecked") @Test public void testPairMap() { // Maps pair -> pair of different type List>> inputData = stringIntKVStream; @@ -864,6 +894,7 @@ public Tuple2 call(Tuple2 in) throws Exception Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairMapPartitions() { // Maps pair -> pair of different type List>> inputData = stringIntKVStream; @@ -901,6 +932,7 @@ public Iterable> call(Iterator> Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairMap2() { // Maps pair -> single List>> inputData = stringIntKVStream; @@ -925,6 +957,7 @@ public Integer call(Tuple2 in) throws Exception { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair List>> inputData = Arrays.asList( @@ -967,6 +1000,7 @@ public Iterable> call(Tuple2 in) throws Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairGroupByKey() { List>> inputData = stringStringKVStream; @@ -989,6 +1023,7 @@ public void testPairGroupByKey() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairReduceByKey() { List>> inputData = stringIntKVStream; @@ -1013,6 +1048,7 @@ public void testPairReduceByKey() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCombineByKey() { List>> inputData = stringIntKVStream; @@ -1043,6 +1079,7 @@ public Integer call(Integer i) throws Exception { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCountByValue() { List> inputData = Arrays.asList( @@ -1068,6 +1105,7 @@ public void testCountByValue() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testGroupByKeyAndWindow() { List>> inputData = stringIntKVStream; @@ -1113,6 +1151,7 @@ private Tuple2> convert(Tuple2> t return new Tuple2>(tuple._1(), new HashSet(tuple._2())); } + @SuppressWarnings("unchecked") @Test public void testReduceByKeyAndWindow() { List>> inputData = stringIntKVStream; @@ -1136,6 +1175,7 @@ public void testReduceByKeyAndWindow() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testUpdateStateByKey() { List>> inputData = stringIntKVStream; @@ -1171,6 +1211,7 @@ public Optional call(List values, Optional state) { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testReduceByKeyAndWindowWithInverse() { List>> inputData = stringIntKVStream; @@ -1194,6 +1235,7 @@ public void testReduceByKeyAndWindowWithInverse() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCountByValueAndWindow() { List> inputData = Arrays.asList( @@ -1227,6 +1269,7 @@ public void testCountByValueAndWindow() { Assert.assertEquals(expected, unorderedResult); } + @SuppressWarnings("unchecked") @Test public void testPairTransform() { List>> inputData = Arrays.asList( @@ -1271,6 +1314,7 @@ public JavaPairRDD call(JavaPairRDD in) thro Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testPairToNormalRDDTransform() { List>> inputData = Arrays.asList( @@ -1312,6 +1356,8 @@ public Integer call(Tuple2 in) { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") + @Test public void testMapValues() { List>> inputData = stringStringKVStream; @@ -1342,6 +1388,7 @@ public String call(String s) throws Exception { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testFlatMapValues() { List>> inputData = stringStringKVStream; @@ -1386,6 +1433,7 @@ public Iterable call(String in) { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCoGroup() { List>> stringStringKVStream1 = Arrays.asList( @@ -1429,6 +1477,7 @@ public void testCoGroup() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testJoin() { List>> stringStringKVStream1 = Arrays.asList( @@ -1472,6 +1521,7 @@ public void testJoin() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testLeftOuterJoin() { List>> stringStringKVStream1 = Arrays.asList( @@ -1503,6 +1553,7 @@ public void testLeftOuterJoin() { Assert.assertEquals(expected, result); } + @SuppressWarnings("unchecked") @Test public void testCheckpointMasterRecovery() throws InterruptedException { List> inputData = Arrays.asList( @@ -1541,7 +1592,8 @@ public Integer call(String s) throws Exception { } - /** TEST DISABLED: Pending a discussion about checkpoint() semantics with TD + /* TEST DISABLED: Pending a discussion about checkpoint() semantics with TD + @SuppressWarnings("unchecked") @Test public void testCheckpointofIndividualStream() throws InterruptedException { List> inputData = Arrays.asList( @@ -1581,16 +1633,14 @@ public void testSocketTextStream() { @Test public void testSocketString() { class Converter extends Function> { - public Iterable call(InputStream in) { + public Iterable call(InputStream in) throws IOException { BufferedReader reader = new BufferedReader(new InputStreamReader(in)); List out = new ArrayList(); - try { - while (true) { - String line = reader.readLine(); - if (line == null) { break; } - out.add(line); - } - } catch (IOException e) { } + while (true) { + String line = reader.readLine(); + if (line == null) { break; } + out.add(line); + } return out; } } diff --git a/yarn/pom.xml b/yarn/pom.xml index e7eba36ba351b..c0e133dd603b1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -133,7 +133,7 @@ true - + @@ -146,7 +146,7 @@ - + From c42557be32fbd38e084e5bb56cd6301299e84cbf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 27 Feb 2014 15:06:20 -0800 Subject: [PATCH 017/397] [HOTFIX] Patching maven build after #6 (SPARK-1121). That patch removed the Maven avro declaration but didn't remove the actual dependency in core. /cc @scrapcodes Author: Patrick Wendell Closes #37 from pwendell/master and squashes the following commits: 0ef3008 [Patrick Wendell] [HOTFIX] Patching maven build after #6 (SPARK-1121). --- core/pom.xml | 8 -------- 1 file changed, 8 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index a333bff28c246..d3a81d564c2e8 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -46,14 +46,6 @@ - - org.apache.avro - avro - - - org.apache.avro - avro-ipc - org.apache.curator curator-recipes From 40e080a68a8fd025435e9ff84fa9280b4aba4dcf Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 27 Feb 2014 21:13:22 -0800 Subject: [PATCH 018/397] Removed reference to incubation in Spark user docs. Author: Reynold Xin Closes #2 from rxin/docs and squashes the following commits: 08bbd5f [Reynold Xin] Removed reference to incubation in Spark user docs. --- docs/README.md | 2 +- docs/_config.yml | 4 ++-- docs/_layouts/global.html | 10 ---------- docs/bagel-programming-guide.md | 2 +- docs/index.md | 12 ++++++------ docs/java-programming-guide.md | 2 +- docs/scala-programming-guide.md | 2 +- docs/spark-debugger.md | 4 ++-- 8 files changed, 14 insertions(+), 24 deletions(-) diff --git a/docs/README.md b/docs/README.md index cc09d6e88f41e..cac65d97e488b 100644 --- a/docs/README.md +++ b/docs/README.md @@ -1,6 +1,6 @@ Welcome to the Spark documentation! -This readme will walk you through navigating and building the Spark documentation, which is included here with the Spark source code. You can also find documentation specific to release versions of Spark at http://spark.incubator.apache.org/documentation.html. +This readme will walk you through navigating and building the Spark documentation, which is included here with the Spark source code. You can also find documentation specific to release versions of Spark at http://spark.apache.org/documentation.html. Read on to learn more about viewing documentation in plain text (i.e., markdown) or building the documentation yourself. Why build it yourself? So that you have the docs that corresponds to whichever version of Spark you currently have checked out of revision control. diff --git a/docs/_config.yml b/docs/_config.yml index 9e5a95fe53af6..aa5a5adbc1743 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -3,10 +3,10 @@ markdown: kramdown # These allow the documentation to be updated with nerw releases # of Spark, Scala, and Mesos. -SPARK_VERSION: 1.0.0-incubating-SNAPSHOT +SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.3" MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net -SPARK_GITHUB_URL: https://github.com/apache/incubator-spark +SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 7114e1f5dd5b9..ebb58e8b9af79 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -159,16 +159,6 @@

    Heading


    --> -
    -
    -

    - Apache Spark is an effort undergoing incubation at the Apache Software Foundation. - - - -

    -
    - diff --git a/docs/bagel-programming-guide.md b/docs/bagel-programming-guide.md index b070d8e73a38b..da6d0c9dcd97b 100644 --- a/docs/bagel-programming-guide.md +++ b/docs/bagel-programming-guide.md @@ -108,7 +108,7 @@ _Example_ ## Operations -Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/incubator-spark/blob/master/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala) for details. +Here are the actions and types in the Bagel API. See [Bagel.scala](https://github.com/apache/spark/blob/master/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala) for details. ### Actions diff --git a/docs/index.md b/docs/index.md index aa9c8666e7d75..4eb297df39144 100644 --- a/docs/index.md +++ b/docs/index.md @@ -9,7 +9,7 @@ It also supports a rich set of higher-level tools including [Shark](http://shark # Downloading -Get Spark by visiting the [downloads page](http://spark.incubator.apache.org/downloads.html) of the Apache Spark site. This documentation is for Spark version {{site.SPARK_VERSION}}. +Get Spark by visiting the [downloads page](http://spark.apache.org/downloads.html) of the Apache Spark site. This documentation is for Spark version {{site.SPARK_VERSION}}. Spark runs on both Windows and UNIX-like systems (e.g. Linux, Mac OS). All you need to run it is to have `java` to installed on your system `PATH`, or the `JAVA_HOME` environment variable pointing to a Java installation. @@ -96,7 +96,7 @@ For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to bui * [Amazon EC2](ec2-scripts.html): scripts that let you launch a cluster on EC2 in about 5 minutes * [Standalone Deploy Mode](spark-standalone.html): launch a standalone cluster quickly without a third-party cluster manager * [Mesos](running-on-mesos.html): deploy a private cluster using - [Apache Mesos](http://incubator.apache.org/mesos) + [Apache Mesos](http://mesos.apache.org) * [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN) **Other documents:** @@ -110,20 +110,20 @@ For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to bui **External resources:** -* [Spark Homepage](http://spark.incubator.apache.org) +* [Spark Homepage](http://spark.apache.org) * [Shark](http://shark.cs.berkeley.edu): Apache Hive over Spark -* [Mailing Lists](http://spark.incubator.apache.org/mailing-lists.html): ask questions about Spark here +* [Mailing Lists](http://spark.apache.org/mailing-lists.html): ask questions about Spark here * [AMP Camps](http://ampcamp.berkeley.edu/): a series of training camps at UC Berkeley that featured talks and exercises about Spark, Shark, Mesos, and more. [Videos](http://ampcamp.berkeley.edu/agenda-2012), [slides](http://ampcamp.berkeley.edu/agenda-2012) and [exercises](http://ampcamp.berkeley.edu/exercises-2012) are available online for free. -* [Code Examples](http://spark.incubator.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/apache/incubator-spark/tree/master/examples/src/main/scala/) of Spark +* [Code Examples](http://spark.apache.org/examples.html): more are also available in the [examples subfolder](https://github.com/apache/spark/tree/master/examples/src/main/scala/) of Spark * [Paper Describing Spark](http://www.cs.berkeley.edu/~matei/papers/2012/nsdi_spark.pdf) * [Paper Describing Spark Streaming](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) # Community -To get help using Spark or keep up with Spark development, sign up for the [user mailing list](http://spark.incubator.apache.org/mailing-lists.html). +To get help using Spark or keep up with Spark development, sign up for the [user mailing list](http://spark.apache.org/mailing-lists.html). If you're in the San Francisco Bay Area, there's a regular [Spark meetup](http://www.meetup.com/spark-users/) every few weeks. Come by to meet the developers and other users. diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index 07732fa1229f3..5c73dbb25ede8 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -189,7 +189,7 @@ We hope to generate documentation with Java-style syntax in the future. # Where to Go from Here Spark includes several sample programs using the Java API in -[`examples/src/main/java`](https://github.com/apache/incubator-spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the +[`examples/src/main/java`](https://github.com/apache/spark/tree/master/examples/src/main/java/org/apache/spark/examples). You can run them by passing the class name to the `bin/run-example` script included in Spark; for example: ./bin/run-example org.apache.spark.examples.JavaWordCount diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 506d3faa767f3..99412733d4268 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -365,7 +365,7 @@ res2: Int = 10 # Where to Go from Here -You can see some [example Spark programs](http://spark.incubator.apache.org/examples.html) on the Spark website. +You can see some [example Spark programs](http://spark.apache.org/examples.html) on the Spark website. In addition, Spark includes several samples in `examples/src/main/scala`. Some of them have both Spark versions and local (non-parallel) versions, allowing you to see what had to be changed to make the program run on a cluster. You can run them using by passing the class name to the `bin/run-example` script included in Spark; for example: ./bin/run-example org.apache.spark.examples.SparkPi diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md index 11c51d5cde7c9..891c2bfa8943d 100644 --- a/docs/spark-debugger.md +++ b/docs/spark-debugger.md @@ -2,7 +2,7 @@ layout: global title: The Spark Debugger --- -**Summary:** The Spark debugger provides replay debugging for deterministic (logic) errors in Spark programs. It's currently in development, but you can try it out in the [arthur branch](https://github.com/apache/incubator-spark/tree/arthur). +**Summary:** The Spark debugger provides replay debugging for deterministic (logic) errors in Spark programs. It's currently in development, but you can try it out in the [arthur branch](https://github.com/apache/spark/tree/arthur). ## Introduction @@ -19,7 +19,7 @@ For deterministic errors, debugging a Spark program is now as easy as debugging ## Approach -As your Spark program runs, the slaves report key events back to the master -- for example, RDD creations, RDD contents, and uncaught exceptions. (A full list of event types is in [EventLogging.scala](https://github.com/apache/incubator-spark/blob/arthur/core/src/main/scala/spark/EventLogging.scala).) The master logs those events, and you can load the event log into the debugger after your program is done running. +As your Spark program runs, the slaves report key events back to the master -- for example, RDD creations, RDD contents, and uncaught exceptions. (A full list of event types is in [EventLogging.scala](https://github.com/apache/spark/blob/arthur/core/src/main/scala/spark/EventLogging.scala).) The master logs those events, and you can load the event log into the debugger after your program is done running. _A note on nondeterminism:_ For fault recovery, Spark requires RDD transformations (for example, the function passed to `RDD.map`) to be deterministic. The Spark debugger also relies on this property, and it can also warn you if your transformation is nondeterministic. This works by checksumming the contents of each RDD and comparing the checksums from the original execution to the checksums after recomputing the RDD in the debugger. From edf8a56ab7eaee1f7c3b4579eb10464984d31d7a Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Thu, 27 Feb 2014 21:52:55 -0800 Subject: [PATCH 019/397] Remote BlockFetchTracker trait This trait seems to have been created a while ago when there were multiple implementations; now that there's just one, I think it makes sense to merge it into the BlockFetcherIterator trait. Author: Kay Ousterhout Closes #39 from kayousterhout/remove_tracker and squashes the following commits: 8173939 [Kay Ousterhout] Remote BlockFetchTracker. --- .../spark/storage/BlockFetchTracker.scala | 27 ------------------ .../spark/storage/BlockFetcherIterator.scala | 28 +++++++++++-------- 2 files changed, 17 insertions(+), 38 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.scala diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.scala deleted file mode 100644 index 2e0b0e6eda765..0000000000000 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetchTracker.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.storage - -private[spark] trait BlockFetchTracker { - def totalBlocks : Int - def numLocalBlocks: Int - def numRemoteBlocks: Int - def remoteFetchTime : Long - def fetchWaitTime: Long - def remoteBytesRead : Long -} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 925022e7fe6fb..fb50b45bd4197 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -44,9 +44,14 @@ import org.apache.spark.util.Utils */ private[storage] -trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] - with Logging with BlockFetchTracker { +trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] with Logging { def initialize() + def totalBlocks: Int + def numLocalBlocks: Int + def numRemoteBlocks: Int + def remoteFetchTime: Long + def fetchWaitTime: Long + def remoteBytesRead: Long } @@ -233,7 +238,16 @@ object BlockFetcherIterator { logDebug("Got local blocks in " + Utils.getUsedTimeMs(startTime) + " ms") } - //an iterator that will read fetched blocks off the queue as they arrive. + override def totalBlocks: Int = numLocal + numRemote + override def numLocalBlocks: Int = numLocal + override def numRemoteBlocks: Int = numRemote + override def remoteFetchTime: Long = _remoteFetchTime + override def fetchWaitTime: Long = _fetchWaitTime + override def remoteBytesRead: Long = _remoteBytesRead + + + // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue + // as they arrive. @volatile protected var resultsGotten = 0 override def hasNext: Boolean = resultsGotten < _numBlocksToFetch @@ -251,14 +265,6 @@ object BlockFetcherIterator { } (result.blockId, if (result.failed) None else Some(result.deserialize())) } - - // Implementing BlockFetchTracker trait. - override def totalBlocks: Int = numLocal + numRemote - override def numLocalBlocks: Int = numLocal - override def numRemoteBlocks: Int = numRemote - override def remoteFetchTime: Long = _remoteFetchTime - override def fetchWaitTime: Long = _fetchWaitTime - override def remoteBytesRead: Long = _remoteBytesRead } // End of BasicBlockFetcherIterator From 5f419bf9f433e8f057237f1d5bfed9f5f4e9427c Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 28 Feb 2014 09:40:47 -0600 Subject: [PATCH 020/397] SPARK-1032. If Yarn app fails before registering, app master stays aroun... ...d long after This reopens https://github.com/apache/incubator-spark/pull/648 against the new repo. Author: Sandy Ryza Closes #28 from sryza/sandy-spark-1032 and squashes the following commits: 5953f50 [Sandy Ryza] SPARK-1032. If Yarn app fails before registering, app master stays around long after --- .../spark/deploy/yarn/ApplicationMaster.scala | 34 ++++++++++++------- .../spark/deploy/yarn/ApplicationMaster.scala | 22 ++++++++---- 2 files changed, 38 insertions(+), 18 deletions(-) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 2e46d750c4a38..560e5de358908 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -65,6 +65,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3)) + private var registered = false + def run() { // Setup the directories so things go to yarn approved directories rather // then user specified and /tmp. @@ -110,7 +112,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, waitForSparkContextInitialized() // Do this after spark master is up and SparkContext is created so that we can register UI Url - val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() + synchronized { + if (!isFinished) { + registerApplicationMaster() + registered = true + } + } // Allocate all containers allocateWorkers() @@ -208,7 +215,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, var count = 0 val waitTime = 10000L val numTries = sparkConf.getInt("spark.yarn.ApplicationMaster.waitTries", 10) - while (ApplicationMaster.sparkContextRef.get() == null && count < numTries) { + while (ApplicationMaster.sparkContextRef.get() == null && count < numTries + && !isFinished) { logInfo("Waiting for spark context initialization ... " + count) count = count + 1 ApplicationMaster.sparkContextRef.wait(waitTime) @@ -341,17 +349,19 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, return } isFinished = true + + logInfo("finishApplicationMaster with " + status) + if (registered) { + val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) + .asInstanceOf[FinishApplicationMasterRequest] + finishReq.setAppAttemptId(appAttemptId) + finishReq.setFinishApplicationStatus(status) + finishReq.setDiagnostics(diagnostics) + // Set tracking url to empty since we don't have a history server. + finishReq.setTrackingUrl("") + resourceManager.finishApplicationMaster(finishReq) + } } - - logInfo("finishApplicationMaster with " + status) - val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) - .asInstanceOf[FinishApplicationMasterRequest] - finishReq.setAppAttemptId(appAttemptId) - finishReq.setFinishApplicationStatus(status) - finishReq.setDiagnostics(diagnostics) - // Set tracking url to empty since we don't have a history server. - finishReq.setTrackingUrl("") - resourceManager.finishApplicationMaster(finishReq) } /** diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 4b777d5fa7a28..0f58c49c69583 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -67,6 +67,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numWorkers * 2, 3)) + private var registered = false + def run() { // Setup the directories so things go to YARN approved directories rather // than user specified and /tmp. @@ -99,7 +101,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, waitForSparkContextInitialized() // Do this after Spark master is up and SparkContext is created so that we can register UI Url. - val appMasterResponse: RegisterApplicationMasterResponse = registerApplicationMaster() + synchronized { + if (!isFinished) { + registerApplicationMaster() + registered = true + } + } // Allocate all containers allocateWorkers() @@ -180,7 +187,8 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, var numTries = 0 val waitTime = 10000L val maxNumTries = sparkConf.getInt("spark.yarn.applicationMaster.waitTries", 10) - while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries) { + while (ApplicationMaster.sparkContextRef.get() == null && numTries < maxNumTries + && !isFinished) { logInfo("Waiting for Spark context initialization ... " + numTries) numTries = numTries + 1 ApplicationMaster.sparkContextRef.wait(waitTime) @@ -313,11 +321,13 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, return } isFinished = true - } - logInfo("finishApplicationMaster with " + status) - // Set tracking URL to empty since we don't have a history server. - amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */) + logInfo("finishApplicationMaster with " + status) + if (registered) { + // Set tracking URL to empty since we don't have a history server. + amClient.unregisterApplicationMaster(status, "" /* appMessage */ , "" /* appTrackingUrl */) + } + } } /** From 46dff34458096e5330073ca58e0723da52aeddcd Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 28 Feb 2014 12:43:01 -0600 Subject: [PATCH 021/397] SPARK-1051. On YARN, executors don't doAs submitting user This reopens https://github.com/apache/incubator-spark/pull/538 against the new repo Author: Sandy Ryza Closes #29 from sryza/sandy-spark-1051 and squashes the following commits: 708ce49 [Sandy Ryza] SPARK-1051. doAs submitting user in YARN --- .../apache/spark/deploy/SparkHadoopUtil.scala | 18 ++++++++++-------- .../spark/deploy/yarn/ApplicationMaster.scala | 6 +++++- .../apache/spark/deploy/yarn/ClientBase.scala | 1 + .../deploy/yarn/YarnSparkHadoopUtil.scala | 4 ++++ .../spark/deploy/yarn/ApplicationMaster.scala | 6 +++++- 5 files changed, 25 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index b479225b45ee9..ec15647e1d9eb 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -25,6 +25,8 @@ import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{SparkContext, SparkException} +import scala.collection.JavaConversions._ + /** * Contains util methods to interact with Hadoop from Spark. */ @@ -33,15 +35,9 @@ class SparkHadoopUtil { UserGroupInformation.setConfiguration(conf) def runAsUser(user: String)(func: () => Unit) { - // if we are already running as the user intended there is no reason to do the doAs. It - // will actually break secure HDFS access as it doesn't fill in the credentials. Also if - // the user is UNKNOWN then we shouldn't be creating a remote unknown user - // (this is actually the path spark on yarn takes) since SPARK_USER is initialized only - // in SparkContext. - val currentUser = Option(System.getProperty("user.name")). - getOrElse(SparkContext.SPARK_UNKNOWN_USER) - if (user != SparkContext.SPARK_UNKNOWN_USER && currentUser != user) { + if (user != SparkContext.SPARK_UNKNOWN_USER) { val ugi = UserGroupInformation.createRemoteUser(user) + transferCredentials(UserGroupInformation.getCurrentUser(), ugi) ugi.doAs(new PrivilegedExceptionAction[Unit] { def run: Unit = func() }) @@ -50,6 +46,12 @@ class SparkHadoopUtil { } } + def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { + for (token <- source.getTokens()) { + dest.addToken(token) + } + } + /** * Return an appropriate (subclass) of Configuration. Creating config can initializes some Hadoop * subsystems. diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 560e5de358908..e045b9f0248f6 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -37,6 +37,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, @@ -67,6 +68,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var registered = false + private val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse( + SparkContext.SPARK_UNKNOWN_USER) + def run() { // Setup the directories so things go to yarn approved directories rather // then user specified and /tmp. @@ -180,7 +184,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, false /* initialize */ , Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { - override def run() { + override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () => var successed = false try { // Copy diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 24520bd21ba98..4b6c7db836b02 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -272,6 +272,7 @@ trait ClientBase extends Logging { ClientBase.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir + env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() // Set the environment variables to be passed on to the Workers. distCacheMgr.setDistFilesEnv(env) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 2ba2366ead171..d6c12a9f5952d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -28,6 +28,10 @@ import org.apache.hadoop.conf.Configuration */ class YarnSparkHadoopUtil extends SparkHadoopUtil { + override def transferCredentials(source: UserGroupInformation, dest: UserGroupInformation) { + dest.addCredentials(source.getCredentials()) + } + // Note that all params which start with SPARK are propagated all the way through, so if in yarn mode, this MUST be set to true. override def isYarnMode(): Boolean = { true } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 0f58c49c69583..dd117d5810949 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -39,6 +39,7 @@ import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -68,6 +69,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, math.max(args.numWorkers * 2, 3)) private var registered = false + + private val sparkUser = Option(System.getenv("SPARK_USER")).getOrElse( + SparkContext.SPARK_UNKNOWN_USER) def run() { // Setup the directories so things go to YARN approved directories rather @@ -152,7 +156,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, false /* initialize */ , Thread.currentThread.getContextClassLoader).getMethod("main", classOf[Array[String]]) val t = new Thread { - override def run() { + override def run(): Unit = SparkHadoopUtil.get.runAsUser(sparkUser) { () => var successed = false try { // Copy From 4ba3f70a4e385368d0e826ba261f8eb60c25c896 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 28 Feb 2014 18:28:33 -0800 Subject: [PATCH 022/397] SPARK-1151: Update dev merge script to use spark.git instead of incubator-spark Author: Thomas Graves Closes #47 from tgravescs/fix_merge_script and squashes the following commits: 8209ab1 [Thomas Graves] Update dev merge script to use spark.git instead of incubator-spark --- dev/merge_spark_pr.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 93621c96daf2d..e8f78fc5f231a 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -38,7 +38,7 @@ # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") -GIT_API_BASE = "https://api.github.com/repos/apache/incubator-spark" +GIT_API_BASE = "https://api.github.com/repos/apache/spark" # Prefix added to temporary branches BRANCH_PREFIX = "PR_TOOL" From 556c56689bbc32c6cec0d07b57bd3ec73ceb243e Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Sat, 1 Mar 2014 11:24:22 -0800 Subject: [PATCH 023/397] [SPARK-979] Randomize order of offers. This commit randomizes the order of resource offers to avoid scheduling all tasks on the same small set of machines. This is a much simpler solution to SPARK-979 than #7. Author: Kay Ousterhout Closes #27 from kayousterhout/randomize and squashes the following commits: 435d817 [Kay Ousterhout] [SPARK-979] Randomize order of offers. --- .../spark/scheduler/TaskSchedulerImpl.scala | 15 +++-- .../org/apache/spark/scheduler/FakeTask.scala | 16 ++++++ .../scheduler/TaskSchedulerImplSuite.scala | 56 ++++++++++++++----- .../spark/scheduler/TaskSetManagerSuite.scala | 29 +++------- 4 files changed, 75 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 8df37c247d0d4..23b06612fd7ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -25,6 +25,7 @@ import scala.concurrent.duration._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +import scala.util.Random import org.apache.spark._ import org.apache.spark.TaskState.TaskState @@ -207,9 +208,11 @@ private[spark] class TaskSchedulerImpl( } } - // Build a list of tasks to assign to each worker - val tasks = offers.map(o => new ArrayBuffer[TaskDescription](o.cores)) - val availableCpus = offers.map(o => o.cores).toArray + // Randomly shuffle offers to avoid always placing tasks on the same set of workers. + val shuffledOffers = Random.shuffle(offers) + // Build a list of tasks to assign to each worker. + val tasks = shuffledOffers.map(o => new ArrayBuffer[TaskDescription](o.cores)) + val availableCpus = shuffledOffers.map(o => o.cores).toArray val sortedTaskSets = rootPool.getSortedTaskSetQueue() for (taskSet <- sortedTaskSets) { logDebug("parentName: %s, name: %s, runningTasks: %s".format( @@ -222,9 +225,9 @@ private[spark] class TaskSchedulerImpl( for (taskSet <- sortedTaskSets; maxLocality <- TaskLocality.values) { do { launchedTask = false - for (i <- 0 until offers.size) { - val execId = offers(i).executorId - val host = offers(i).host + for (i <- 0 until shuffledOffers.size) { + val execId = shuffledOffers(i).executorId + val host = shuffledOffers(i).host for (task <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) { tasks(i) += task val tid = task.taskId diff --git a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala index 0b90c4e74c8a4..0a7cb69416a08 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/FakeTask.scala @@ -24,3 +24,19 @@ class FakeTask(stageId: Int, prefLocs: Seq[TaskLocation] = Nil) extends Task[Int override def preferredLocations: Seq[TaskLocation] = prefLocs } + +object FakeTask { + /** + * Utility method to create a TaskSet, potentially setting a particular sequence of preferred + * locations for each task (given as varargs) if this sequence is not empty. + */ + def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { + if (prefLocs.size != 0 && prefLocs.size != numTasks) { + throw new IllegalArgumentException("Wrong number of task locations") + } + val tasks = Array.tabulate[Task[_]](numTasks) { i => + new FakeTask(i, if (prefLocs.size != 0) prefLocs(i) else Nil) + } + new TaskSet(tasks, 0, 0, 0, null) + } +} diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index f4e62c64daf12..6b0800af9c6d0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -25,6 +25,13 @@ import org.scalatest.FunSuite import org.apache.spark._ +class FakeSchedulerBackend extends SchedulerBackend { + def start() {} + def stop() {} + def reviveOffers() {} + def defaultParallelism() = 1 +} + class FakeTaskSetManager( initPriority: Int, initStageId: Int, @@ -107,7 +114,8 @@ class FakeTaskSetManager( class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Logging { - def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskSchedulerImpl, taskSet: TaskSet): FakeTaskSetManager = { + def createDummyTaskSetManager(priority: Int, stage: Int, numTasks: Int, cs: TaskSchedulerImpl, + taskSet: TaskSet): FakeTaskSetManager = { new FakeTaskSetManager(priority, stage, numTasks, cs , taskSet) } @@ -135,10 +143,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin test("FIFO Scheduler Test") { sc = new SparkContext("local", "TaskSchedulerImplSuite") val taskScheduler = new TaskSchedulerImpl(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new FakeTask(0) - tasks += task - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + val taskSet = FakeTask.createTaskSet(1) val rootPool = new Pool("", SchedulingMode.FIFO, 0, 0) val schedulableBuilder = new FIFOSchedulableBuilder(rootPool) @@ -162,10 +167,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin test("Fair Scheduler Test") { sc = new SparkContext("local", "TaskSchedulerImplSuite") val taskScheduler = new TaskSchedulerImpl(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new FakeTask(0) - tasks += task - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + val taskSet = FakeTask.createTaskSet(1) val xmlPath = getClass.getClassLoader.getResource("fairscheduler.xml").getFile() System.setProperty("spark.scheduler.allocation.file", xmlPath) @@ -219,10 +221,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin test("Nested Pool Test") { sc = new SparkContext("local", "TaskSchedulerImplSuite") val taskScheduler = new TaskSchedulerImpl(sc) - var tasks = ArrayBuffer[Task[_]]() - val task = new FakeTask(0) - tasks += task - val taskSet = new TaskSet(tasks.toArray,0,0,0,null) + val taskSet = FakeTask.createTaskSet(1) val rootPool = new Pool("", SchedulingMode.FAIR, 0, 0) val pool0 = new Pool("0", SchedulingMode.FAIR, 3, 1) @@ -265,4 +264,35 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin checkTaskSetId(rootPool, 6) checkTaskSetId(rootPool, 2) } + + test("Scheduler does not always schedule tasks on the same workers") { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskScheduler = new TaskSchedulerImpl(sc) + taskScheduler.initialize(new FakeSchedulerBackend) + // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. + var dagScheduler = new DAGScheduler(taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} + override def executorGained(execId: String, host: String) {} + } + + val numFreeCores = 1 + val workerOffers = Seq(new WorkerOffer("executor0", "host0", numFreeCores), + new WorkerOffer("executor1", "host1", numFreeCores)) + // Repeatedly try to schedule a 1-task job, and make sure that it doesn't always + // get scheduled on the same executor. While there is a chance this test will fail + // because the task randomly gets placed on the first executor all 1000 times, the + // probability of that happening is 2^-1000 (so sufficiently small to be considered + // negligible). + val numTrials = 1000 + val selectedExecutorIds = 1.to(numTrials).map { _ => + val taskSet = FakeTask.createTaskSet(1) + taskScheduler.submitTasks(taskSet) + val taskDescriptions = taskScheduler.resourceOffers(workerOffers).flatten + assert(1 === taskDescriptions.length) + taskDescriptions(0).executorId + } + var count = selectedExecutorIds.count(_ == workerOffers(0).executorId) + assert(count > 0) + assert(count < numTrials) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 20f6e503872ac..33cc7588b919c 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -88,7 +88,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("TaskSet with no preferences") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = createTaskSet(1) + val taskSet = FakeTask.createTaskSet(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) // Offer a host with no CPUs @@ -114,7 +114,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("multiple offers with no preferences") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = createTaskSet(3) + val taskSet = FakeTask.createTaskSet(3) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) // First three offers should all find tasks @@ -145,7 +145,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("basic delay scheduling") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) - val taskSet = createTaskSet(4, + val taskSet = FakeTask.createTaskSet(4, Seq(TaskLocation("host1", "exec1")), Seq(TaskLocation("host2", "exec2")), Seq(TaskLocation("host1"), TaskLocation("host2", "exec2")), @@ -190,7 +190,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2"), ("exec3", "host3")) - val taskSet = createTaskSet(5, + val taskSet = FakeTask.createTaskSet(5, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), Seq(TaskLocation("host2")), @@ -229,7 +229,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("delay scheduling with failed hosts") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), ("exec2", "host2")) - val taskSet = createTaskSet(3, + val taskSet = FakeTask.createTaskSet(3, Seq(TaskLocation("host1")), Seq(TaskLocation("host2")), Seq(TaskLocation("host3")) @@ -261,7 +261,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("task result lost") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = createTaskSet(1) + val taskSet = FakeTask.createTaskSet(1) val clock = new FakeClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) @@ -278,7 +278,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { test("repeated failures lead to task set abortion") { sc = new SparkContext("local", "test") val sched = new FakeTaskScheduler(sc, ("exec1", "host1")) - val taskSet = createTaskSet(1) + val taskSet = FakeTask.createTaskSet(1) val clock = new FakeClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) @@ -298,21 +298,6 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { } } - - /** - * Utility method to create a TaskSet, potentially setting a particular sequence of preferred - * locations for each task (given as varargs) if this sequence is not empty. - */ - def createTaskSet(numTasks: Int, prefLocs: Seq[TaskLocation]*): TaskSet = { - if (prefLocs.size != 0 && prefLocs.size != numTasks) { - throw new IllegalArgumentException("Wrong number of task locations") - } - val tasks = Array.tabulate[Task[_]](numTasks) { i => - new FakeTask(i, if (prefLocs.size != 0) prefLocs(i) else Nil) - } - new TaskSet(tasks, 0, 0, 0, null) - } - def createTaskResult(id: Int): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics) From 9aa095711858ce8670e51488f66a3d7c1a821c30 Mon Sep 17 00:00:00 2001 From: Mark Grover Date: Sat, 1 Mar 2014 16:21:22 -0800 Subject: [PATCH 024/397] [SPARK-1150] fix repo location in create script https://spark-project.atlassian.net/browse/SPARK-1150 fix the repo location in create_release script Author: Mark Grover Closes #48 from CodingCat/script_fixes and squashes the following commits: 01f4bf7 [Mark Grover] Fixing some nitpicks d2244d4 [Mark Grover] SPARK-676: Abbreviation in SPARK_MEM but not in SPARK_WORKER_MEMORY --- conf/spark-env.sh.template | 2 +- .../apache/spark/deploy/worker/WorkerArguments.scala | 10 ++++++++-- docs/spark-standalone.md | 4 ++-- 3 files changed, 11 insertions(+), 5 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 6432a566089be..619fc27d53bbe 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -15,7 +15,7 @@ # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports # - SPARK_WORKER_CORES, to set the number of cores to use on this machine -# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) +# - SPARK_WORKER_MEM, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index d35d5be73ff97..52c44196391e3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -18,13 +18,15 @@ package org.apache.spark.deploy.worker import java.lang.management.ManagementFactory +import org.apache.spark.Logging import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** * Command-line parser for the master. */ -private[spark] class WorkerArguments(args: Array[String]) { +private[spark] class WorkerArguments(args: Array[String]) extends Logging { + initLogging() var host = Utils.localHostName() var port = 0 var webUiPort = 8081 @@ -40,9 +42,13 @@ private[spark] class WorkerArguments(args: Array[String]) { if (System.getenv("SPARK_WORKER_CORES") != null) { cores = System.getenv("SPARK_WORKER_CORES").toInt } - if (System.getenv("SPARK_WORKER_MEMORY") != null) { + if (System.getenv("SPARK_WORKER_MEM") != null) { + memory = Utils.memoryStringToMb(System.getenv("SPARK_WORKER_MEM")) + } else if (System.getenv("SPARK_WORKER_MEMORY") != null) { + logWarning("SPARK_WORKER_MEMORY is deprecated. Please use SPARK_WORKER_MEM instead") memory = Utils.memoryStringToMb(System.getenv("SPARK_WORKER_MEMORY")) } + if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt } diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 51fb3a4f7f8c5..a2dec86be11d8 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -104,8 +104,8 @@ You can optionally configure the cluster further by setting environment variable Total number of cores to allow Spark applications to use on the machine (default: all available cores). - SPARK_WORKER_MEMORY - Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GB); note that each application's individual memory is configured using its spark.executor.memory property. + SPARK_WORKER_MEM + Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GB); note that each application's individual memory is configured using its spark.executor.memory property. The old variable SPARK_WORKER_MEMORY has been deprecated. SPARK_WORKER_WEBUI_PORT From ec992e182231da7313d85d10b3d5fd5975c44c8b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 1 Mar 2014 17:15:38 -0800 Subject: [PATCH 025/397] Revert "[SPARK-1150] fix repo location in create script" This reverts commit 9aa095711858ce8670e51488f66a3d7c1a821c30. --- conf/spark-env.sh.template | 2 +- .../apache/spark/deploy/worker/WorkerArguments.scala | 10 ++-------- docs/spark-standalone.md | 4 ++-- 3 files changed, 5 insertions(+), 11 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 619fc27d53bbe..6432a566089be 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -15,7 +15,7 @@ # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports # - SPARK_WORKER_CORES, to set the number of cores to use on this machine -# - SPARK_WORKER_MEM, to set how much memory to use (e.g. 1000m, 2g) +# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index 52c44196391e3..d35d5be73ff97 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -18,15 +18,13 @@ package org.apache.spark.deploy.worker import java.lang.management.ManagementFactory -import org.apache.spark.Logging import org.apache.spark.util.{IntParam, MemoryParam, Utils} /** * Command-line parser for the master. */ -private[spark] class WorkerArguments(args: Array[String]) extends Logging { - initLogging() +private[spark] class WorkerArguments(args: Array[String]) { var host = Utils.localHostName() var port = 0 var webUiPort = 8081 @@ -42,13 +40,9 @@ private[spark] class WorkerArguments(args: Array[String]) extends Logging { if (System.getenv("SPARK_WORKER_CORES") != null) { cores = System.getenv("SPARK_WORKER_CORES").toInt } - if (System.getenv("SPARK_WORKER_MEM") != null) { - memory = Utils.memoryStringToMb(System.getenv("SPARK_WORKER_MEM")) - } else if (System.getenv("SPARK_WORKER_MEMORY") != null) { - logWarning("SPARK_WORKER_MEMORY is deprecated. Please use SPARK_WORKER_MEM instead") + if (System.getenv("SPARK_WORKER_MEMORY") != null) { memory = Utils.memoryStringToMb(System.getenv("SPARK_WORKER_MEMORY")) } - if (System.getenv("SPARK_WORKER_WEBUI_PORT") != null) { webUiPort = System.getenv("SPARK_WORKER_WEBUI_PORT").toInt } diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index a2dec86be11d8..51fb3a4f7f8c5 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -104,8 +104,8 @@ You can optionally configure the cluster further by setting environment variable Total number of cores to allow Spark applications to use on the machine (default: all available cores). - SPARK_WORKER_MEM - Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GB); note that each application's individual memory is configured using its spark.executor.memory property. The old variable SPARK_WORKER_MEMORY has been deprecated. + SPARK_WORKER_MEMORY + Total amount of memory to allow Spark applications to use on the machine, e.g. 1000m, 2g (default: total memory minus 1 GB); note that each application's individual memory is configured using its spark.executor.memory property. SPARK_WORKER_WEBUI_PORT From fe195ae113941766b3921b1e4ec222ed830b5b8f Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sat, 1 Mar 2014 17:24:53 -0800 Subject: [PATCH 026/397] [SPARK-1150] fix repo location in create script (re-open) reopen for https://spark-project.atlassian.net/browse/SPARK-1150 Author: CodingCat Closes #52 from CodingCat/script_fixes and squashes the following commits: fc05a71 [CodingCat] fix repo location in create script --- dev/create-release/create-release.sh | 16 ++++++++-------- 1 file changed, 8 insertions(+), 8 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 7cebace5069f8..d3294f04e3d3a 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -39,8 +39,8 @@ GIT_TAG=v$RELEASE_VERSION # Artifact publishing -git clone https://git-wip-us.apache.org/repos/asf/incubator-spark.git -b $GIT_BRANCH -cd incubator-spark +git clone https://git-wip-us.apache.org/repos/asf/spark.git -b $GIT_BRANCH +cd spark export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" mvn -Pyarn release:clean @@ -59,11 +59,11 @@ mvn -DskipTests \ -Pyarn \ release:perform -rm -rf incubator-spark +rm -rf spark # Source and binary tarballs -git clone https://git-wip-us.apache.org/repos/asf/incubator-spark.git -cd incubator-spark +git clone https://git-wip-us.apache.org/repos/asf/spark.git +cd spark git checkout --force $GIT_TAG release_hash=`git rev-parse HEAD` @@ -71,7 +71,7 @@ rm .gitignore rm -rf .git cd .. -cp -r incubator-spark spark-$RELEASE_VERSION +cp -r spark spark-$RELEASE_VERSION tar cvzf spark-$RELEASE_VERSION.tgz spark-$RELEASE_VERSION echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour --output spark-$RELEASE_VERSION.tgz.asc \ --detach-sig spark-$RELEASE_VERSION.tgz @@ -85,7 +85,7 @@ make_binary_release() { NAME=$1 MAVEN_FLAGS=$2 - cp -r incubator-spark spark-$RELEASE_VERSION-bin-$NAME + cp -r spark spark-$RELEASE_VERSION-bin-$NAME cd spark-$RELEASE_VERSION-bin-$NAME export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" mvn $MAVEN_FLAGS -DskipTests clean package @@ -118,7 +118,7 @@ scp spark* \ $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/ # Docs -cd incubator-spark +cd spark cd docs jekyll build echo "Copying release documentation" From 3a8b698e961ac05d9d53e2bbf0c2844fcb1010d1 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sat, 1 Mar 2014 17:27:54 -0800 Subject: [PATCH 027/397] [SPARK-1100] prevent Spark from overwriting directory silently Thanks for Diana Carroll to report this issue (https://spark-project.atlassian.net/browse/SPARK-1100) the current saveAsTextFile/SequenceFile will overwrite the output directory silently if the directory already exists, this behaviour is not desirable because overwriting the data silently is not user-friendly if the partition number of two writing operation changed, then the output directory will contain the results generated by two runnings My fix includes: add some new APIs with a flag for users to define whether he/she wants to overwrite the directory: if the flag is set to true, then the output directory is deleted first and then written into the new data to prevent the output directory contains results from multiple rounds of running; if the flag is set to false, Spark will throw an exception if the output directory already exists changed JavaAPI part default behaviour is overwriting Two questions should we deprecate the old APIs without such a flag? I noticed that Spark Streaming also called these APIs, I thought we don't need to change the related part in streaming? @tdas Author: CodingCat Closes #11 from CodingCat/SPARK-1100 and squashes the following commits: 6a4e3a3 [CodingCat] code clean ef2d43f [CodingCat] add new test cases and code clean ac63136 [CodingCat] checkOutputSpecs not applicable to FSOutputFormat ec490e8 [CodingCat] prevent Spark from overwriting directory silently and leaving dirty directory --- .../apache/spark/rdd/PairRDDFunctions.scala | 27 +++++++----- .../scala/org/apache/spark/FileSuite.scala | 42 +++++++++++++++++++ 2 files changed, 59 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index d29a1a9881cd4..5aa0b030dbdd8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -30,18 +30,15 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.fs.Path +import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} -import org.apache.hadoop.mapreduce.{Job => NewAPIHadoopJob} -import org.apache.hadoop.mapreduce.{RecordWriter => NewRecordWriter} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job => NewAPIHadoopJob, RecordWriter => NewRecordWriter, JobContext, SparkHadoopMapReduceUtil} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} // SparkHadoopWriter and SparkHadoopMapReduceUtil are actually source files defined in Spark. import org.apache.hadoop.mapred.SparkHadoopWriter -import org.apache.hadoop.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner @@ -604,8 +601,12 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) val job = new NewAPIHadoopJob(conf) job.setOutputKeyClass(keyClass) job.setOutputValueClass(valueClass) + val wrappedConf = new SerializableWritable(job.getConfiguration) - NewFileOutputFormat.setOutputPath(job, new Path(path)) + val outpath = new Path(path) + NewFileOutputFormat.setOutputPath(job, outpath) + val jobFormat = outputFormatClass.newInstance + jobFormat.checkOutputSpecs(job) val formatter = new SimpleDateFormat("yyyyMMddHHmm") val jobtrackerID = formatter.format(new Date()) val stageId = self.id @@ -633,7 +634,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) committer.commitTask(hadoopContext) return 1 } - val jobFormat = outputFormatClass.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. @@ -642,7 +643,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) jobCommitter.setupJob(jobTaskContext) - val count = self.context.runJob(self, writeShard _).sum + self.context.runJob(self, writeShard _) jobCommitter.commitJob(jobTaskContext) } @@ -696,10 +697,10 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * MapReduce job. */ def saveAsHadoopDataset(conf: JobConf) { - val outputFormatClass = conf.getOutputFormat + val outputFormatInstance = conf.getOutputFormat val keyClass = conf.getOutputKeyClass val valueClass = conf.getOutputValueClass - if (outputFormatClass == null) { + if (outputFormatInstance == null) { throw new SparkException("Output format class not set") } if (keyClass == null) { @@ -712,6 +713,12 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + valueClass.getSimpleName + ")") + if (outputFormatInstance.isInstanceOf[FileOutputFormat[_, _]]) { + // FileOutputFormat ignores the filesystem parameter + val ignoredFs = FileSystem.get(conf) + conf.getOutputFormat.checkOutputSpecs(ignoredFs, conf) + } + val writer = new SparkHadoopWriter(conf) writer.preSetup() diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 8ff02aef67aa0..76173608e9f70 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -24,9 +24,11 @@ import scala.io.Source import com.google.common.io.Files import org.apache.hadoop.io._ import org.apache.hadoop.io.compress.DefaultCodec +import org.apache.hadoop.mapred.FileAlreadyExistsException import org.scalatest.FunSuite import org.apache.spark.SparkContext._ +import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat class FileSuite extends FunSuite with LocalSparkContext { @@ -208,4 +210,44 @@ class FileSuite extends FunSuite with LocalSparkContext { assert(rdd.count() === 3) assert(rdd.count() === 3) } + + test ("prevent user from overwriting the empty directory (old Hadoop API)") { + sc = new SparkContext("local", "test") + val tempdir = Files.createTempDir() + val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) + intercept[FileAlreadyExistsException] { + randomRDD.saveAsTextFile(tempdir.getPath) + } + } + + test ("prevent user from overwriting the non-empty directory (old Hadoop API)") { + sc = new SparkContext("local", "test") + val tempdir = Files.createTempDir() + val randomRDD = sc.parallelize(Array((1, "a"), (1, "a"), (2, "b"), (3, "c")), 1) + randomRDD.saveAsTextFile(tempdir.getPath + "/output") + assert(new File(tempdir.getPath + "/output/part-00000").exists() === true) + intercept[FileAlreadyExistsException] { + randomRDD.saveAsTextFile(tempdir.getPath + "/output") + } + } + + test ("prevent user from overwriting the empty directory (new Hadoop API)") { + sc = new SparkContext("local", "test") + val tempdir = Files.createTempDir() + val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + intercept[FileAlreadyExistsException] { + randomRDD.saveAsNewAPIHadoopFile[TextOutputFormat[String, String]](tempdir.getPath) + } + } + + test ("prevent user from overwriting the non-empty directory (new Hadoop API)") { + sc = new SparkContext("local", "test") + val tempdir = Files.createTempDir() + val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + randomRDD.saveAsTextFile(tempdir.getPath + "/output") + assert(new File(tempdir.getPath + "/output/part-00000").exists() === true) + intercept[FileAlreadyExistsException] { + randomRDD.saveAsNewAPIHadoopFile[TextOutputFormat[String, String]](tempdir.getPath) + } + } } From 6fc76e49c19310ec0d6cdf4754271ad09d652576 Mon Sep 17 00:00:00 2001 From: DB Tsai Date: Sun, 2 Mar 2014 00:31:59 -0800 Subject: [PATCH 028/397] Initialized the regVal for first iteration in SGD optimizer Ported from https://github.com/apache/incubator-spark/pull/633 In runMiniBatchSGD, the regVal (for 1st iter) should be initialized as sum of sqrt of weights if it's L2 update; for L1 update, the same logic is followed. It maybe not be important here for SGD since the updater doesn't take the loss as parameter to find the new weights. But it will give us the correct history of loss. However, for LBFGS optimizer we implemented, the correct loss with regVal is crucial to find the new weights. Author: DB Tsai Closes #40 from dbtsai/dbtsai-smallRegValFix and squashes the following commits: 77d47da [DB Tsai] In runMiniBatchSGD, the regVal (for 1st iter) should be initialized as sum of sqrt of weights if it's L2 update; for L1 update, the same logic is followed. --- .../mllib/optimization/GradientDescent.scala | 8 +++- .../spark/mllib/optimization/Updater.scala | 2 + .../optimization/GradientDescentSuite.scala | 41 +++++++++++++++++++ 3 files changed, 50 insertions(+), 1 deletion(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 8e87b98bac061..b967b22e818d3 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -149,7 +149,13 @@ object GradientDescent extends Logging { // Initialize weights as a column vector var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) - var regVal = 0.0 + + /** + * For the first iteration, the regVal will be initialized as sum of sqrt of + * weights if it's L2 update; for L1 update; the same logic is followed. + */ + var regVal = updater.compute( + weights, new DoubleMatrix(initialWeights.length, 1), 0, 1, regParam)._2 for (i <- 1 to numIterations) { // Sample a subset (fraction miniBatchFraction) of the total data diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index 889a03e3e61d2..bf8f731459e99 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -111,6 +111,8 @@ class SquaredL2Updater extends Updater { val step = gradient.mul(thisIterStepSize) // add up both updates from the gradient of the loss (= step) as well as // the gradient of the regularizer (= regParam * weightsOld) + // w' = w - thisIterStepSize * (gradient + regParam * w) + // w' = (1 - thisIterStepSize * regParam) * w - thisIterStepSize * gradient val newWeights = weightsOld.mul(1.0 - thisIterStepSize * regParam).sub(step) (newWeights, 0.5 * pow(newWeights.norm2, 2.0) * regParam) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index a453de6767aa2..631d0e2ad9cdb 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -104,4 +104,45 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa val lossDiff = loss.init.zip(loss.tail).map { case (lhs, rhs) => lhs - rhs } assert(lossDiff.count(_ > 0).toDouble / lossDiff.size > 0.8) } + + test("Test the loss and gradient of first iteration with regularization.") { + + val gradient = new LogisticGradient() + val updater = new SquaredL2Updater() + + // Add a extra variable consisting of all 1.0's for the intercept. + val testData = GradientDescentSuite.generateGDInput(2.0, -1.5, 10000, 42) + val data = testData.map { case LabeledPoint(label, features) => + label -> Array(1.0, features: _*) + } + + val dataRDD = sc.parallelize(data, 2).cache() + + // Prepare non-zero weights + val initialWeightsWithIntercept = Array(1.0, 0.5) + + val regParam0 = 0 + val (newWeights0, loss0) = GradientDescent.runMiniBatchSGD( + dataRDD, gradient, updater, 1, 1, regParam0, 1.0, initialWeightsWithIntercept) + + val regParam1 = 1 + val (newWeights1, loss1) = GradientDescent.runMiniBatchSGD( + dataRDD, gradient, updater, 1, 1, regParam1, 1.0, initialWeightsWithIntercept) + + def compareDouble(x: Double, y: Double, tol: Double = 1E-3): Boolean = { + math.abs(x - y) / (math.abs(y) + 1e-15) < tol + } + + assert(compareDouble( + loss1(0), + loss0(0) + (math.pow(initialWeightsWithIntercept(0), 2) + + math.pow(initialWeightsWithIntercept(1), 2)) / 2), + """For non-zero weights, the regVal should be \frac{1}{2}\sum_i w_i^2.""") + + assert( + compareDouble(newWeights1(0) , newWeights0(0) - initialWeightsWithIntercept(0)) && + compareDouble(newWeights1(1) , newWeights0(1) - initialWeightsWithIntercept(1)), + "The different between newWeights with/without regularization " + + "should be initialWeightsWithIntercept.") + } } From 012bd5fbc97dc40bb61e0e2b9cc97ed0083f37f6 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sun, 2 Mar 2014 00:35:23 -0800 Subject: [PATCH 029/397] Merge the old sbt-launch-lib.bash with the new sbt-launcher jar downloading logic. This allows developers to pass options (such as -D) to sbt. I also modified the SparkBuild to ensure spark specific properties are propagated to forked test JVMs. Author: Michael Armbrust Closes #14 from marmbrus/sbtScripts and squashes the following commits: c008b18 [Michael Armbrust] Merge the old sbt-launch-lib.bash with the new sbt-launcher jar downloading logic. --- LICENSE | 32 +++++++ project/SparkBuild.scala | 3 + sbt/sbt | 153 ++++++++++++++++++++++----------- sbt/sbt-launch-lib.bash | 178 +++++++++++++++++++++++++++++++++++++++ 4 files changed, 315 insertions(+), 51 deletions(-) create mode 100755 sbt/sbt-launch-lib.bash diff --git a/LICENSE b/LICENSE index 1c166d1333614..1c1c2c0255fa9 100644 --- a/LICENSE +++ b/LICENSE @@ -396,3 +396,35 @@ INTERRUPTION) HOWEVER CAUSED AND ON ANY THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. + + +======================================================================== +For sbt and sbt-launch-lib.bash in sbt/: +======================================================================== + +// Generated from http://www.opensource.org/licenses/bsd-license.php +Copyright (c) 2011, Paul Phillips. +All rights reserved. + +Redistribution and use in source and binary forms, with or without +modification, are permitted provided that the following conditions are met: + + * Redistributions of source code must retain the above copyright notice, + this list of conditions and the following disclaimer. + * Redistributions in binary form must reproduce the above copyright notice, + this list of conditions and the following disclaimer in the documentation + and/or other materials provided with the distribution. + * Neither the name of the author nor the names of its contributors may be + used to endorse or promote products derived from this software without + specific prior written permission. + +THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS "AS IS" +AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT LIMITED TO, THE +IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR A PARTICULAR PURPOSE +ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT HOLDER OR CONTRIBUTORS BE +LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL, SPECIAL, EXEMPLARY, OR CONSEQUENTIAL +DAMAGES (INCLUDING, BUT NOT LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; +LOSS OF USE, DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY +THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT (INCLUDING +NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE OF THIS SOFTWARE, +EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c402415742b5e..86340ada958e9 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._ @@ -140,6 +142,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 8472dce589bcc..3ffa4ed9ab5a7 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,51 +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. -# - -# 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: Sun, 2 Mar 2014 00:48:50 -0800 Subject: [PATCH 030/397] Update io.netty from 4.0.13 Final to 4.0.17.Final This update contains a lot of bug fixes and some new perf improvements. It is also binary compatible with the current 4.0.13.Final For more information: http://netty.io/news/2014/02/25/4-0-17-Final.html Author: Binh Nguyen Author: Binh Nguyen Closes #41 from ngbinh/master and squashes the following commits: a9498f4 [Binh Nguyen] update io.netty to 4.0.17.Final --- pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 21060ee69c041..5e8f58ee154c3 100644 --- a/pom.xml +++ b/pom.xml @@ -298,7 +298,7 @@ io.netty netty-all - 4.0.13.Final + 4.0.17.Final org.apache.derby diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 86340ada958e9..e004f90033fdc 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -214,7 +214,7 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "io.netty" % "netty-all" % "4.0.13.Final", + "io.netty" % "netty-all" % "4.0.17.Final", "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"), From 1fd2bfd3dd6c27a54880f0d7a658b39f358aa804 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 2 Mar 2014 01:00:16 -0800 Subject: [PATCH 031/397] Remove remaining references to incubation This removes some loose ends not caught by the other (incubating -> tlp) patches. @markhamstra this updates the version as you mentioned earlier. Author: Patrick Wendell Closes #51 from pwendell/tlp and squashes the following commits: d553b1b [Patrick Wendell] Remove remaining references to incubation --- assembly/pom.xml | 4 ++-- bagel/pom.xml | 4 ++-- core/pom.xml | 4 ++-- dev/audit-release/audit_release.py | 8 ++------ ec2/README | 2 +- ec2/spark_ec2.py | 2 +- examples/pom.xml | 4 ++-- external/flume/pom.xml | 4 ++-- external/kafka/pom.xml | 4 ++-- external/mqtt/pom.xml | 4 ++-- external/twitter/pom.xml | 4 ++-- external/zeromq/pom.xml | 4 ++-- graphx/pom.xml | 2 +- mllib/pom.xml | 4 ++-- pom.xml | 30 +++++++++++++++--------------- project/SparkBuild.scala | 10 +++++----- repl/pom.xml | 4 ++-- streaming/pom.xml | 4 ++-- tools/pom.xml | 4 ++-- yarn/alpha/pom.xml | 2 +- yarn/pom.xml | 2 +- yarn/stable/pom.xml | 2 +- 22 files changed, 54 insertions(+), 58 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 82396040251d3..7ce30179e9ca2 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -21,14 +21,14 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml org.apache.spark spark-assembly_2.10 Spark Project Assembly - http://spark.incubator.apache.org/ + http://spark.apache.org/ ${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar diff --git a/bagel/pom.xml b/bagel/pom.xml index 6155ab5f20155..355f437c5b16a 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml @@ -29,7 +29,7 @@ spark-bagel_2.10 jar Spark Project Bagel - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/core/pom.xml b/core/pom.xml index d3a81d564c2e8..6626d7c1b96b5 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml @@ -29,7 +29,7 @@ spark-core_2.10 jar Spark Project Core - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index 4408658f5e33f..52c367d9b030d 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -31,10 +31,10 @@ import urllib2 ## Fill in release details here: -RELEASE_URL = "http://people.apache.org/~pwendell/spark-0.9.0-incubating-rc5/" +RELEASE_URL = "http://people.apache.org/~pwendell/spark-1.0.0-rc1/" RELEASE_KEY = "9E4FE3AF" RELEASE_REPOSITORY = "https://repository.apache.org/content/repositories/orgapachespark-1006/" -RELEASE_VERSION = "0.9.0-incubating" +RELEASE_VERSION = "1.0.0" SCALA_VERSION = "2.10.3" SCALA_BINARY_VERSION = "2.10" ## @@ -191,10 +191,6 @@ def ensure_path_not_present(x): test("NOTICE" in base_files, "Tarball contains NOTICE file") test("LICENSE" in base_files, "Tarball contains LICENSE file") - os.chdir(os.path.join(WORK_DIR, dir_name)) - readme = "".join(open("README.md").readlines()) - disclaimer_part = "is an effort undergoing incubation" - test(disclaimer_part in readme, "README file contains disclaimer") os.chdir(WORK_DIR) for artifact in artifacts: diff --git a/ec2/README b/ec2/README index 433da37b4c37c..72434f24bf98d 100644 --- a/ec2/README +++ b/ec2/README @@ -1,4 +1,4 @@ This folder contains a script, spark-ec2, for launching Spark clusters on Amazon EC2. Usage instructions are available online at: -http://spark.incubator.apache.org/docs/latest/ec2-scripts.html +http://spark.apache.org/docs/latest/ec2-scripts.html diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index b0512ca891ad6..25e85381896b0 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -73,7 +73,7 @@ def parse_args(): parser.add_option("-v", "--spark-version", default="0.9.0", help="Version of Spark to use: 'X.Y.Z' or a specific git hash") parser.add_option("--spark-git-repo", - default="https://github.com/apache/incubator-spark", + default="https://github.com/apache/spark", help="Github repo from which to checkout supplied commit hash") parser.add_option("--hadoop-major-version", default="1", help="Major version of Hadoop (default: 1)") diff --git a/examples/pom.xml b/examples/pom.xml index 12a11821a4947..709907cb7ed87 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml @@ -29,7 +29,7 @@ spark-examples_2.10 jar Spark Project Examples - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/external/flume/pom.xml b/external/flume/pom.xml index a0e8b84514ef6..95ff872196dd0 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml @@ -29,7 +29,7 @@ spark-streaming-flume_2.10 jar Spark Project External Flume - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index fb37cd79884c8..979eb0ca624bd 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml @@ -29,7 +29,7 @@ spark-streaming-kafka_2.10 jar Spark Project External Kafka - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index cfa1870e982fe..2c476b402e83d 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml @@ -29,7 +29,7 @@ spark-streaming-mqtt_2.10 jar Spark Project External MQTT - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 077f88dc59bab..a443459594710 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml @@ -29,7 +29,7 @@ spark-streaming-twitter_2.10 jar Spark Project External Twitter - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 4c68294b7b5af..a40e55876e640 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../../pom.xml @@ -29,7 +29,7 @@ spark-streaming-zeromq_2.10 jar Spark Project External ZeroMQ - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/graphx/pom.xml b/graphx/pom.xml index 4823ed1d4eaec..2b4d674221b18 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml diff --git a/mllib/pom.xml b/mllib/pom.xml index 9a61d7c3e46c0..c0e745dadb22c 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml @@ -29,7 +29,7 @@ spark-mllib_2.10 jar Spark Project ML Library - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/pom.xml b/pom.xml index 5e8f58ee154c3..39995396b38f4 100644 --- a/pom.xml +++ b/pom.xml @@ -25,10 +25,10 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT pom Spark Project Parent POM - http://spark.incubator.apache.org/ + http://spark.apache.org/ Apache 2.0 License @@ -37,9 +37,9 @@ - scm:git:git@github.com:apache/incubator-spark.git - scm:git:https://git-wip-us.apache.org/repos/asf/incubator-spark.git - scm:git:git@github.com:apache/incubator-spark.git + scm:git:git@github.com:apache/spark.git + scm:git:https://git-wip-us.apache.org/repos/asf/spark.git + scm:git:git@github.com:apache/spark.git HEAD @@ -49,7 +49,7 @@ matei.zaharia@gmail.com http://www.cs.berkeley.edu/~matei Apache Software Foundation - http://spark.incubator.apache.org + http://spark.apache.org @@ -64,23 +64,23 @@ Dev Mailing List - dev@spark.incubator.apache.org - dev-subscribe@spark.incubator.apache.org - dev-unsubscribe@spark.incubator.apache.org + dev@spark.apache.org + dev-subscribe@spark.apache.org + dev-unsubscribe@spark.apache.org User Mailing List - user@spark.incubator.apache.org - user-subscribe@spark.incubator.apache.org - user-unsubscribe@spark.incubator.apache.org + user@spark.apache.org + user-subscribe@spark.apache.org + user-unsubscribe@spark.apache.org Commits Mailing List - commits@spark.incubator.apache.org - commits-subscribe@spark.incubator.apache.org - commits-unsubscribe@spark.incubator.apache.org + commits@spark.apache.org + commits-subscribe@spark.apache.org + commits-unsubscribe@spark.apache.org diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e004f90033fdc..399886ca1b821 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -125,7 +125,7 @@ object SparkBuild extends Build { def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.apache.spark", - version := "1.0.0-incubating-SNAPSHOT", + version := "1.0.0-SNAPSHOT", scalaVersion := "2.10.3", scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-target:" + SCALAC_JVM_VERSION), @@ -174,7 +174,7 @@ object SparkBuild extends Build { apache 13 - http://spark.incubator.apache.org/ + http://spark.apache.org/ Apache 2.0 License @@ -183,8 +183,8 @@ object SparkBuild extends Build { - scm:git:git@github.com:apache/incubator-spark.git - scm:git:git@github.com:apache/incubator-spark.git + scm:git:git@github.com:apache/spark.git + scm:git:git@github.com:apache/spark.git @@ -193,7 +193,7 @@ object SparkBuild extends Build { matei.zaharia@gmail.com http://www.cs.berkeley.edu/~matei Apache Software Foundation - http://spark.incubator.apache.org + http://spark.apache.org diff --git a/repl/pom.xml b/repl/pom.xml index 4c5f9720c802a..3a6baf5c42f6a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml @@ -29,7 +29,7 @@ spark-repl_2.10 jar Spark Project REPL - http://spark.incubator.apache.org/ + http://spark.apache.org/ /usr/share/spark diff --git a/streaming/pom.xml b/streaming/pom.xml index acc1d23c7dc20..2f358c99380a6 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -21,7 +21,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml @@ -29,7 +29,7 @@ spark-streaming_2.10 jar Spark Project Streaming - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/tools/pom.xml b/tools/pom.xml index a27f0db6e5628..67f2d780f312e 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml @@ -28,7 +28,7 @@ spark-tools_2.10 jar Spark Project Tools - http://spark.incubator.apache.org/ + http://spark.apache.org/ diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index b026128980cb8..e076ca1d44b97 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml diff --git a/yarn/pom.xml b/yarn/pom.xml index c0e133dd603b1..be51679292933 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -20,7 +20,7 @@ org.apache.spark spark-parent - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 7c312206d16d3..0780f251b595c 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -20,7 +20,7 @@ org.apache.spark yarn-parent_2.10 - 1.0.0-incubating-SNAPSHOT + 1.0.0-SNAPSHOT ../pom.xml From 46bcb9551eb918ac4a31cd4cca924b432f6dc352 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 2 Mar 2014 01:00:42 -0800 Subject: [PATCH 032/397] SPARK-1137: Make ZK PersistenceEngine not crash for wrong serialVersionUID Previously, ZooKeeperPersistenceEngine would crash the whole Master process if there was stored data from a prior Spark version. Now, we just delete these files. Author: Aaron Davidson Closes #4 from aarondav/zookeeper2 and squashes the following commits: fa8b40f [Aaron Davidson] SPARK-1137: Make ZK PersistenceEngine not crash for wrong serialVersionUID --- .../master/ZooKeeperPersistenceEngine.scala | 18 +++++++++++++----- 1 file changed, 13 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 939006239d2b1..5413ff671ad8d 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -64,11 +64,11 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) override def readPersistedData(): (Seq[ApplicationInfo], Seq[DriverInfo], Seq[WorkerInfo]) = { val sortedFiles = zk.getChildren().forPath(WORKING_DIR).toList.sorted val appFiles = sortedFiles.filter(_.startsWith("app_")) - val apps = appFiles.map(deserializeFromFile[ApplicationInfo]) + val apps = appFiles.map(deserializeFromFile[ApplicationInfo]).flatten val driverFiles = sortedFiles.filter(_.startsWith("driver_")) - val drivers = driverFiles.map(deserializeFromFile[DriverInfo]) + val drivers = driverFiles.map(deserializeFromFile[DriverInfo]).flatten val workerFiles = sortedFiles.filter(_.startsWith("worker_")) - val workers = workerFiles.map(deserializeFromFile[WorkerInfo]) + val workers = workerFiles.map(deserializeFromFile[WorkerInfo]).flatten (apps, drivers, workers) } @@ -78,10 +78,18 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) zk.create().withMode(CreateMode.PERSISTENT).forPath(path, serialized) } - def deserializeFromFile[T](filename: String)(implicit m: Manifest[T]): T = { + def deserializeFromFile[T](filename: String)(implicit m: Manifest[T]): Option[T] = { val fileData = zk.getData().forPath(WORKING_DIR + "/" + filename) val clazz = m.runtimeClass.asInstanceOf[Class[T]] val serializer = serialization.serializerFor(clazz) - serializer.fromBinary(fileData).asInstanceOf[T] + try { + Some(serializer.fromBinary(fileData).asInstanceOf[T]) + } catch { + case e: Exception => { + logWarning("Exception while reading persisted file, deleting", e) + zk.delete().forPath(WORKING_DIR + "/" + filename) + None + } + } } } From 353ac6b4fa8f0740a7faabc1309022420c7b52fc Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 2 Mar 2014 14:27:19 -0800 Subject: [PATCH 033/397] Ignore RateLimitedOutputStreamSuite for now. This test has been flaky. We can re-enable it after @tdas has a chance to look at it. Author: Reynold Xin Closes #54 from rxin/ratelimit and squashes the following commits: 1a12198 [Reynold Xin] Ignore RateLimitedOutputStreamSuite for now. --- .../spark/streaming/util/RateLimitedOutputStreamSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala index 15f13d5b19946..e5bf6d70db5f9 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala @@ -29,7 +29,7 @@ class RateLimitedOutputStreamSuite extends FunSuite { System.nanoTime - start } - test("write") { + ignore("write") { val underlying = new ByteArrayOutputStream val data = "X" * 41000 val stream = new RateLimitedOutputStream(underlying, 10000) From fd31adbf27d824f00f62646e13c23f632d1b77d3 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 2 Mar 2014 14:27:53 -0800 Subject: [PATCH 034/397] SPARK-1084.2 (resubmitted) (Ported from https://github.com/apache/incubator-spark/pull/650 ) This adds one more change though, to fix the scala version warning introduced by json4s recently. Author: Sean Owen Closes #32 from srowen/SPARK-1084.2 and squashes the following commits: 9240abd [Sean Owen] Avoid scala version conflict in scalap induced by json4s dependency 1561cec [Sean Owen] Remove "exclude *" dependencies that are causing Maven warnings, and that are apparently unneeded anyway --- core/pom.xml | 9 +++++++++ examples/pom.xml | 8 -------- external/flume/pom.xml | 4 ---- pom.xml | 37 ++++--------------------------------- project/SparkBuild.scala | 20 +++++++++----------- streaming/pom.xml | 5 ----- 6 files changed, 22 insertions(+), 61 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 6626d7c1b96b5..2afd25082534e 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -125,6 +125,15 @@ org.json4s json4s-jackson_${scala.binary.version} 3.2.6 + + + + org.scala-lang + scalap + + it.unimi.dsi diff --git a/examples/pom.xml b/examples/pom.xml index 709907cb7ed87..d952e2ca4e5dc 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -169,14 +169,6 @@ org.apache.cassandra.deps avro - - org.sonatype.sisu.inject - * - - - org.xerial.snappy - * - diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 95ff872196dd0..bc00ab4573c39 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -53,10 +53,6 @@ org.jboss.netty netty - - org.xerial.snappy - * - diff --git a/pom.xml b/pom.xml index 39995396b38f4..7e04efaa9cefc 100644 --- a/pom.xml +++ b/pom.xml @@ -310,6 +310,9 @@ net.liftweb lift-json_${scala.binary.version} 2.5.1 + org.scala-lang @@ -374,7 +377,7 @@ 3.1 test - + org.mockito mockito-all 1.8.5 @@ -416,14 +419,6 @@ org.jboss.netty netty - - org.codehaus.jackson - * - - - org.sonatype.sisu.inject - * - commons-logging commons-logging @@ -448,14 +443,6 @@ org.jboss.netty netty - - org.codehaus.jackson - * - - - org.sonatype.sisu.inject - * - @@ -471,14 +458,6 @@ org.jboss.netty netty - - org.codehaus.jackson - * - - - org.sonatype.sisu.inject - * - @@ -495,14 +474,6 @@ org.jboss.netty netty - - org.codehaus.jackson - * - - - org.sonatype.sisu.inject - * - diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 399886ca1b821..d45f6773fad78 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -243,13 +243,11 @@ object SparkBuild extends Build { val slf4jVersion = "1.7.5" - val excludeCglib = ExclusionRule(organization = "org.sonatype.sisu.inject") - val excludeJackson = ExclusionRule(organization = "org.codehaus.jackson") val excludeNetty = ExclusionRule(organization = "org.jboss.netty") val excludeAsm = ExclusionRule(organization = "asm") - val excludeSnappy = ExclusionRule(organization = "org.xerial.snappy") val excludeCommonsLogging = ExclusionRule(organization = "commons-logging") val excludeSLF4J = ExclusionRule(organization = "org.slf4j") + val excludeScalap = ExclusionRule(organization = "org.scala-lang", artifact = "scalap") def coreSettings = sharedSettings ++ Seq( name := "spark-core", @@ -273,13 +271,13 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", - "org.json4s" %% "json4s-jackson" % "3.2.6", + "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib, excludeCommonsLogging, excludeSLF4J), + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J), "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", @@ -316,7 +314,7 @@ object SparkBuild extends Build { exclude("io.netty", "netty") exclude("jline","jline") exclude("org.apache.cassandra.deps", "avro") - excludeAll(excludeSnappy, excludeCglib, excludeSLF4J) + excludeAll(excludeSLF4J) ) ) ++ assemblySettings ++ extraAssemblySettings @@ -379,10 +377,10 @@ object SparkBuild extends Build { def yarnEnabledSettings = Seq( libraryDependencies ++= Seq( // Exclude rule required for all ? - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), - "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeJackson, excludeNetty, excludeAsm, excludeCglib) + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeNetty, excludeAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm) ) ) @@ -428,7 +426,7 @@ object SparkBuild extends Build { def flumeSettings() = sharedSettings ++ Seq( name := "spark-streaming-flume", libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty, excludeSnappy) + "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty) ) ) diff --git a/streaming/pom.xml b/streaming/pom.xml index 2f358c99380a6..1f3366e25f6a9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -55,11 +55,6 @@ org.eclipse.jetty jetty-server - - org.codehaus.jackson - jackson-mapper-asl - 1.9.11 - org.scala-lang scala-library From c3f5e075335a65ea522b2f76716921ec056c52ed Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 2 Mar 2014 15:18:19 -0800 Subject: [PATCH 035/397] SPARK-1121: Include avro for yarn-alpha builds This lets us explicitly include Avro based on a profile for 0.23.X builds. It makes me sad how convoluted it is to express this logic in Maven. @tgraves and @sryza curious if this works for you. I'm also considering just reverting to how it was before. The only real problem was that Spark advertised a dependency on Avro even though it only really depends transitively on Avro through other deps. Author: Patrick Wendell Closes #49 from pwendell/avro-build-fix and squashes the following commits: 8d6ee92 [Patrick Wendell] SPARK-1121: Add avro to yarn-alpha profile --- bagel/pom.xml | 14 ++++++++++++++ core/pom.xml | 14 ++++++++++++++ docs/building-with-maven.md | 4 ---- examples/pom.xml | 14 ++++++++++++++ external/flume/pom.xml | 14 ++++++++++++++ external/kafka/pom.xml | 14 ++++++++++++++ external/mqtt/pom.xml | 14 ++++++++++++++ external/twitter/pom.xml | 14 ++++++++++++++ external/zeromq/pom.xml | 14 ++++++++++++++ graphx/pom.xml | 14 ++++++++++++++ mllib/pom.xml | 14 ++++++++++++++ pom.xml | 15 +++++++++++++++ repl/pom.xml | 14 ++++++++++++++ streaming/pom.xml | 14 ++++++++++++++ tools/pom.xml | 14 ++++++++++++++ yarn/alpha/pom.xml | 14 ++++++++++++++ yarn/pom.xml | 17 +++++++++-------- yarn/stable/pom.xml | 14 ++++++++++++++ 18 files changed, 234 insertions(+), 12 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 355f437c5b16a..41aacbd88a7d7 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -31,6 +31,20 @@ Spark Project Bagel http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/core/pom.xml b/core/pom.xml index 2afd25082534e..99c841472b3eb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -31,6 +31,20 @@ Spark Project Core http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.hadoop diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 40cac8eb4f0db..ded12926885b9 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -76,7 +76,3 @@ The maven build includes support for building a Debian package containing the as $ mvn -Pdeb -DskipTests clean package The debian package can then be found under assembly/target. We added the short commit hash to the file name so that we can distinguish individual packages built for SNAPSHOT versions. - -## A note about Hadoop version 0.23.x - -For building spark with hadoop 0.23.x and also yarn, you will have to manually add a dependency on avro (org.apache.avro, avro, 1.7.4). diff --git a/examples/pom.xml b/examples/pom.xml index d952e2ca4e5dc..3aba343f4cf50 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -31,6 +31,20 @@ Spark Project Examples http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + apache-repo diff --git a/external/flume/pom.xml b/external/flume/pom.xml index bc00ab4573c39..8783aea3e4a5b 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -31,6 +31,20 @@ Spark Project External Flume http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 979eb0ca624bd..79dc38f9844a0 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -31,6 +31,20 @@ Spark Project External Kafka http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 2c476b402e83d..06c751df7f9c8 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -31,6 +31,20 @@ Spark Project External MQTT http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + mqtt-repo diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index a443459594710..37bb4fad64f68 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -31,6 +31,20 @@ Spark Project External Twitter http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index a40e55876e640..65ec0e26da881 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -31,6 +31,20 @@ Spark Project External ZeroMQ http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/graphx/pom.xml b/graphx/pom.xml index 2b4d674221b18..5b54dd27efb44 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -31,6 +31,20 @@ Spark Project GraphX http://spark-project.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/mllib/pom.xml b/mllib/pom.xml index c0e745dadb22c..760a2a85d5ffa 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -31,6 +31,20 @@ Spark Project ML Library http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/pom.xml b/pom.xml index 7e04efaa9cefc..7e28d7c1941bb 100644 --- a/pom.xml +++ b/pom.xml @@ -425,6 +425,21 @@ + + org.apache.avro + avro + 1.7.4 + + + org.jboss.netty + netty + + + io.netty + netty + + + net.java.dev.jets3t jets3t diff --git a/repl/pom.xml b/repl/pom.xml index 3a6baf5c42f6a..aa01a1760285a 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -31,6 +31,20 @@ Spark Project REPL http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + /usr/share/spark root diff --git a/streaming/pom.xml b/streaming/pom.xml index 1f3366e25f6a9..91d6a1375a18c 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -31,6 +31,20 @@ Spark Project Streaming http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + apache-repo diff --git a/tools/pom.xml b/tools/pom.xml index 67f2d780f312e..b8dd255d40ac4 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -30,6 +30,20 @@ Spark Project Tools http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index e076ca1d44b97..bfe12ecec0c09 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -24,6 +24,20 @@ ../pom.xml + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark spark-yarn-alpha_2.10 jar diff --git a/yarn/pom.xml b/yarn/pom.xml index be51679292933..35e31760c1f02 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -52,14 +52,6 @@ hadoop-client ${yarn.version} - - org.apache.avro - avro - - - org.apache.avro - avro-ipc - org.scalatest scalatest_${scala.binary.version} @@ -78,6 +70,15 @@ alpha + + + + + org.apache.avro + avro + + diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 0780f251b595c..9d68603251d1c 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -24,6 +24,20 @@ ../pom.xml + + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark spark-yarn_2.10 jar From 55a4f11b5064650024bb13c68639665394c03a0c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 2 Mar 2014 18:19:01 -0800 Subject: [PATCH 036/397] Add Jekyll tag to isolate "production-only" doc components. Author: Patrick Wendell Closes #56 from pwendell/jekyll-prod and squashes the following commits: 1bdc3a8 [Patrick Wendell] Add Jekyll tag to isolate "production-only" doc components. --- dev/create-release/create-release.sh | 2 +- docs/README.md | 19 ++++++++++++++++--- docs/_layouts/global.html | 4 ++-- docs/_plugins/production_tag.rb | 14 ++++++++++++++ 4 files changed, 33 insertions(+), 6 deletions(-) create mode 100644 docs/_plugins/production_tag.rb diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index d3294f04e3d3a..b9088eac376a2 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -120,7 +120,7 @@ scp spark* \ # Docs cd spark cd docs -jekyll build +PRODUCTION=1 jekyll build echo "Copying release documentation" rc_docs_folder=${rc_folder}-docs rsync -r _site/* $USER_NAME@people.apache.org /home/$USER_NAME/public_html/$rc_docs_folder diff --git a/docs/README.md b/docs/README.md index cac65d97e488b..0678fc5c86706 100644 --- a/docs/README.md +++ b/docs/README.md @@ -10,9 +10,22 @@ We include the Spark documentation as part of the source (as opposed to using a In this directory you will find textfiles formatted using Markdown, with an ".md" suffix. You can read those text files directly if you want. Start with index.md. -To make things quite a bit prettier and make the links easier to follow, generate the html version of the documentation based on the src directory by running `jekyll build` in the docs directory. Use the command `SKIP_SCALADOC=1 jekyll build` to skip building and copying over the scaladoc which can be timely. To use the `jekyll` command, you will need to have Jekyll installed, the easiest way to do this is via a Ruby Gem, see the [jekyll installation instructions](http://jekyllrb.com/docs/installation). This will create a directory called _site containing index.html as well as the rest of the compiled files. Read more about Jekyll at https://github.com/mojombo/jekyll/wiki. - -In addition to generating the site as html from the markdown files, jekyll can serve up the site via a webserver. To build and run a local webserver use the command `jekyll serve` (or the faster variant `SKIP_SCALADOC=1 jekyll serve`), which runs the webserver on port 4000, then visit the site at http://localhost:4000. +The markdown code can be compiled to HTML using the +[Jekyll tool](http://jekyllrb.com). +To use the `jekyll` command, you will need to have Jekyll installed. +The easiest way to do this is via a Ruby Gem, see the +[jekyll installation instructions](http://jekyllrb.com/docs/installation). +Compiling the site with Jekyll will create a directory called +_site containing index.html as well as the rest of the compiled files. + +You can modify the default Jekyll build as follows: + + # Skip generating API docs (which takes a while) + $ SKIP_SCALADOC=1 jekyll build + # Serve content locally on port 4000 + $ jekyll serve --watch + # Build the site with extra features used on the live page + $ PRODUCTION=1 jekyll build ## Pygments diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index ebb58e8b9af79..49fd78ca98655 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -24,9 +24,9 @@ + {% production %} + {% endproduction %} diff --git a/docs/_plugins/production_tag.rb b/docs/_plugins/production_tag.rb new file mode 100644 index 0000000000000..9f870cf2137af --- /dev/null +++ b/docs/_plugins/production_tag.rb @@ -0,0 +1,14 @@ +module Jekyll + class ProductionTag < Liquid::Block + + def initialize(tag_name, markup, tokens) + super + end + + def render(context) + if ENV['PRODUCTION'] then super else "" end + end + end +end + +Liquid::Template.register_tag('production', Jekyll::ProductionTag) From 2b53447f325fa7adcfb9c69fd824467bf420af04 Mon Sep 17 00:00:00 2001 From: Aaron Kimball Date: Sun, 2 Mar 2014 23:26:47 -0800 Subject: [PATCH 037/397] SPARK-1173. Improve scala streaming docs. Clarify imports to add implicit conversions to DStream and fix other small typos in the streaming intro documentation. Tested by inspecting output via a local jekyll server, c&p'ing the scala commands into a spark terminal. Author: Aaron Kimball Closes #64 from kimballa/spark-1173-streaming-docs and squashes the following commits: 6fbff0e [Aaron Kimball] SPARK-1173. Improve scala streaming docs. --- docs/streaming-programming-guide.md | 38 +++++++++++++++++++++++++---- 1 file changed, 33 insertions(+), 5 deletions(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 57e88581616a2..0cc572d1fde36 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -58,11 +58,21 @@ do is as follows.
    +First, we import the names of the Spark Streaming classes, and some implicit +conversions from StreamingContext into our environment, to add useful methods to +other classes we need (like DStream). -First, we create a -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) object, -which is the main entry point for all streaming -functionality. Besides Spark's configuration, we specify that any DStream will be processed +[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) is the +main entry point for all streaming functionality. + +{% highlight scala %} +import org.apache.spark.streaming._ +import org.apache.spark.streaming.StreamingContext._ +{% endhighlight %} + +Then we create a +[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) object. +Besides Spark's configuration, we specify that any DStream will be processed in 1 second batches. {% highlight scala %} @@ -98,7 +108,7 @@ val pairs = words.map(word => (word, 1)) val wordCounts = pairs.reduceByKey(_ + _) // Print a few of the counts to the console -wordCount.print() +wordCounts.print() {% endhighlight %} The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word, @@ -262,6 +272,24 @@ Time: 1357008430000 ms +If you plan to run the Scala code for Spark Streaming-based use cases in the Spark +shell, you should start the shell with the SparkConfiguration pre-configured to +discard old batches periodically: + +{% highlight bash %} +$ SPARK_JAVA_OPTS=-Dspark.cleaner.ttl=10000 bin/spark-shell +{% endhighlight %} + +... and create your StreamingContext by wrapping the existing interactive shell +SparkContext object, `sc`: + +{% highlight scala %} +val ssc = new StreamingContext(sc, Seconds(1)) +{% endhighlight %} + +When working with the shell, you may also need to send a `^D` to your netcat session +to force the pipeline to print the word counts to the console at the sink. + *************************************************************************************************** # Basics From f65c1f38eb7ed99a578a5430831a4a2c1d774e7a Mon Sep 17 00:00:00 2001 From: Aaron Kimball Date: Sun, 2 Mar 2014 23:48:48 -0800 Subject: [PATCH 038/397] SPARK-1173. (#2) Fix typo in Java streaming example. Companion commit to pull request #64, fix the typo on the Java side of the docs. Author: Aaron Kimball Closes #65 from kimballa/spark-1173-java-doc-update and squashes the following commits: 8ce11d3 [Aaron Kimball] SPARK-1173. (#2) Fix typo in Java streaming example. --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 0cc572d1fde36..2a56cf07d0cfc 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -188,7 +188,7 @@ JavaPairDStream wordCounts = pairs.reduceByKey( return i1 + i2; } }); -wordCount.print(); // Print a few of the counts to the console +wordCounts.print(); // Print a few of the counts to the console {% endhighlight %} The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word, From 369aad6f9e25d49b0d32f37e5069b6d4e41a59d9 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 3 Mar 2014 14:39:49 -0800 Subject: [PATCH 039/397] Removed accidentally checked in comment It looks like this comment was added a while ago by @mridulm as part of a merge and was accidentally checked in. We should remove it. Author: Kay Ousterhout Closes #61 from kayousterhout/remove_comment and squashes the following commits: 0b2b3f2 [Kay Ousterhout] Removed accidentally checked in comment --- .../scala/org/apache/spark/network/ConnectionManager.scala | 3 --- 1 file changed, 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 3dd82bee0b5fd..a7f20f8c51a5a 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -550,9 +550,6 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi newConnection } - // I removed the lookupKey stuff as part of merge ... should I re-add it ? We did not find it - // useful in our test-env ... If we do re-add it, we should consistently use it everywhere I - // guess ? val connection = connectionsById.getOrElseUpdate(connectionManagerId, startNewConnection()) message.senderAddress = id.toSocketAddress() logDebug("Sending [" + message + "] to [" + connectionManagerId + "]") From 9d225a91043ac92a0e727ba281b10c250a945614 Mon Sep 17 00:00:00 2001 From: Chen Chao Date: Mon, 3 Mar 2014 14:41:25 -0800 Subject: [PATCH 040/397] update proportion of memory The default value of "spark.storage.memoryFraction" has been changed from 0.66 to 0.6 . So it should be 60% of the memory to cache while 40% used for task execution. Author: Chen Chao Closes #66 from CrazyJvm/master and squashes the following commits: 0f84d86 [Chen Chao] update proportion of memory --- docs/tuning.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/tuning.md b/docs/tuning.md index 704778681cb8f..26ff1325bb59c 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -163,8 +163,8 @@ their work directories), *not* on your driver program. **Cache Size Tuning** One important configuration parameter for GC is the amount of memory that should be used for caching RDDs. -By default, Spark uses 66% of the configured executor memory (`spark.executor.memory` or `SPARK_MEM`) to -cache RDDs. This means that 33% of memory is available for any objects created during task execution. +By default, Spark uses 60% of the configured executor memory (`spark.executor.memory` or `SPARK_MEM`) to +cache RDDs. This means that 40% of memory is available for any objects created during task execution. In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of memory, lowering this value will help reduce the memory consumption. To change this to say 50%, you can call From b55cade853003d86356a50c6dba82210c8adb667 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 3 Mar 2014 16:12:00 -0800 Subject: [PATCH 041/397] Remove the remoteFetchTime metric. This metric is confusing: it adds up all of the time to fetch shuffle inputs, but fetches often happen in parallel, so remoteFetchTime can be much longer than the task execution time. @squito it looks like you added this metric -- do you have a use case for it? cc @shivaram -- I know you've looked at the shuffle performance a lot so chime in here if this metric has turned out to be useful for you! Author: Kay Ousterhout Closes #62 from kayousterhout/remove_fetch_variable and squashes the following commits: 43341eb [Kay Ousterhout] Remote the remoteFetchTime metric. --- .../scala/org/apache/spark/BlockStoreShuffleFetcher.scala | 1 - .../main/scala/org/apache/spark/executor/TaskMetrics.scala | 7 ------- .../main/scala/org/apache/spark/scheduler/JobLogger.scala | 1 - .../org/apache/spark/storage/BlockFetcherIterator.scala | 4 ---- .../org/apache/spark/scheduler/SparkListenerSuite.scala | 1 - 5 files changed, 14 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala index 754b46a4c7df2..a67392441ed29 100644 --- a/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/BlockStoreShuffleFetcher.scala @@ -79,7 +79,6 @@ private[spark] class BlockStoreShuffleFetcher extends ShuffleFetcher with Loggin val completionIter = CompletionIterator[T, Iterator[T]](itr, { val shuffleMetrics = new ShuffleReadMetrics shuffleMetrics.shuffleFinishTime = System.currentTimeMillis - shuffleMetrics.remoteFetchTime = blockFetcherItr.remoteFetchTime shuffleMetrics.fetchWaitTime = blockFetcherItr.fetchWaitTime shuffleMetrics.remoteBytesRead = blockFetcherItr.remoteBytesRead shuffleMetrics.totalBlocksFetched = blockFetcherItr.totalBlocks diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 455339943f42d..760458cb02a9b 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -103,13 +103,6 @@ class ShuffleReadMetrics extends Serializable { */ var fetchWaitTime: Long = _ - /** - * Total time spent fetching remote shuffle blocks. This aggregates the time spent fetching all - * input blocks. Since block fetches are both pipelined and parallelized, this can - * exceed fetchWaitTime and executorRunTime. - */ - var remoteFetchTime: Long = _ - /** * Total number of remote bytes read from the shuffle by this task */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 006e2a3335428..80f9ec7d03007 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -275,7 +275,6 @@ class JobLogger(val user: String, val logDirName: String) " BLOCK_FETCHED_LOCAL=" + metrics.localBlocksFetched + " BLOCK_FETCHED_REMOTE=" + metrics.remoteBlocksFetched + " REMOTE_FETCH_WAIT_TIME=" + metrics.fetchWaitTime + - " REMOTE_FETCH_TIME=" + metrics.remoteFetchTime + " REMOTE_BYTES_READ=" + metrics.remoteBytesRead case None => "" } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index fb50b45bd4197..bcfc39146a61e 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -49,7 +49,6 @@ trait BlockFetcherIterator extends Iterator[(BlockId, Option[Iterator[Any]])] wi def totalBlocks: Int def numLocalBlocks: Int def numRemoteBlocks: Int - def remoteFetchTime: Long def fetchWaitTime: Long def remoteBytesRead: Long } @@ -79,7 +78,6 @@ object BlockFetcherIterator { import blockManager._ private var _remoteBytesRead = 0L - private var _remoteFetchTime = 0L private var _fetchWaitTime = 0L if (blocksByAddress == null) { @@ -125,7 +123,6 @@ object BlockFetcherIterator { future.onSuccess { case Some(message) => { val fetchDone = System.currentTimeMillis() - _remoteFetchTime += fetchDone - fetchStart val bufferMessage = message.asInstanceOf[BufferMessage] val blockMessageArray = BlockMessageArray.fromBufferMessage(bufferMessage) for (blockMessage <- blockMessageArray) { @@ -241,7 +238,6 @@ object BlockFetcherIterator { override def totalBlocks: Int = numLocal + numRemote override def numLocalBlocks: Int = numLocal override def numRemoteBlocks: Int = numRemote - override def remoteFetchTime: Long = _remoteFetchTime override def fetchWaitTime: Long = _fetchWaitTime override def remoteBytesRead: Long = _remoteBytesRead diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 368c5154ea3b9..7c4f2b4361892 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -129,7 +129,6 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sm.localBlocksFetched should be > (0) sm.remoteBlocksFetched should be (0) sm.remoteBytesRead should be (0l) - sm.remoteFetchTime should be (0l) } } } From 923dba5096d4f7a96d67e4ee243b3b1085984bb9 Mon Sep 17 00:00:00 2001 From: Bryn Keller Date: Mon, 3 Mar 2014 16:38:57 -0800 Subject: [PATCH 042/397] Added a unit test for PairRDDFunctions.lookup Lookup didn't have a unit test. Added two tests, one for with a partitioner, and one for without. Author: Bryn Keller Closes #36 from xoltar/lookup and squashes the following commits: 3bc0d44 [Bryn Keller] Added a unit test for PairRDDFunctions.lookup --- .../spark/rdd/PairRDDFunctionsSuite.scala | 26 +++++++++++++++++++ 1 file changed, 26 insertions(+) diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index e3e23775f011d..85e8eb5dc3a1e 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -347,6 +347,32 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { */ pairs.saveAsNewAPIHadoopFile[ConfigTestFormat]("ignored") } + + test("lookup") { + val pairs = sc.parallelize(Array((1,2), (3,4), (5,6), (5,7))) + + assert(pairs.partitioner === None) + assert(pairs.lookup(1) === Seq(2)) + assert(pairs.lookup(5) === Seq(6,7)) + assert(pairs.lookup(-1) === Seq()) + + } + + test("lookup with partitioner") { + val pairs = sc.parallelize(Array((1,2), (3,4), (5,6), (5,7))) + + val p = new Partitioner { + def numPartitions: Int = 2 + + def getPartition(key: Any): Int = Math.abs(key.hashCode() % 2) + } + val shuffled = pairs.partitionBy(p) + + assert(shuffled.partitioner === Some(p)) + assert(shuffled.lookup(1) === Seq(2)) + assert(shuffled.lookup(5) === Seq(6,7)) + assert(shuffled.lookup(-1) === Seq()) + } } /* From f5ae38af87d4e6f266cb9af6265743c593f77d41 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 3 Mar 2014 21:24:19 -0800 Subject: [PATCH 043/397] SPARK-1158: Fix flaky RateLimitedOutputStreamSuite. There was actually a problem with the RateLimitedOutputStream implementation where the first second doesn't write anything because of integer rounding. So RateLimitedOutputStream was overly aggressive in throttling. Author: Reynold Xin Closes #55 from rxin/ratelimitest and squashes the following commits: 52ce1b7 [Reynold Xin] SPARK-1158: Fix flaky RateLimitedOutputStreamSuite. --- .../util/RateLimitedOutputStream.scala | 39 ++++++++++++------- .../util/RateLimitedOutputStreamSuite.scala | 13 ++++--- 2 files changed, 32 insertions(+), 20 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala index 2b8cdb72b8d0e..a96e2924a0b44 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RateLimitedOutputStream.scala @@ -22,12 +22,20 @@ import scala.annotation.tailrec import java.io.OutputStream import java.util.concurrent.TimeUnit._ +import org.apache.spark.Logging + + private[streaming] -class RateLimitedOutputStream(out: OutputStream, bytesPerSec: Int) extends OutputStream { - val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS) - val CHUNK_SIZE = 8192 - var lastSyncTime = System.nanoTime - var bytesWrittenSinceSync: Long = 0 +class RateLimitedOutputStream(out: OutputStream, desiredBytesPerSec: Int) + extends OutputStream + with Logging { + + require(desiredBytesPerSec > 0) + + private val SYNC_INTERVAL = NANOSECONDS.convert(10, SECONDS) + private val CHUNK_SIZE = 8192 + private var lastSyncTime = System.nanoTime + private var bytesWrittenSinceSync = 0L override def write(b: Int) { waitToWrite(1) @@ -59,9 +67,9 @@ class RateLimitedOutputStream(out: OutputStream, bytesPerSec: Int) extends Outpu @tailrec private def waitToWrite(numBytes: Int) { val now = System.nanoTime - val elapsedSecs = SECONDS.convert(math.max(now - lastSyncTime, 1), NANOSECONDS) - val rate = bytesWrittenSinceSync.toDouble / elapsedSecs - if (rate < bytesPerSec) { + val elapsedNanosecs = math.max(now - lastSyncTime, 1) + val rate = bytesWrittenSinceSync.toDouble * 1000000000 / elapsedNanosecs + if (rate < desiredBytesPerSec) { // It's okay to write; just update some variables and return bytesWrittenSinceSync += numBytes if (now > lastSyncTime + SYNC_INTERVAL) { @@ -71,13 +79,14 @@ class RateLimitedOutputStream(out: OutputStream, bytesPerSec: Int) extends Outpu } } else { // Calculate how much time we should sleep to bring ourselves to the desired rate. - // Based on throttler in Kafka - // scalastyle:off - // (https://github.com/kafka-dev/kafka/blob/master/core/src/main/scala/kafka/utils/Throttler.scala) - // scalastyle:on - val sleepTime = MILLISECONDS.convert((bytesWrittenSinceSync / bytesPerSec - elapsedSecs), - SECONDS) - if (sleepTime > 0) Thread.sleep(sleepTime) + val targetTimeInMillis = bytesWrittenSinceSync * 1000 / desiredBytesPerSec + val elapsedTimeInMillis = elapsedNanosecs / 1000000 + val sleepTimeInMillis = targetTimeInMillis - elapsedTimeInMillis + if (sleepTimeInMillis > 0) { + logTrace("Natural rate is " + rate + " per second but desired rate is " + + desiredBytesPerSec + ", sleeping for " + sleepTimeInMillis + " ms to compensate.") + Thread.sleep(sleepTimeInMillis) + } waitToWrite(numBytes) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala index e5bf6d70db5f9..7d18a0fcf7ba8 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala @@ -17,10 +17,11 @@ package org.apache.spark.streaming.util -import org.scalatest.FunSuite import java.io.ByteArrayOutputStream import java.util.concurrent.TimeUnit._ +import org.scalatest.FunSuite + class RateLimitedOutputStreamSuite extends FunSuite { private def benchmark[U](f: => U): Long = { @@ -29,12 +30,14 @@ class RateLimitedOutputStreamSuite extends FunSuite { System.nanoTime - start } - ignore("write") { + test("write") { val underlying = new ByteArrayOutputStream val data = "X" * 41000 - val stream = new RateLimitedOutputStream(underlying, 10000) + val stream = new RateLimitedOutputStream(underlying, desiredBytesPerSec = 10000) val elapsedNs = benchmark { stream.write(data.getBytes("UTF-8")) } - assert(SECONDS.convert(elapsedNs, NANOSECONDS) == 4) - assert(underlying.toString("UTF-8") == data) + + // We accept anywhere from 4.0 to 4.99999 seconds since the value is rounded down. + assert(SECONDS.convert(elapsedNs, NANOSECONDS) === 4) + assert(underlying.toString("UTF-8") === data) } } From b14ede789abfabe25144385e8dc2fb96691aba81 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Mon, 3 Mar 2014 21:27:18 -0800 Subject: [PATCH 044/397] Remove broken/unused Connection.getChunkFIFO method. This method appears to be broken -- since it never removes anything from messages, and it adds new messages to it, the while loop is an infinite loop. The method also does not appear to have ever been used since the code was added in 2012, so this commit removes it. cc @mateiz who originally added this method in case there's a reason it should be here! (https://github.com/apache/spark/commit/63051dd2bcc4bf09d413ff7cf89a37967edc33ba) Author: Kay Ousterhout Closes #69 from kayousterhout/remove_get_fifo and squashes the following commits: 053bc59 [Kay Ousterhout] Remove broken/unused Connection.getChunkFIFO method. --- .../org/apache/spark/network/Connection.scala | 36 ++----------------- 1 file changed, 2 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index f2e3c1a14ecc6..8219a185ea983 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -171,7 +171,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, remoteId_ : ConnectionManagerId) extends Connection(SocketChannel.open, selector_, remoteId_) { - private class Outbox(fair: Int = 0) { + private class Outbox { val messages = new Queue[Message]() val defaultChunkSize = 65536 //32768 //16384 var nextMessageToBeUsed = 0 @@ -186,38 +186,6 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } def getChunk(): Option[MessageChunk] = { - fair match { - case 0 => getChunkFIFO() - case 1 => getChunkRR() - case _ => throw new Exception("Unexpected fairness policy in outbox") - } - } - - private def getChunkFIFO(): Option[MessageChunk] = { - /*logInfo("Using FIFO")*/ - messages.synchronized { - while (!messages.isEmpty) { - val message = messages(0) - val chunk = message.getChunkForSending(defaultChunkSize) - if (chunk.isDefined) { - messages += message // this is probably incorrect, it wont work as fifo - if (!message.started) { - logDebug("Starting to send [" + message + "]") - message.started = true - message.startTime = System.currentTimeMillis - } - return chunk - } else { - message.finishTime = System.currentTimeMillis - logDebug("Finished sending [" + message + "] to [" + getRemoteConnectionManagerId() + - "] in " + message.timeTaken ) - } - } - } - None - } - - private def getChunkRR(): Option[MessageChunk] = { messages.synchronized { while (!messages.isEmpty) { /*nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ @@ -249,7 +217,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, // outbox is used as a lock - ensure that it is always used as a leaf (since methods which // lock it are invoked in context of other locks) - private val outbox = new Outbox(1) + private val outbox = new Outbox() /* This is orthogonal to whether we have pending bytes to write or not - and satisfies a slightly different purpose. This flag is to see if we need to force reregister for write even when we From 181ec5030792a10f3ce77e997d0e2eda9bcd6139 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 3 Mar 2014 22:31:30 -0800 Subject: [PATCH 045/397] [java8API] SPARK-964 Investigate the potential for using JDK 8 lambda expressions for the Java/Scala APIs Author: Prashant Sharma Author: Patrick Wendell Closes #17 from ScrapCodes/java8-lambdas and squashes the following commits: 95850e6 [Patrick Wendell] Some doc improvements and build changes to the Java 8 patch. 85a954e [Prashant Sharma] Nit. import orderings. 673f7ac [Prashant Sharma] Added support for -java-home as well 80a13e8 [Prashant Sharma] Used fake class tag syntax 26eb3f6 [Prashant Sharma] Patrick's comments on PR. 35d8d79 [Prashant Sharma] Specified java 8 building in the docs 31d4cd6 [Prashant Sharma] Maven build to support -Pjava8-tests flag. 4ab87d3 [Prashant Sharma] Review feedback on the pr c33dc2c [Prashant Sharma] SPARK-964, Java 8 API Support. --- .../java/function/DoubleFlatMapFunction.java} | 11 +- .../api/java/function/DoubleFunction.java | 27 + .../api/java/function/FlatMapFunction.java} | 10 +- .../api/java/function/FlatMapFunction2.java} | 10 +- .../spark/api/java/function/Function.java} | 12 +- .../spark/api/java/function/Function2.java} | 10 +- .../spark/api/java/function/Function3.java} | 9 +- .../java/function/PairFlatMapFunction.java | 30 + .../spark/api/java/function/PairFunction.java | 29 + .../spark/api/java/function/VoidFunction.java | 27 + .../apache/spark/api/java/JavaDoubleRDD.scala | 2 +- .../apache/spark/api/java/JavaPairRDD.scala | 67 +- .../org/apache/spark/api/java/JavaRDD.scala | 2 +- .../apache/spark/api/java/JavaRDDLike.scala | 82 +- .../api/java/function/DoubleFunction.scala | 29 - .../java/function/PairFlatMapFunction.scala | 36 - .../api/java/function/PairFunction.scala | 33 - .../api/java/function/VoidFunction.scala | 33 - .../api/java/function/WrappedFunction1.scala | 32 - .../api/java/function/WrappedFunction2.scala | 32 - .../api/java/function/WrappedFunction3.scala | 34 - .../java/org/apache/spark/JavaAPISuite.java | 38 +- dev/run-tests | 10 + docs/building-with-maven.md | 12 + docs/java-programming-guide.md | 56 +- .../org/apache/spark/examples/JavaHdfsLR.java | 6 +- .../org/apache/spark/examples/JavaKMeans.java | 2 +- .../apache/spark/examples/JavaLogQuery.java | 2 +- .../apache/spark/examples/JavaPageRank.java | 6 +- .../org/apache/spark/examples/JavaTC.java | 6 +- .../apache/spark/examples/JavaWordCount.java | 2 +- .../apache/spark/mllib/examples/JavaALS.java | 4 +- .../spark/mllib/examples/JavaKMeans.java | 2 +- .../apache/spark/mllib/examples/JavaLR.java | 2 +- .../examples/JavaKafkaWordCount.java | 2 +- .../examples/JavaNetworkWordCount.java | 2 +- .../streaming/examples/JavaQueueStream.java | 2 +- .../spark/streaming/zeromq/ZeroMQUtils.scala | 6 +- extras/README.md | 1 + extras/java8-tests/README.md | 24 + extras/java8-tests/pom.xml | 151 ++++ .../java/org/apache/spark/Java8APISuite.java | 391 ++++++++ .../apache/spark/streaming/Java8APISuite.java | 841 ++++++++++++++++++ .../src/test/resources/log4j.properties | 28 + pom.xml | 25 + project/SparkBuild.scala | 20 +- sbt/sbt-launch-lib.bash | 11 +- .../streaming/api/java/JavaDStream.scala | 2 +- .../streaming/api/java/JavaDStreamLike.scala | 119 ++- .../streaming/api/java/JavaPairDStream.scala | 101 +-- .../api/java/JavaStreamingContext.scala | 4 +- .../apache/spark/streaming/JavaAPISuite.java | 62 +- 52 files changed, 1946 insertions(+), 551 deletions(-) rename core/src/main/{scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.scala => java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java} (69%) create mode 100644 core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java rename core/src/main/{scala/org/apache/spark/api/java/function/FlatMapFunction.scala => java/org/apache/spark/api/java/function/FlatMapFunction.java} (79%) rename core/src/main/{scala/org/apache/spark/api/java/function/FlatMapFunction2.scala => java/org/apache/spark/api/java/function/FlatMapFunction2.java} (78%) rename core/src/main/{scala/org/apache/spark/api/java/function/Function.scala => java/org/apache/spark/api/java/function/Function.java} (72%) rename core/src/main/{scala/org/apache/spark/api/java/function/Function2.scala => java/org/apache/spark/api/java/function/Function2.java} (76%) rename core/src/main/{scala/org/apache/spark/api/java/function/Function3.scala => java/org/apache/spark/api/java/function/Function3.java} (75%) create mode 100644 core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java create mode 100644 core/src/main/java/org/apache/spark/api/java/function/PairFunction.java create mode 100644 core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java delete mode 100644 core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.scala delete mode 100644 core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.scala delete mode 100644 core/src/main/scala/org/apache/spark/api/java/function/PairFunction.scala delete mode 100644 core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala delete mode 100644 core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala delete mode 100644 core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction2.scala delete mode 100644 core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction3.scala create mode 100644 extras/README.md create mode 100644 extras/java8-tests/README.md create mode 100644 extras/java8-tests/pom.xml create mode 100644 extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java create mode 100644 extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java create mode 100644 extras/java8-tests/src/test/resources/log4j.properties diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.scala b/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java similarity index 69% rename from core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.scala rename to core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java index 7500a8943634b..57fd0a7a80494 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFlatMapFunction.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/DoubleFlatMapFunction.java @@ -15,16 +15,13 @@ * limitations under the License. */ -package org.apache.spark.api.java.function +package org.apache.spark.api.java.function; -import java.lang.{Double => JDouble, Iterable => JIterable} +import java.io.Serializable; /** * A function that returns zero or more records of type Double from each input record. */ -// DoubleFlatMapFunction does not extend FlatMapFunction because flatMap is -// overloaded for both FlatMapFunction and DoubleFlatMapFunction. -abstract class DoubleFlatMapFunction[T] extends WrappedFunction1[T, JIterable[JDouble]] - with Serializable { - // Intentionally left blank +public interface DoubleFlatMapFunction extends Serializable { + public Iterable call(T t) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java b/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java new file mode 100644 index 0000000000000..150144e0e418c --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/DoubleFunction.java @@ -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.api.java.function; + +import java.io.Serializable; + +/** + * A function that returns Doubles, and can be used to construct DoubleRDDs. + */ +public interface DoubleFunction extends Serializable { + public double call(T t) throws Exception; +} diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java similarity index 79% rename from core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala rename to core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java index bdb01f7670356..fa75842047c6a 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.api.java.function +package org.apache.spark.api.java.function; -import scala.reflect.ClassTag +import java.io.Serializable; /** * A function that returns zero or more output records from each input record. */ -abstract class FlatMapFunction[T, R] extends Function[T, java.lang.Iterable[R]] { - def elementType(): ClassTag[R] = ClassTag.Any.asInstanceOf[ClassTag[R]] -} +public interface FlatMapFunction extends Serializable { + public Iterable call(T t) throws Exception; +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java similarity index 78% rename from core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala rename to core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java index aae1349c5e17c..d1fdec072443d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/FlatMapFunction2.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.api.java.function +package org.apache.spark.api.java.function; -import scala.reflect.ClassTag +import java.io.Serializable; /** * A function that takes two inputs and returns zero or more output records. */ -abstract class FlatMapFunction2[A, B, C] extends Function2[A, B, java.lang.Iterable[C]] { - def elementType() : ClassTag[C] = ClassTag.Any.asInstanceOf[ClassTag[C]] -} +public interface FlatMapFunction2 extends Serializable { + public Iterable call(T1 t1, T2 t2) throws Exception; +} \ No newline at end of file diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function.scala b/core/src/main/java/org/apache/spark/api/java/function/Function.java similarity index 72% rename from core/src/main/scala/org/apache/spark/api/java/function/Function.scala rename to core/src/main/java/org/apache/spark/api/java/function/Function.java index a5e1701f7718f..d00551bb0add6 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/Function.java @@ -15,17 +15,15 @@ * limitations under the License. */ -package org.apache.spark.api.java.function +package org.apache.spark.api.java.function; -import scala.reflect.ClassTag -import org.apache.spark.api.java.JavaSparkContext +import java.io.Serializable; /** - * Base class for functions whose return types do not create special RDDs. PairFunction and + * Base interface for functions whose return types do not create special RDDs. PairFunction and * DoubleFunction are handled separately, to allow PairRDDs and DoubleRDDs to be constructed * when mapping RDDs of other types. */ -abstract class Function[T, R] extends WrappedFunction1[T, R] with Serializable { - def returnType(): ClassTag[R] = JavaSparkContext.fakeClassTag +public interface Function extends Serializable { + public R call(T1 v1) throws Exception; } - diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function2.scala b/core/src/main/java/org/apache/spark/api/java/function/Function2.java similarity index 76% rename from core/src/main/scala/org/apache/spark/api/java/function/Function2.scala rename to core/src/main/java/org/apache/spark/api/java/function/Function2.java index fa3616cbcb4d2..793caaa61ac5a 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function2.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/Function2.java @@ -15,15 +15,13 @@ * limitations under the License. */ -package org.apache.spark.api.java.function +package org.apache.spark.api.java.function; -import scala.reflect.ClassTag -import org.apache.spark.api.java.JavaSparkContext +import java.io.Serializable; /** * A two-argument function that takes arguments of type T1 and T2 and returns an R. */ -abstract class Function2[T1, T2, R] extends WrappedFunction2[T1, T2, R] with Serializable { - def returnType(): ClassTag[R] = JavaSparkContext.fakeClassTag +public interface Function2 extends Serializable { + public R call(T1 v1, T2 v2) throws Exception; } - diff --git a/core/src/main/scala/org/apache/spark/api/java/function/Function3.scala b/core/src/main/java/org/apache/spark/api/java/function/Function3.java similarity index 75% rename from core/src/main/scala/org/apache/spark/api/java/function/Function3.scala rename to core/src/main/java/org/apache/spark/api/java/function/Function3.java index 45152891e9272..b4151c3417df4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/function/Function3.scala +++ b/core/src/main/java/org/apache/spark/api/java/function/Function3.java @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.spark.api.java.function +package org.apache.spark.api.java.function; -import org.apache.spark.api.java.JavaSparkContext -import scala.reflect.ClassTag +import java.io.Serializable; /** * A three-argument function that takes arguments of type T1, T2 and T3 and returns an R. */ -abstract class Function3[T1, T2, T3, R] extends WrappedFunction3[T1, T2, T3, R] with Serializable { - def returnType(): ClassTag[R] = JavaSparkContext.fakeClassTag +public interface Function3 extends Serializable { + public R call(T1 v1, T2 v2, T3 v3) throws Exception; } diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java new file mode 100644 index 0000000000000..691ef2eceb1f6 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/PairFlatMapFunction.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.api.java.function; + +import java.io.Serializable; + +import scala.Tuple2; + +/** + * A function that returns zero or more key-value pair records from each input record. The + * key-value pairs are represented as scala.Tuple2 objects. + */ +public interface PairFlatMapFunction extends Serializable { + public Iterable> call(T t) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java new file mode 100644 index 0000000000000..abd9bcc07ac61 --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/PairFunction.java @@ -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.api.java.function; + +import java.io.Serializable; + +import scala.Tuple2; + +/** + * A function that returns key-value pairs (Tuple2), and can be used to construct PairRDDs. + */ +public interface PairFunction extends Serializable { + public Tuple2 call(T t) throws Exception; +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java new file mode 100644 index 0000000000000..2a10435b7523a --- /dev/null +++ b/core/src/main/java/org/apache/spark/api/java/function/VoidFunction.java @@ -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.api.java.function; + +import java.io.Serializable; + +/** + * A function with no return value. + */ +public interface VoidFunction extends Serializable { + public void call(T t) throws Exception; +} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 071044463d980..d1787061bc642 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -83,7 +83,7 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja * Return a new RDD containing only the elements that satisfy a predicate. */ def filter(f: JFunction[JDouble, java.lang.Boolean]): JavaDoubleRDD = - fromRDD(srdd.filter(x => f(x).booleanValue())) + fromRDD(srdd.filter(x => f.call(x).booleanValue())) /** * Return a new RDD that is reduced into `numPartitions` partitions. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 3f672900cb90f..857626fe84af9 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -32,7 +32,7 @@ import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions import org.apache.spark.api.java.JavaSparkContext.fakeClassTag -import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.storage.StorageLevel @@ -89,7 +89,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Return a new RDD containing only the elements that satisfy a predicate. */ def filter(f: JFunction[(K, V), java.lang.Boolean]): JavaPairRDD[K, V] = - new JavaPairRDD[K, V](rdd.filter(x => f(x).booleanValue())) + new JavaPairRDD[K, V](rdd.filter(x => f.call(x).booleanValue())) /** * Return a new RDD that is reduced into `numPartitions` partitions. @@ -165,9 +165,9 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Simplified version of combineByKey that hash-partitions the output RDD. */ def combineByKey[C](createCombiner: JFunction[V, C], - mergeValue: JFunction2[C, V, C], - mergeCombiners: JFunction2[C, C, C], - numPartitions: Int): JavaPairRDD[K, C] = + mergeValue: JFunction2[C, V, C], + mergeCombiners: JFunction2[C, C, C], + numPartitions: Int): JavaPairRDD[K, C] = combineByKey(createCombiner, mergeValue, mergeCombiners, new HashPartitioner(numPartitions)) /** @@ -442,7 +442,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) */ def flatMapValues[U](f: JFunction[V, java.lang.Iterable[U]]): JavaPairRDD[K, U] = { import scala.collection.JavaConverters._ - def fn = (x: V) => f.apply(x).asScala + def fn = (x: V) => f.call(x).asScala implicit val ctag: ClassTag[U] = fakeClassTag fromRDD(rdd.flatMapValues(fn)) } @@ -511,49 +511,49 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** Output the RDD to any Hadoop-supported file system. */ def saveAsHadoopFile[F <: OutputFormat[_, _]]( - path: String, - keyClass: Class[_], - valueClass: Class[_], - outputFormatClass: Class[F], - conf: JobConf) { + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[F], + conf: JobConf) { rdd.saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, conf) } /** Output the RDD to any Hadoop-supported file system. */ def saveAsHadoopFile[F <: OutputFormat[_, _]]( - path: String, - keyClass: Class[_], - valueClass: Class[_], - outputFormatClass: Class[F]) { + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[F]) { rdd.saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass) } /** Output the RDD to any Hadoop-supported file system, compressing with the supplied codec. */ def saveAsHadoopFile[F <: OutputFormat[_, _]]( - path: String, - keyClass: Class[_], - valueClass: Class[_], - outputFormatClass: Class[F], - codec: Class[_ <: CompressionCodec]) { + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[F], + codec: Class[_ <: CompressionCodec]) { rdd.saveAsHadoopFile(path, keyClass, valueClass, outputFormatClass, codec) } /** Output the RDD to any Hadoop-supported file system. */ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[_, _]]( - path: String, - keyClass: Class[_], - valueClass: Class[_], - outputFormatClass: Class[F], - conf: Configuration) { + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[F], + conf: Configuration) { rdd.saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass, conf) } /** Output the RDD to any Hadoop-supported file system. */ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[_, _]]( - path: String, - keyClass: Class[_], - valueClass: Class[_], - outputFormatClass: Class[F]) { + path: String, + keyClass: Class[_], + valueClass: Class[_], + outputFormatClass: Class[F]) { rdd.saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass) } @@ -700,6 +700,15 @@ object JavaPairRDD { implicit def toRDD[K, V](rdd: JavaPairRDD[K, V]): RDD[(K, V)] = rdd.rdd + private[spark] + implicit def toScalaFunction2[T1, T2, R](fun: JFunction2[T1, T2, R]): Function2[T1, T2, R] = { + (x: T1, x1: T2) => fun.call(x, x1) + } + + private[spark] implicit def toScalaFunction[T, R](fun: JFunction[T, R]): T => R = x => fun.call(x) + + private[spark] + implicit def pairFunToScalaFun[A, B, C](x: PairFunction[A, B, C]): A => (B, C) = y => x.call(y) /** Convert a JavaRDD of key-value pairs to JavaPairRDD. */ def fromJavaRDD[K, V](rdd: JavaRDD[(K, V)]): JavaPairRDD[K, V] = { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index d7ce8fdfc23f4..e973c46edd1ce 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -70,7 +70,7 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) * Return a new RDD containing only the elements that satisfy a predicate. */ def filter(f: JFunction[T, java.lang.Boolean]): JavaRDD[T] = - wrapRDD(rdd.filter((x => f(x).booleanValue()))) + wrapRDD(rdd.filter((x => f.call(x).booleanValue()))) /** * Return a new RDD that is reduced into `numPartitions` partitions. diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 729668fb679b4..af0114bee3f49 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -67,7 +67,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by applying a function to all elements of this RDD. */ def map[R](f: JFunction[T, R]): JavaRDD[R] = - new JavaRDD(rdd.map(f)(f.returnType()))(f.returnType()) + new JavaRDD(rdd.map(f)(fakeClassTag))(fakeClassTag) /** * Return a new RDD by applying a function to each partition of this RDD, while tracking the index @@ -82,15 +82,16 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return a new RDD by applying a function to all elements of this RDD. */ - def map[R](f: DoubleFunction[T]): JavaDoubleRDD = - new JavaDoubleRDD(rdd.map(x => f(x).doubleValue())) + def mapToDouble[R](f: DoubleFunction[T]): JavaDoubleRDD = { + new JavaDoubleRDD(rdd.map(x => f.call(x).doubleValue())) + } /** * Return a new RDD by applying a function to all elements of this RDD. */ - def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { - val ctag = implicitly[ClassTag[Tuple2[K2, V2]]] - new JavaPairRDD(rdd.map(f)(ctag))(f.keyType(), f.valueType()) + def mapToPair[K2, V2](f: PairFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { + def cm = implicitly[ClassTag[(K2, V2)]] + new JavaPairRDD(rdd.map[(K2, V2)](f)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } /** @@ -99,17 +100,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def flatMap[U](f: FlatMapFunction[T, U]): JavaRDD[U] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.apply(x).asScala - JavaRDD.fromRDD(rdd.flatMap(fn)(f.elementType()))(f.elementType()) + def fn = (x: T) => f.call(x).asScala + JavaRDD.fromRDD(rdd.flatMap(fn)(fakeClassTag[U]))(fakeClassTag[U]) } /** * Return a new RDD by first applying a function to all elements of this * RDD, and then flattening the results. */ - def flatMap(f: DoubleFlatMapFunction[T]): JavaDoubleRDD = { + def flatMapToDouble(f: DoubleFlatMapFunction[T]): JavaDoubleRDD = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.apply(x).asScala + def fn = (x: T) => f.call(x).asScala new JavaDoubleRDD(rdd.flatMap(fn).map((x: java.lang.Double) => x.doubleValue())) } @@ -117,19 +118,19 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return a new RDD by first applying a function to all elements of this * RDD, and then flattening the results. */ - def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { + def flatMapToPair[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairRDD[K2, V2] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.apply(x).asScala - val ctag = implicitly[ClassTag[Tuple2[K2, V2]]] - JavaPairRDD.fromRDD(rdd.flatMap(fn)(ctag))(f.keyType(), f.valueType()) + def fn = (x: T) => f.call(x).asScala + def cm = implicitly[ClassTag[(K2, V2)]] + JavaPairRDD.fromRDD(rdd.flatMap(fn)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } /** * Return a new RDD by applying a function to each partition of this RDD. */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaRDD[U] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) - JavaRDD.fromRDD(rdd.mapPartitions(fn)(f.elementType()))(f.elementType()) + def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + JavaRDD.fromRDD(rdd.mapPartitions(fn)(fakeClassTag[U]))(fakeClassTag[U]) } /** @@ -137,52 +138,53 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U], preservesPartitioning: Boolean): JavaRDD[U] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) - JavaRDD.fromRDD(rdd.mapPartitions(fn, preservesPartitioning)(f.elementType()))(f.elementType()) + def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + JavaRDD.fromRDD( + rdd.mapPartitions(fn, preservesPartitioning)(fakeClassTag[U]))(fakeClassTag[U]) } /** - * Return a new RDD by applying a function to each partition of this RDD. + * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions(f: DoubleFlatMapFunction[java.util.Iterator[T]]): JavaDoubleRDD = { - def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) + def mapPartitionsToDouble(f: DoubleFlatMapFunction[java.util.Iterator[T]]): JavaDoubleRDD = { + def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) new JavaDoubleRDD(rdd.mapPartitions(fn).map((x: java.lang.Double) => x.doubleValue())) } /** - * Return a new RDD by applying a function to each partition of this RDD. + * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]): + def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]): JavaPairRDD[K2, V2] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) - JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(f.keyType(), f.valueType()) + def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + JavaPairRDD.fromRDD(rdd.mapPartitions(fn))(fakeClassTag[K2], fakeClassTag[V2]) } - /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions(f: DoubleFlatMapFunction[java.util.Iterator[T]], - preservesPartitioning: Boolean): JavaDoubleRDD = { - def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) + def mapPartitionsToDouble(f: DoubleFlatMapFunction[java.util.Iterator[T]], + preservesPartitioning: Boolean): JavaDoubleRDD = { + def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) new JavaDoubleRDD(rdd.mapPartitions(fn, preservesPartitioning) - .map((x: java.lang.Double) => x.doubleValue())) + .map(x => x.doubleValue())) } /** * Return a new RDD by applying a function to each partition of this RDD. */ - def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], + def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2], preservesPartitioning: Boolean): JavaPairRDD[K2, V2] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) - JavaPairRDD.fromRDD(rdd.mapPartitions(fn, preservesPartitioning))(f.keyType(), f.valueType()) + def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + JavaPairRDD.fromRDD( + rdd.mapPartitions(fn, preservesPartitioning))(fakeClassTag[K2], fakeClassTag[V2]) } /** * Applies a function f to each partition of this RDD. */ def foreachPartition(f: VoidFunction[java.util.Iterator[T]]) { - rdd.foreachPartition((x => f(asJavaIterator(x)))) + rdd.foreachPartition((x => f.call(asJavaIterator(x)))) } /** @@ -205,7 +207,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag - JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(f.returnType))) + JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag))) } /** @@ -215,7 +217,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag - JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(f.returnType))) + JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[K]))) } /** @@ -255,9 +257,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { other: JavaRDDLike[U, _], f: FlatMapFunction2[java.util.Iterator[T], java.util.Iterator[U], V]): JavaRDD[V] = { def fn = (x: Iterator[T], y: Iterator[U]) => asScalaIterator( - f.apply(asJavaIterator(x), asJavaIterator(y)).iterator()) + f.call(asJavaIterator(x), asJavaIterator(y)).iterator()) JavaRDD.fromRDD( - rdd.zipPartitions(other.rdd)(fn)(other.classTag, f.elementType()))(f.elementType()) + rdd.zipPartitions(other.rdd)(fn)(other.classTag, fakeClassTag[V]))(fakeClassTag[V]) } // Actions (launch a job to return a value to the user program) @@ -266,7 +268,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Applies a function f to all elements of this RDD. */ def foreach(f: VoidFunction[T]) { - val cleanF = rdd.context.clean(f) + val cleanF = rdd.context.clean((x: T) => f.call(x)) rdd.foreach(cleanF) } @@ -320,7 +322,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { */ def aggregate[U](zeroValue: U)(seqOp: JFunction2[U, T, U], combOp: JFunction2[U, U, U]): U = - rdd.aggregate(zeroValue)(seqOp, combOp)(seqOp.returnType) + rdd.aggregate(zeroValue)(seqOp, combOp)(fakeClassTag[U]) /** * Return the number of elements in the RDD. diff --git a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.scala deleted file mode 100644 index 2cdf2e92c3daa..0000000000000 --- a/core/src/main/scala/org/apache/spark/api/java/function/DoubleFunction.scala +++ /dev/null @@ -1,29 +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.api.java.function - -import java.lang.{Double => JDouble} - -/** - * A function that returns Doubles, and can be used to construct DoubleRDDs. - */ -// DoubleFunction does not extend Function because some UDF functions, like map, -// are overloaded for both Function and DoubleFunction. -abstract class DoubleFunction[T] extends WrappedFunction1[T, JDouble] with Serializable { - // Intentionally left blank -} diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.scala deleted file mode 100644 index 8467bbb892ab0..0000000000000 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFlatMapFunction.scala +++ /dev/null @@ -1,36 +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.api.java.function - -import java.lang.{Iterable => JIterable} -import org.apache.spark.api.java.JavaSparkContext -import scala.reflect.ClassTag - -/** - * A function that returns zero or more key-value pair records from each input record. The - * key-value pairs are represented as scala.Tuple2 objects. - */ -// PairFlatMapFunction does not extend FlatMapFunction because flatMap is -// overloaded for both FlatMapFunction and PairFlatMapFunction. -abstract class PairFlatMapFunction[T, K, V] extends WrappedFunction1[T, JIterable[(K, V)]] - with Serializable { - - def keyType(): ClassTag[K] = JavaSparkContext.fakeClassTag - - def valueType(): ClassTag[V] = JavaSparkContext.fakeClassTag -} diff --git a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.scala deleted file mode 100644 index d0ba0b6307ee9..0000000000000 --- a/core/src/main/scala/org/apache/spark/api/java/function/PairFunction.scala +++ /dev/null @@ -1,33 +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.api.java.function - -import scala.reflect.ClassTag -import org.apache.spark.api.java.JavaSparkContext - -/** - * A function that returns key-value pairs (Tuple2), and can be used to construct PairRDDs. - */ -// PairFunction does not extend Function because some UDF functions, like map, -// are overloaded for both Function and PairFunction. -abstract class PairFunction[T, K, V] extends WrappedFunction1[T, (K, V)] with Serializable { - - def keyType(): ClassTag[K] = JavaSparkContext.fakeClassTag - - def valueType(): ClassTag[V] = JavaSparkContext.fakeClassTag -} diff --git a/core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala b/core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala deleted file mode 100644 index ea94313a4ab59..0000000000000 --- a/core/src/main/scala/org/apache/spark/api/java/function/VoidFunction.scala +++ /dev/null @@ -1,33 +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.api.java.function - -/** - * A function with no return value. - */ -// This allows Java users to write void methods without having to return Unit. -abstract class VoidFunction[T] extends Serializable { - @throws(classOf[Exception]) - def call(t: T) : Unit -} - -// VoidFunction cannot extend AbstractFunction1 (because that would force users to explicitly -// return Unit), so it is implicitly converted to a Function1[T, Unit]: -object VoidFunction { - implicit def toFunction[T](f: VoidFunction[T]) : Function1[T, Unit] = ((x : T) => f.call(x)) -} diff --git a/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala deleted file mode 100644 index cfe694f65d558..0000000000000 --- a/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction1.scala +++ /dev/null @@ -1,32 +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.api.java.function - -import scala.runtime.AbstractFunction1 - -/** - * Subclass of Function1 for ease of calling from Java. The main thing it does is re-expose the - * apply() method as call() and declare that it can throw Exception (since AbstractFunction1.apply - * isn't marked to allow that). - */ -private[spark] abstract class WrappedFunction1[T, R] extends AbstractFunction1[T, R] { - @throws(classOf[Exception]) - def call(t: T): R - - final def apply(t: T): R = call(t) -} diff --git a/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction2.scala b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction2.scala deleted file mode 100644 index eb9277c6fb4cb..0000000000000 --- a/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction2.scala +++ /dev/null @@ -1,32 +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.api.java.function - -import scala.runtime.AbstractFunction2 - -/** - * Subclass of Function2 for ease of calling from Java. The main thing it does is re-expose the - * apply() method as call() and declare that it can throw Exception (since AbstractFunction2.apply - * isn't marked to allow that). - */ -private[spark] abstract class WrappedFunction2[T1, T2, R] extends AbstractFunction2[T1, T2, R] { - @throws(classOf[Exception]) - def call(t1: T1, t2: T2): R - - final def apply(t1: T1, t2: T2): R = call(t1, t2) -} diff --git a/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction3.scala b/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction3.scala deleted file mode 100644 index d314dbdf1d980..0000000000000 --- a/core/src/main/scala/org/apache/spark/api/java/function/WrappedFunction3.scala +++ /dev/null @@ -1,34 +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.api.java.function - -import scala.runtime.AbstractFunction3 - -/** - * Subclass of Function3 for ease of calling from Java. The main thing it does is re-expose the - * apply() method as call() and declare that it can throw Exception (since AbstractFunction3.apply - * isn't marked to allow that). - */ -private[spark] abstract class WrappedFunction3[T1, T2, T3, R] - extends AbstractFunction3[T1, T2, T3, R] { - @throws(classOf[Exception]) - def call(t1: T1, t2: T2, t3: T3): R - - final def apply(t1: T1, t2: T2, t3: T3): R = call(t1, t2, t3) -} - diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index aa5079c159830..c7d0e2d577726 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -386,14 +386,14 @@ public void javaDoubleRDDHistoGram() { @Test public void map() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - JavaDoubleRDD doubles = rdd.map(new DoubleFunction() { + JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction() { @Override - public Double call(Integer x) { + public double call(Integer x) { return 1.0 * x; } }).cache(); doubles.collect(); - JavaPairRDD pairs = rdd.map(new PairFunction() { + JavaPairRDD pairs = rdd.mapToPair(new PairFunction() { @Override public Tuple2 call(Integer x) { return new Tuple2(x, x); @@ -422,7 +422,7 @@ public Iterable call(String x) { Assert.assertEquals("Hello", words.first()); Assert.assertEquals(11, words.count()); - JavaPairRDD pairs = rdd.flatMap( + JavaPairRDD pairs = rdd.flatMapToPair( new PairFlatMapFunction() { @Override @@ -436,7 +436,7 @@ public Iterable> call(String s) { Assert.assertEquals(new Tuple2("Hello", "Hello"), pairs.first()); Assert.assertEquals(11, pairs.count()); - JavaDoubleRDD doubles = rdd.flatMap(new DoubleFlatMapFunction() { + JavaDoubleRDD doubles = rdd.flatMapToDouble(new DoubleFlatMapFunction() { @Override public Iterable call(String s) { List lengths = new LinkedList(); @@ -459,7 +459,7 @@ public void mapsFromPairsToPairs() { JavaPairRDD pairRDD = sc.parallelizePairs(pairs); // Regression test for SPARK-668: - JavaPairRDD swapped = pairRDD.flatMap( + JavaPairRDD swapped = pairRDD.flatMapToPair( new PairFlatMapFunction, String, Integer>() { @Override public Iterable> call(Tuple2 item) throws Exception { @@ -469,7 +469,7 @@ public Iterable> call(Tuple2 item) thro swapped.collect(); // There was never a bug here, but it's worth testing: - pairRDD.map(new PairFunction, String, Integer>() { + pairRDD.mapToPair(new PairFunction, String, Integer>() { @Override public Tuple2 call(Tuple2 item) throws Exception { return item.swap(); @@ -592,7 +592,7 @@ public void sequenceFile() { ); JavaPairRDD rdd = sc.parallelizePairs(pairs); - rdd.map(new PairFunction, IntWritable, Text>() { + rdd.mapToPair(new PairFunction, IntWritable, Text>() { @Override public Tuple2 call(Tuple2 pair) { return new Tuple2(new IntWritable(pair._1()), new Text(pair._2())); @@ -601,7 +601,7 @@ public Tuple2 call(Tuple2 pair) { // Try reading the output back as an object file JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, - Text.class).map(new PairFunction, Integer, String>() { + Text.class).mapToPair(new PairFunction, Integer, String>() { @Override public Tuple2 call(Tuple2 pair) { return new Tuple2(pair._1().get(), pair._2().toString()); @@ -622,7 +622,7 @@ public void writeWithNewAPIHadoopFile() { ); JavaPairRDD rdd = sc.parallelizePairs(pairs); - rdd.map(new PairFunction, IntWritable, Text>() { + rdd.mapToPair(new PairFunction, IntWritable, Text>() { @Override public Tuple2 call(Tuple2 pair) { return new Tuple2(new IntWritable(pair._1()), new Text(pair._2())); @@ -653,7 +653,7 @@ public void readWithNewAPIHadoopFile() throws IOException { ); JavaPairRDD rdd = sc.parallelizePairs(pairs); - rdd.map(new PairFunction, IntWritable, Text>() { + rdd.mapToPair(new PairFunction, IntWritable, Text>() { @Override public Tuple2 call(Tuple2 pair) { return new Tuple2(new IntWritable(pair._1()), new Text(pair._2())); @@ -713,7 +713,7 @@ public void hadoopFile() { ); JavaPairRDD rdd = sc.parallelizePairs(pairs); - rdd.map(new PairFunction, IntWritable, Text>() { + rdd.mapToPair(new PairFunction, IntWritable, Text>() { @Override public Tuple2 call(Tuple2 pair) { return new Tuple2(new IntWritable(pair._1()), new Text(pair._2())); @@ -743,7 +743,7 @@ public void hadoopFileCompressed() { ); JavaPairRDD rdd = sc.parallelizePairs(pairs); - rdd.map(new PairFunction, IntWritable, Text>() { + rdd.mapToPair(new PairFunction, IntWritable, Text>() { @Override public Tuple2 call(Tuple2 pair) { return new Tuple2(new IntWritable(pair._1()), new Text(pair._2())); @@ -766,9 +766,9 @@ public String call(Tuple2 x) { @Test public void zip() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); - JavaDoubleRDD doubles = rdd.map(new DoubleFunction() { + JavaDoubleRDD doubles = rdd.mapToDouble(new DoubleFunction() { @Override - public Double call(Integer x) { + public double call(Integer x) { return 1.0 * x; } }); @@ -893,13 +893,13 @@ public void checkpointAndRestore() { @Test public void mapOnPairRDD() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1,2,3,4)); - JavaPairRDD rdd2 = rdd1.map(new PairFunction() { + JavaPairRDD rdd2 = rdd1.mapToPair(new PairFunction() { @Override public Tuple2 call(Integer i) throws Exception { return new Tuple2(i, i % 2); } }); - JavaPairRDD rdd3 = rdd2.map( + JavaPairRDD rdd3 = rdd2.mapToPair( new PairFunction, Integer, Integer>() { @Override public Tuple2 call(Tuple2 in) throws Exception { @@ -919,7 +919,7 @@ public Tuple2 call(Tuple2 in) throws Excepti public void collectPartitions() { JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); - JavaPairRDD rdd2 = rdd1.map(new PairFunction() { + JavaPairRDD rdd2 = rdd1.mapToPair(new PairFunction() { @Override public Tuple2 call(Integer i) throws Exception { return new Tuple2(i, i % 2); @@ -984,7 +984,7 @@ public void countApproxDistinctByKey() { public void collectAsMapWithIntArrayValues() { // Regression test for SPARK-1040 JavaRDD rdd = sc.parallelize(Arrays.asList(1)); - JavaPairRDD pairRDD = rdd.map(new PairFunction() { + JavaPairRDD pairRDD = rdd.mapToPair(new PairFunction() { @Override public Tuple2 call(Integer x) throws Exception { return new Tuple2(x, new int[] { x }); diff --git a/dev/run-tests b/dev/run-tests index d65a397b4c8c7..cf0b940c09a81 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -27,6 +27,16 @@ rm -rf ./work # Fail fast set -e +if test -x "$JAVA_HOME/bin/java"; then + declare java_cmd="$JAVA_HOME/bin/java" +else + declare java_cmd=java +fi + +JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') +[ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run, because JDK version is < 1.8." + + echo "=========================================================================" echo "Running Scala style checks" echo "=========================================================================" diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index ded12926885b9..a982c4dbac7d4 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -25,6 +25,8 @@ If you don't run this, you may see errors like the following: You can fix this by setting the `MAVEN_OPTS` variable as discussed before. +*Note: For Java 1.8 and above this step is not required.* + ## Specifying the Hadoop version ## Because HDFS is not protocol-compatible across versions, if you want to read from HDFS, you'll need to build Spark against the specific HDFS version in your environment. You can do this through the "hadoop.version" property. If unset, Spark will build against Hadoop 1.0.4 by default. @@ -76,3 +78,13 @@ The maven build includes support for building a Debian package containing the as $ mvn -Pdeb -DskipTests clean package The debian package can then be found under assembly/target. We added the short commit hash to the file name so that we can distinguish individual packages built for SNAPSHOT versions. + +## Running java 8 test suites. + +Running only java 8 tests and nothing else. + + $ mvn install -DskipTests -Pjava8-tests + +Java 8 tests are run when -Pjava8-tests profile is enabled, they will run in spite of -DskipTests. +For these tests to run your system must have a JDK 8 installation. +If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. diff --git a/docs/java-programming-guide.md b/docs/java-programming-guide.md index 5c73dbb25ede8..6632360f6e3ca 100644 --- a/docs/java-programming-guide.md +++ b/docs/java-programming-guide.md @@ -21,15 +21,21 @@ operations (e.g. map) and handling RDDs of different types, as discussed next. There are a few key differences between the Java and Scala APIs: -* Java does not support anonymous or first-class functions, so functions must - be implemented by extending the +* Java does not support anonymous or first-class functions, so functions are passed + using anonymous classes that implement the [`org.apache.spark.api.java.function.Function`](api/core/index.html#org.apache.spark.api.java.function.Function), [`Function2`](api/core/index.html#org.apache.spark.api.java.function.Function2), etc. - classes. + interfaces. * To maintain type safety, the Java API defines specialized Function and RDD classes for key-value pairs and doubles. For example, [`JavaPairRDD`](api/core/index.html#org.apache.spark.api.java.JavaPairRDD) stores key-value pairs. +* Some methods are defined on the basis of the passed anonymous function's + (a.k.a lambda expression) return type, + for example mapToPair(...) or flatMapToPair returns + [`JavaPairRDD`](api/core/index.html#org.apache.spark.api.java.JavaPairRDD), + similarly mapToDouble and flatMapToDouble returns + [`JavaDoubleRDD`](api/core/index.html#org.apache.spark.api.java.JavaDoubleRDD). * RDD methods like `collect()` and `countByKey()` return Java collections types, such as `java.util.List` and `java.util.Map`. * Key-value pairs, which are simply written as `(key, value)` in Scala, are represented @@ -53,10 +59,10 @@ each specialized RDD class, so filtering a `PairRDD` returns a new `PairRDD`, etc (this acheives the "same-result-type" principle used by the [Scala collections framework](http://docs.scala-lang.org/overviews/core/architecture-of-scala-collections.html)). -## Function Classes +## Function Interfaces -The following table lists the function classes used by the Java API. Each -class has a single abstract method, `call()`, that must be implemented. +The following table lists the function interfaces used by the Java API. Each +interface has a single abstract method, `call()`, that must be implemented. @@ -78,7 +84,6 @@ RDD [storage level](scala-programming-guide.html#rdd-persistence) constants, suc declared in the [org.apache.spark.api.java.StorageLevels](api/core/index.html#org.apache.spark.api.java.StorageLevels) class. To define your own storage level, you can use StorageLevels.create(...). - # Other Features The Java API supports other Spark features, including @@ -86,6 +91,21 @@ The Java API supports other Spark features, including [broadcast variables](scala-programming-guide.html#broadcast-variables), and [caching](scala-programming-guide.html#rdd-persistence). +# Upgrading From Pre-1.0 Versions of Spark + +In version 1.0 of Spark the Java API was refactored to better support Java 8 +lambda expressions. Users upgrading from older versions of Spark should note +the following changes: + +* All `org.apache.spark.api.java.function.*` have been changed from abstract + classes to interfaces. This means that concrete implementations of these + `Function` classes will need to use `implements` rather than `extends`. +* Certain transformation functions now have multiple versions depending + on the return type. In Spark core, the map functions (map, flatMap, + mapPartitons) have type-specific versions, e.g. + [`mapToPair`](api/core/index.html#org.apache.spark.api.java.JavaRDD@mapToPair[K2,V2](f:org.apache.spark.api.java.function.PairFunction[T,K2,V2]):org.apache.spark.api.java.JavaPairRDD[K2,V2]) + and [`mapToDouble`](api/core/index.html#org.apache.spark.api.java.JavaRDD@mapToDouble[R](f:org.apache.spark.api.java.function.DoubleFunction[T]):org.apache.spark.api.java.JavaDoubleRDD). + Spark Streaming also uses the same approach, e.g. [`transformToPair`](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaDStream@transformToPair[K2,V2](transformFunc:org.apache.spark.api.java.function.Function[R,org.apache.spark.api.java.JavaPairRDD[K2,V2]]):org.apache.spark.streaming.api.java.JavaPairDStream[K2,V2]). # Example @@ -127,11 +147,20 @@ class Split extends FlatMapFunction { JavaRDD words = lines.flatMap(new Split()); {% endhighlight %} +Java 8+ users can also write the above `FlatMapFunction` in a more concise way using +a lambda expression: + +{% highlight java %} +JavaRDD words = lines.flatMap(s -> Arrays.asList(s.split(" "))); +{% endhighlight %} + +This lambda syntax can be applied to all anonymous classes in Java 8. + Continuing with the word count example, we map each word to a `(word, 1)` pair: {% highlight java %} import scala.Tuple2; -JavaPairRDD ones = words.map( +JavaPairRDD ones = words.mapToPair( new PairFunction() { public Tuple2 call(String s) { return new Tuple2(s, 1); @@ -140,7 +169,7 @@ JavaPairRDD ones = words.map( ); {% endhighlight %} -Note that `map` was passed a `PairFunction` and +Note that `mapToPair` was passed a `PairFunction` and returned a `JavaPairRDD`. To finish the word count program, we will use `reduceByKey` to count the @@ -164,7 +193,7 @@ possible to chain the RDD transformations, so the word count example could also be written as: {% highlight java %} -JavaPairRDD counts = lines.flatMap( +JavaPairRDD counts = lines.flatMapToPair( ... ).map( ... @@ -180,10 +209,11 @@ just a matter of style. We currently provide documentation for the Java API as Scaladoc, in the [`org.apache.spark.api.java` package](api/core/index.html#org.apache.spark.api.java.package), because -some of the classes are implemented in Scala. The main downside is that the types and function +some of the classes are implemented in Scala. It is important to note that the types and function definitions show Scala syntax (for example, `def reduce(func: Function2[T, T]): T` instead of -`T reduce(Function2 func)`). -We hope to generate documentation with Java-style syntax in the future. +`T reduce(Function2 func)`). In addition, the Scala `trait` modifier is used for Java +interface classes. We hope to generate documentation with Java-style syntax in the future to +avoid these quirks. # Where to Go from Here diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index d552c47b22231..6b49244ba459d 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -45,7 +45,7 @@ static class DataPoint implements Serializable { double y; } - static class ParsePoint extends Function { + static class ParsePoint implements Function { private static final Pattern SPACE = Pattern.compile(" "); @Override @@ -60,7 +60,7 @@ public DataPoint call(String line) { } } - static class VectorSum extends Function2 { + static class VectorSum implements Function2 { @Override public double[] call(double[] a, double[] b) { double[] result = new double[D]; @@ -71,7 +71,7 @@ public double[] call(double[] a, double[] b) { } } - static class ComputeGradient extends Function { + static class ComputeGradient implements Function { private final double[] weights; ComputeGradient(double[] weights) { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java index 0dc879275a22a..2d797279d5bcc 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java @@ -98,7 +98,7 @@ public Vector call(String line) { double tempDist; do { // allocate each vector to closest centroid - JavaPairRDD closest = data.map( + JavaPairRDD closest = data.mapToPair( new PairFunction() { @Override public Tuple2 call(Vector vector) { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 9eb1cadd71d22..a518fe2f27eb0 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -108,7 +108,7 @@ public static void main(String[] args) { JavaRDD dataSet = (args.length == 2) ? jsc.textFile(args[1]) : jsc.parallelize(exampleApacheLogs); - JavaPairRDD, Stats> extracted = dataSet.map(new PairFunction, Stats>() { + JavaPairRDD, Stats> extracted = dataSet.mapToPair(new PairFunction, Stats>() { @Override public Tuple2, Stats> call(String s) { return new Tuple2, Stats>(extractKey(s), extractStats(s)); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index a84245b0c7449..e53925b50c2ce 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -42,7 +42,7 @@ public final class JavaPageRank { private static final Pattern SPACES = Pattern.compile("\\s+"); - private static class Sum extends Function2 { + private static class Sum implements Function2 { @Override public Double call(Double a, Double b) { return a + b; @@ -66,7 +66,7 @@ public static void main(String[] args) throws Exception { JavaRDD lines = ctx.textFile(args[1], 1); // Loads all URLs from input file and initialize their neighbors. - JavaPairRDD> links = lines.map(new PairFunction() { + JavaPairRDD> links = lines.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { String[] parts = SPACES.split(s); @@ -86,7 +86,7 @@ public Double call(List rs) { for (int current = 0; current < Integer.parseInt(args[2]); current++) { // Calculates URL contributions to the rank of other URLs. JavaPairRDD contribs = links.join(ranks).values() - .flatMap(new PairFlatMapFunction, Double>, String, Double>() { + .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { @Override public Iterable> call(Tuple2, Double> s) { List> results = new ArrayList>(); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 2ceb0fd94ba65..6cfe25c80ecc6 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -50,7 +50,7 @@ static List> generateGraph() { return new ArrayList>(edges); } - static class ProjectFn extends PairFunction>, + static class ProjectFn implements PairFunction>, Integer, Integer> { static final ProjectFn INSTANCE = new ProjectFn(); @@ -77,7 +77,7 @@ public static void main(String[] args) { // the graph to obtain the path (x, z). // Because join() joins on keys, the edges are stored in reversed order. - JavaPairRDD edges = tc.map( + JavaPairRDD edges = tc.mapToPair( new PairFunction, Integer, Integer>() { @Override public Tuple2 call(Tuple2 e) { @@ -91,7 +91,7 @@ public Tuple2 call(Tuple2 e) { oldCount = nextCount; // Perform the join, obtaining an RDD of (y, (z, x)) pairs, // then project the result to obtain the new (x, z) paths. - tc = tc.union(tc.join(edges).map(ProjectFn.INSTANCE)).distinct().cache(); + tc = tc.union(tc.join(edges).mapToPair(ProjectFn.INSTANCE)).distinct().cache(); nextCount = tc.count(); } while (nextCount != oldCount); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 6651f98d56711..fa1b977ab19f1 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -49,7 +49,7 @@ public Iterable call(String s) { } }); - JavaPairRDD ones = words.map(new PairFunction() { + JavaPairRDD ones = words.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { return new Tuple2(s, 1); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 435a86e62abc5..64a3a04fb7296 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -35,7 +35,7 @@ */ public final class JavaALS { - static class ParseRating extends Function { + static class ParseRating implements Function { private static final Pattern COMMA = Pattern.compile(","); @Override @@ -48,7 +48,7 @@ public Rating call(String line) { } } - static class FeaturesToString extends Function, String> { + static class FeaturesToString implements Function, String> { @Override public String call(Tuple2 element) { return element._1() + "," + Arrays.toString(element._2()); diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index 4b2658f257b3c..76ebdccfd6b67 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -32,7 +32,7 @@ */ public final class JavaKMeans { - static class ParsePoint extends Function { + static class ParsePoint implements Function { private static final Pattern SPACE = Pattern.compile(" "); @Override diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index 21586ce817d09..667c72f379e71 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -34,7 +34,7 @@ */ public final class JavaLR { - static class ParsePoint extends Function { + static class ParsePoint implements Function { private static final Pattern COMMA = Pattern.compile(","); private static final Pattern SPACE = Pattern.compile(" "); diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java index 2ffd351b4e498..d704be08d6945 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaKafkaWordCount.java @@ -89,7 +89,7 @@ public Iterable call(String x) { } }); - JavaPairDStream wordCounts = words.map( + JavaPairDStream wordCounts = words.mapToPair( new PairFunction() { @Override public Tuple2 call(String s) { diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java index 7777c9832abd3..7f68d451e9b31 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaNetworkWordCount.java @@ -69,7 +69,7 @@ public Iterable call(String x) { return Lists.newArrayList(SPACE.split(x)); } }); - JavaPairDStream wordCounts = words.map( + JavaPairDStream wordCounts = words.mapToPair( new PairFunction() { @Override public Tuple2 call(String s) { diff --git a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java index 26c44620abec1..88ad341641e0a 100644 --- a/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java +++ b/examples/src/main/java/org/apache/spark/streaming/examples/JavaQueueStream.java @@ -63,7 +63,7 @@ public static void main(String[] args) throws Exception { // Create the QueueInputDStream and use it do some processing JavaDStream inputStream = ssc.queueStream(rddQueue); - JavaPairDStream mappedStream = inputStream.map( + JavaPairDStream mappedStream = inputStream.mapToPair( new PairFunction() { @Override public Tuple2 call(Integer i) { diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala index c989ec0f27465..b254e00714621 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala @@ -75,7 +75,7 @@ object ZeroMQUtils { ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel, supervisorStrategy) } @@ -99,7 +99,7 @@ object ZeroMQUtils { ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator createStream[T](jssc.ssc, publisherUrl, subscribe, fn, storageLevel) } @@ -122,7 +122,7 @@ object ZeroMQUtils { ): JavaDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - val fn = (x: Seq[ByteString]) => bytesToObjects.apply(x.map(_.toArray).toArray).toIterator + val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator createStream[T](jssc.ssc, publisherUrl, subscribe, fn) } } diff --git a/extras/README.md b/extras/README.md new file mode 100644 index 0000000000000..1b4174b7d5cff --- /dev/null +++ b/extras/README.md @@ -0,0 +1 @@ +This directory contains build components not included by default in Spark's build. diff --git a/extras/java8-tests/README.md b/extras/java8-tests/README.md new file mode 100644 index 0000000000000..e95b73ac7702a --- /dev/null +++ b/extras/java8-tests/README.md @@ -0,0 +1,24 @@ +# Java 8 Test Suites + +These tests require having Java 8 installed and are isolated from the main Spark build. +If Java 8 is not your system's default Java version, you will need to point Spark's build +to your Java location. The set-up depends a bit on the build system: + +* Sbt users can either set JAVA_HOME to the location of a Java 8 JDK or explicitly pass + `-java-home` to the sbt launch script. If a Java 8 JDK is detected sbt will automatically + include the Java 8 test project. + + `$ JAVA_HOME=/opt/jdk1.8.0/ sbt/sbt clean "test-only org.apache.spark.Java8APISuite"` + +* For Maven users, + + Maven users can also refer to their Java 8 directory using JAVA_HOME. However, Maven will not + automatically detect the presence of a Java 8 JDK, so a special build profile `-Pjava8-tests` + must be used. + + `$ JAVA_HOME=/opt/jdk1.8.0/ mvn clean install -DskipTests` + `$ JAVA_HOME=/opt/jdk1.8.0/ mvn test -Pjava8-tests -DwildcardSuites=org.apache.spark.Java8APISuite` + + Note that the above command can only be run from project root directory since this module + depends on core and the test-jars of core and streaming. This means an install step is + required to make the test dependencies visible to the Java 8 sub-project. diff --git a/extras/java8-tests/pom.xml b/extras/java8-tests/pom.xml new file mode 100644 index 0000000000000..602f66f9c5cf1 --- /dev/null +++ b/extras/java8-tests/pom.xml @@ -0,0 +1,151 @@ + + + + 4.0.0 + + org.apache.spark + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + org.apache.spark + java8-tests_2.10 + pom + Spark Project Java8 Tests POM + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + + + org.apache.spark + spark-streaming_${scala.binary.version} + ${project.version} + test-jar + + + com.novocode + junit-interface + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + + + + java8-tests + + + + + + org.apache.maven.plugins + maven-surefire-plugin + + + test + + test + + + + + + + + file:src/test/resources/log4j.properties + + + false + + **/Suite*.java + **/*Suite.java + + + + + org.apache.maven.plugins + maven-compiler-plugin + + + test-compile-first + process-test-resources + + testCompile + + + + + true + true + true + 1.8 + 1.8 + 1.8 + UTF-8 + 1024m + + + + + net.alchim31.maven + scala-maven-plugin + + + none + + + scala-compile-first + none + + + scala-test-compile-first + none + + + attach-scaladocs + none + + + + + org.scalatest + scalatest-maven-plugin + + + test + none + + + + + + diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java new file mode 100644 index 0000000000000..f67251217ed4a --- /dev/null +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -0,0 +1,391 @@ +/* + * 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; + +import java.io.File; +import java.io.Serializable; +import java.util.*; + +import scala.Tuple2; + +import com.google.common.base.Optional; +import com.google.common.io.Files; +import org.apache.hadoop.io.IntWritable; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.mapred.SequenceFileOutputFormat; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import org.apache.spark.api.java.JavaDoubleRDD; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.*; + +/** + * Most of these tests replicate org.apache.spark.JavaAPISuite using java 8 + * lambda syntax. + */ +public class Java8APISuite implements Serializable { + static int foreachCalls = 0; + private transient JavaSparkContext sc; + + @Before + public void setUp() { + sc = new JavaSparkContext("local", "JavaAPISuite"); + } + + @After + public void tearDown() { + sc.stop(); + sc = null; + // To avoid Akka rebinding to the same port, since it doesn't unbind immediately on shutdown + System.clearProperty("spark.driver.port"); + } + + @Test + public void foreachWithAnonymousClass() { + foreachCalls = 0; + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); + rdd.foreach(new VoidFunction() { + @Override + public void call(String s) { + foreachCalls++; + } + }); + Assert.assertEquals(2, foreachCalls); + } + + @Test + public void foreach() { + foreachCalls = 0; + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello", "World")); + rdd.foreach((x) -> foreachCalls++); + Assert.assertEquals(2, foreachCalls); + } + + @Test + public void groupBy() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function isOdd = x -> x % 2 == 0; + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + Assert.assertEquals(2, oddsAndEvens.count()); + Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens + Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + + oddsAndEvens = rdd.groupBy(isOdd, 1); + Assert.assertEquals(2, oddsAndEvens.count()); + Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens + Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + } + + @Test + public void leftOuterJoin() { + JavaPairRDD rdd1 = sc.parallelizePairs(Arrays.asList( + new Tuple2(1, 1), + new Tuple2(1, 2), + new Tuple2(2, 1), + new Tuple2(3, 1) + )); + JavaPairRDD rdd2 = sc.parallelizePairs(Arrays.asList( + new Tuple2(1, 'x'), + new Tuple2(2, 'y'), + new Tuple2(2, 'z'), + new Tuple2(4, 'w') + )); + List>>> joined = + rdd1.leftOuterJoin(rdd2).collect(); + Assert.assertEquals(5, joined.size()); + Tuple2>> firstUnmatched = + rdd1.leftOuterJoin(rdd2).filter(tup -> !tup._2()._2().isPresent()).first(); + Assert.assertEquals(3, firstUnmatched._1().intValue()); + } + + @Test + public void foldReduce() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); + Function2 add = (a, b) -> a + b; + + int sum = rdd.fold(0, add); + Assert.assertEquals(33, sum); + + sum = rdd.reduce(add); + Assert.assertEquals(33, sum); + } + + @Test + public void foldByKey() { + List> pairs = Arrays.asList( + new Tuple2(2, 1), + new Tuple2(2, 1), + new Tuple2(1, 1), + new Tuple2(3, 2), + new Tuple2(3, 1) + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + JavaPairRDD sums = rdd.foldByKey(0, (a, b) -> a + b); + Assert.assertEquals(1, sums.lookup(1).get(0).intValue()); + Assert.assertEquals(2, sums.lookup(2).get(0).intValue()); + Assert.assertEquals(3, sums.lookup(3).get(0).intValue()); + } + + @Test + public void reduceByKey() { + List> pairs = Arrays.asList( + new Tuple2(2, 1), + new Tuple2(2, 1), + new Tuple2(1, 1), + new Tuple2(3, 2), + new Tuple2(3, 1) + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + JavaPairRDD counts = rdd.reduceByKey((a, b) -> a + b); + Assert.assertEquals(1, counts.lookup(1).get(0).intValue()); + Assert.assertEquals(2, counts.lookup(2).get(0).intValue()); + Assert.assertEquals(3, counts.lookup(3).get(0).intValue()); + + Map localCounts = counts.collectAsMap(); + Assert.assertEquals(1, localCounts.get(1).intValue()); + Assert.assertEquals(2, localCounts.get(2).intValue()); + Assert.assertEquals(3, localCounts.get(3).intValue()); + + localCounts = rdd.reduceByKeyLocally((a, b) -> a + b); + Assert.assertEquals(1, localCounts.get(1).intValue()); + Assert.assertEquals(2, localCounts.get(2).intValue()); + Assert.assertEquals(3, localCounts.get(3).intValue()); + } + + @Test + public void map() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x).cache(); + doubles.collect(); + JavaPairRDD pairs = rdd.mapToPair(x -> new Tuple2(x, x)) + .cache(); + pairs.collect(); + JavaRDD strings = rdd.map(x -> x.toString()).cache(); + strings.collect(); + } + + @Test + public void flatMap() { + JavaRDD rdd = sc.parallelize(Arrays.asList("Hello World!", + "The quick brown fox jumps over the lazy dog.")); + JavaRDD words = rdd.flatMap(x -> Arrays.asList(x.split(" "))); + + Assert.assertEquals("Hello", words.first()); + Assert.assertEquals(11, words.count()); + + JavaPairRDD pairs = rdd.flatMapToPair(s -> { + List> pairs2 = new LinkedList>(); + for (String word : s.split(" ")) pairs2.add(new Tuple2(word, word)); + return pairs2; + }); + + Assert.assertEquals(new Tuple2("Hello", "Hello"), pairs.first()); + Assert.assertEquals(11, pairs.count()); + + JavaDoubleRDD doubles = rdd.flatMapToDouble(s -> { + List lengths = new LinkedList(); + for (String word : s.split(" ")) lengths.add(word.length() * 1.0); + return lengths; + }); + + Double x = doubles.first(); + Assert.assertEquals(5.0, doubles.first().doubleValue(), 0.01); + Assert.assertEquals(11, pairs.count()); + } + + @Test + public void mapsFromPairsToPairs() { + List> pairs = Arrays.asList( + new Tuple2(1, "a"), + new Tuple2(2, "aa"), + new Tuple2(3, "aaa") + ); + JavaPairRDD pairRDD = sc.parallelizePairs(pairs); + + // Regression test for SPARK-668: + JavaPairRDD swapped = + pairRDD.flatMapToPair(x -> Collections.singletonList(x.swap())); + swapped.collect(); + + // There was never a bug here, but it's worth testing: + pairRDD.map(item -> item.swap()).collect(); + } + + @Test + public void mapPartitions() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4), 2); + JavaRDD partitionSums = rdd.mapPartitions(iter -> { + int sum = 0; + while (iter.hasNext()) { + sum += iter.next(); + } + return Collections.singletonList(sum); + }); + + Assert.assertEquals("[3, 7]", partitionSums.collect().toString()); + } + + @Test + public void sequenceFile() { + File tempDir = Files.createTempDir(); + String outputDir = new File(tempDir, "output").getAbsolutePath(); + List> pairs = Arrays.asList( + new Tuple2(1, "a"), + new Tuple2(2, "aa"), + new Tuple2(3, "aaa") + ); + JavaPairRDD rdd = sc.parallelizePairs(pairs); + + rdd.mapToPair(pair -> + new Tuple2(new IntWritable(pair._1()), new Text(pair._2()))) + .saveAsHadoopFile(outputDir, IntWritable.class, Text.class, SequenceFileOutputFormat.class); + + // Try reading the output back as an object file + JavaPairRDD readRDD = sc.sequenceFile(outputDir, IntWritable.class, Text.class) + .mapToPair(pair -> new Tuple2(pair._1().get(), pair._2().toString())); + Assert.assertEquals(pairs, readRDD.collect()); + } + + @Test + public void zip() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + JavaDoubleRDD doubles = rdd.mapToDouble(x -> 1.0 * x); + JavaPairRDD zipped = rdd.zip(doubles); + zipped.count(); + } + + @Test + public void zipPartitions() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6), 2); + JavaRDD rdd2 = sc.parallelize(Arrays.asList("1", "2", "3", "4"), 2); + FlatMapFunction2, Iterator, Integer> sizesFn = + (Iterator i, Iterator s) -> { + int sizeI = 0; + int sizeS = 0; + while (i.hasNext()) { + sizeI += 1; + i.next(); + } + while (s.hasNext()) { + sizeS += 1; + s.next(); + } + return Arrays.asList(sizeI, sizeS); + }; + JavaRDD sizes = rdd1.zipPartitions(rdd2, sizesFn); + Assert.assertEquals("[3, 2, 3, 2]", sizes.collect().toString()); + } + + @Test + public void accumulators() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5)); + + final Accumulator intAccum = sc.intAccumulator(10); + rdd.foreach(x -> intAccum.add(x)); + Assert.assertEquals((Integer) 25, intAccum.value()); + + final Accumulator doubleAccum = sc.doubleAccumulator(10.0); + rdd.foreach(x -> doubleAccum.add((double) x)); + Assert.assertEquals((Double) 25.0, doubleAccum.value()); + + // Try a custom accumulator type + AccumulatorParam floatAccumulatorParam = new AccumulatorParam() { + public Float addInPlace(Float r, Float t) { + return r + t; + } + + public Float addAccumulator(Float r, Float t) { + return r + t; + } + + public Float zero(Float initialValue) { + return 0.0f; + } + }; + + final Accumulator floatAccum = sc.accumulator((Float) 10.0f, floatAccumulatorParam); + rdd.foreach(x -> floatAccum.add((float) x)); + Assert.assertEquals((Float) 25.0f, floatAccum.value()); + + // Test the setValue method + floatAccum.setValue(5.0f); + Assert.assertEquals((Float) 5.0f, floatAccum.value()); + } + + @Test + public void keyBy() { + JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2)); + List> s = rdd.keyBy(x -> x.toString()).collect(); + Assert.assertEquals(new Tuple2("1", 1), s.get(0)); + Assert.assertEquals(new Tuple2("2", 2), s.get(1)); + } + + @Test + public void mapOnPairRDD() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4)); + JavaPairRDD rdd2 = + rdd1.mapToPair(i -> new Tuple2(i, i % 2)); + JavaPairRDD rdd3 = + rdd2.mapToPair(in -> new Tuple2(in._2(), in._1())); + Assert.assertEquals(Arrays.asList( + new Tuple2(1, 1), + new Tuple2(0, 2), + new Tuple2(1, 3), + new Tuple2(0, 4)), rdd3.collect()); + } + + @Test + public void collectPartitions() { + JavaRDD rdd1 = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5, 6, 7), 3); + + JavaPairRDD rdd2 = + rdd1.mapToPair(i -> new Tuple2(i, i % 2)); + List[] parts = rdd1.collectPartitions(new int[]{0}); + Assert.assertEquals(Arrays.asList(1, 2), parts[0]); + + parts = rdd1.collectPartitions(new int[]{1, 2}); + Assert.assertEquals(Arrays.asList(3, 4), parts[0]); + Assert.assertEquals(Arrays.asList(5, 6, 7), parts[1]); + + Assert.assertEquals(Arrays.asList(new Tuple2(1, 1), + new Tuple2(2, 0)), + rdd2.collectPartitions(new int[]{0})[0]); + + parts = rdd2.collectPartitions(new int[]{1, 2}); + Assert.assertEquals(Arrays.asList(new Tuple2(3, 1), + new Tuple2(4, 0)), parts[0]); + Assert.assertEquals(Arrays.asList(new Tuple2(5, 1), + new Tuple2(6, 0), + new Tuple2(7, 1)), parts[1]); + } + + @Test + public void collectAsMapWithIntArrayValues() { + // Regression test for SPARK-1040 + JavaRDD rdd = sc.parallelize(Arrays.asList(new Integer[]{1})); + JavaPairRDD pairRDD = + rdd.mapToPair(x -> new Tuple2(x, new int[]{x})); + pairRDD.collect(); // Works fine + Map map = pairRDD.collectAsMap(); // Used to crash with ClassCastException + } +} diff --git a/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java new file mode 100644 index 0000000000000..43df0dea614bc --- /dev/null +++ b/extras/java8-tests/src/test/java/org/apache/spark/streaming/Java8APISuite.java @@ -0,0 +1,841 @@ +/* + * 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.streaming; + +import java.io.Serializable; +import java.util.*; + +import scala.Tuple2; + +import com.google.common.base.Optional; +import com.google.common.collect.Lists; +import com.google.common.collect.Sets; +import org.junit.Assert; +import org.junit.Test; + +import org.apache.spark.HashPartitioner; +import org.apache.spark.api.java.JavaPairRDD; +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.function.PairFunction; +import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaPairDStream; + +/** + * Most of these tests replicate org.apache.spark.streaming.JavaAPISuite using java 8 + * lambda syntax. + */ +public class Java8APISuite extends LocalJavaStreamingContext implements Serializable { + + @Test + public void testMap() { + List> inputData = Arrays.asList( + Arrays.asList("hello", "world"), + Arrays.asList("goodnight", "moon")); + + List> expected = Arrays.asList( + Arrays.asList(5, 5), + Arrays.asList(9, 4)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream letterCount = stream.map(s -> s.length()); + JavaTestUtils.attachTestOutputStream(letterCount); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + assertOrderInvariantEquals(expected, result); + } + + @Test + public void testFilter() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red socks")); + + List> expected = Arrays.asList( + Arrays.asList("giants"), + Arrays.asList("yankees")); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream filtered = stream.filter(s -> s.contains("a")); + JavaTestUtils.attachTestOutputStream(filtered); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + assertOrderInvariantEquals(expected, result); + } + + @Test + public void testMapPartitions() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red socks")); + + List> expected = Arrays.asList( + Arrays.asList("GIANTSDODGERS"), + Arrays.asList("YANKEESRED SOCKS")); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream mapped = stream.mapPartitions(in -> { + String out = ""; + while (in.hasNext()) { + out = out + in.next().toUpperCase(); + } + return Lists.newArrayList(out); + }); + JavaTestUtils.attachTestOutputStream(mapped); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testReduce() { + List> inputData = Arrays.asList( + Arrays.asList(1, 2, 3), + Arrays.asList(4, 5, 6), + Arrays.asList(7, 8, 9)); + + List> expected = Arrays.asList( + Arrays.asList(6), + Arrays.asList(15), + Arrays.asList(24)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reduced = stream.reduce((x, y) -> x + y); + JavaTestUtils.attachTestOutputStream(reduced); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @Test + public void testReduceByWindow() { + List> inputData = Arrays.asList( + Arrays.asList(1, 2, 3), + Arrays.asList(4, 5, 6), + Arrays.asList(7, 8, 9)); + + List> expected = Arrays.asList( + Arrays.asList(6), + Arrays.asList(21), + Arrays.asList(39), + Arrays.asList(24)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream reducedWindowed = stream.reduceByWindow((x, y) -> x + y, + (x, y) -> x - y, new Duration(2000), new Duration(1000)); + JavaTestUtils.attachTestOutputStream(reducedWindowed); + List> result = JavaTestUtils.runStreams(ssc, 4, 4); + + Assert.assertEquals(expected, result); + } + + @Test + public void testTransform() { + List> inputData = Arrays.asList( + Arrays.asList(1, 2, 3), + Arrays.asList(4, 5, 6), + Arrays.asList(7, 8, 9)); + + List> expected = Arrays.asList( + Arrays.asList(3, 4, 5), + Arrays.asList(6, 7, 8), + Arrays.asList(9, 10, 11)); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream transformed = stream.transform(in -> in.map(i -> i + 2)); + + JavaTestUtils.attachTestOutputStream(transformed); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + assertOrderInvariantEquals(expected, result); + } + + @Test + public void testVariousTransform() { + // tests whether all variations of transform can be called from Java + + List> inputData = Arrays.asList(Arrays.asList(1)); + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + + List>> pairInputData = + Arrays.asList(Arrays.asList(new Tuple2("x", 1))); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairInputData, 1)); + + JavaDStream transformed1 = stream.transform(in -> null); + JavaDStream transformed2 = stream.transform((x, time) -> null); + JavaPairDStream transformed3 = stream.transformToPair(x -> null); + JavaPairDStream transformed4 = stream.transformToPair((x, time) -> null); + JavaDStream pairTransformed1 = pairStream.transform(x -> null); + JavaDStream pairTransformed2 = pairStream.transform((x, time) -> null); + JavaPairDStream pairTransformed3 = pairStream.transformToPair(x -> null); + JavaPairDStream pairTransformed4 = + pairStream.transformToPair((x, time) -> null); + + } + + @Test + public void testTransformWith() { + List>> stringStringKVStream1 = Arrays.asList( + Arrays.asList( + new Tuple2("california", "dodgers"), + new Tuple2("new york", "yankees")), + Arrays.asList( + new Tuple2("california", "sharks"), + new Tuple2("new york", "rangers"))); + + List>> stringStringKVStream2 = Arrays.asList( + Arrays.asList( + new Tuple2("california", "giants"), + new Tuple2("new york", "mets")), + Arrays.asList( + new Tuple2("california", "ducks"), + new Tuple2("new york", "islanders"))); + + + List>>> expected = Arrays.asList( + Sets.newHashSet( + new Tuple2>("california", + new Tuple2("dodgers", "giants")), + new Tuple2>("new york", + new Tuple2("yankees", "mets"))), + Sets.newHashSet( + new Tuple2>("california", + new Tuple2("sharks", "ducks")), + new Tuple2>("new york", + new Tuple2("rangers", "islanders")))); + + JavaDStream> stream1 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream1, 1); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream(stream1); + + JavaDStream> stream2 = JavaTestUtils.attachTestInputStream( + ssc, stringStringKVStream2, 1); + JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); + + JavaPairDStream> joined = + pairStream1.transformWithToPair(pairStream2,(x, y, z) -> x.join(y)); + + JavaTestUtils.attachTestOutputStream(joined); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + List>>> unorderedResult = Lists.newArrayList(); + for (List>> res : result) { + unorderedResult.add(Sets.newHashSet(res)); + } + + Assert.assertEquals(expected, unorderedResult); + } + + + @Test + public void testVariousTransformWith() { + // tests whether all variations of transformWith can be called from Java + + List> inputData1 = Arrays.asList(Arrays.asList(1)); + List> inputData2 = Arrays.asList(Arrays.asList("x")); + JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, inputData1, 1); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, inputData2, 1); + + List>> pairInputData1 = + Arrays.asList(Arrays.asList(new Tuple2("x", 1))); + List>> pairInputData2 = + Arrays.asList(Arrays.asList(new Tuple2(1.0, 'x'))); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairInputData1, 1)); + JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairInputData2, 1)); + + JavaDStream transformed1 = stream1.transformWith(stream2, (x, y, z) -> null); + JavaDStream transformed2 = stream1.transformWith(pairStream1,(x, y, z) -> null); + + JavaPairDStream transformed3 = + stream1.transformWithToPair(stream2,(x, y, z) -> null); + + JavaPairDStream transformed4 = + stream1.transformWithToPair(pairStream1,(x, y, z) -> null); + + JavaDStream pairTransformed1 = pairStream1.transformWith(stream2,(x, y, z) -> null); + + JavaDStream pairTransformed2_ = + pairStream1.transformWith(pairStream1,(x, y, z) -> null); + + JavaPairDStream pairTransformed3 = + pairStream1.transformWithToPair(stream2,(x, y, z) -> null); + + JavaPairDStream pairTransformed4 = + pairStream1.transformWithToPair(pairStream2,(x, y, z) -> null); + } + + @Test + public void testStreamingContextTransform() { + List> stream1input = Arrays.asList( + Arrays.asList(1), + Arrays.asList(2) + ); + + List> stream2input = Arrays.asList( + Arrays.asList(3), + Arrays.asList(4) + ); + + List>> pairStream1input = Arrays.asList( + Arrays.asList(new Tuple2(1, "x")), + Arrays.asList(new Tuple2(2, "y")) + ); + + List>>> expected = Arrays.asList( + Arrays.asList(new Tuple2>(1, new Tuple2(1, "x"))), + Arrays.asList(new Tuple2>(2, new Tuple2(2, "y"))) + ); + + JavaDStream stream1 = JavaTestUtils.attachTestInputStream(ssc, stream1input, 1); + JavaDStream stream2 = JavaTestUtils.attachTestInputStream(ssc, stream2input, 1); + JavaPairDStream pairStream1 = JavaPairDStream.fromJavaDStream( + JavaTestUtils.attachTestInputStream(ssc, pairStream1input, 1)); + + List> listOfDStreams1 = Arrays.>asList(stream1, stream2); + + // This is just to test whether this transform to JavaStream compiles + JavaDStream transformed1 = ssc.transform( + listOfDStreams1, (List> listOfRDDs, Time time) -> { + assert (listOfRDDs.size() == 2); + return null; + }); + + List> listOfDStreams2 = + Arrays.>asList(stream1, stream2, pairStream1.toJavaDStream()); + + JavaPairDStream> transformed2 = ssc.transformToPair( + listOfDStreams2, (List> listOfRDDs, Time time) -> { + assert (listOfRDDs.size() == 3); + JavaRDD rdd1 = (JavaRDD) listOfRDDs.get(0); + JavaRDD rdd2 = (JavaRDD) listOfRDDs.get(1); + JavaRDD> rdd3 = (JavaRDD>) listOfRDDs.get(2); + JavaPairRDD prdd3 = JavaPairRDD.fromJavaRDD(rdd3); + PairFunction mapToTuple = + (Integer i) -> new Tuple2(i, i); + return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3); + }); + JavaTestUtils.attachTestOutputStream(transformed2); + List>>> result = + JavaTestUtils.runStreams(ssc, 2, 2); + Assert.assertEquals(expected, result); + } + + @Test + public void testFlatMap() { + List> inputData = Arrays.asList( + Arrays.asList("go", "giants"), + Arrays.asList("boo", "dodgers"), + Arrays.asList("athletics")); + + List> expected = Arrays.asList( + Arrays.asList("g", "o", "g", "i", "a", "n", "t", "s"), + Arrays.asList("b", "o", "o", "d", "o", "d", "g", "e", "r", "s"), + Arrays.asList("a", "t", "h", "l", "e", "t", "i", "c", "s")); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaDStream flatMapped = stream.flatMap(s -> Lists.newArrayList(s.split("(?!^)"))); + JavaTestUtils.attachTestOutputStream(flatMapped); + List> result = JavaTestUtils.runStreams(ssc, 3, 3); + + assertOrderInvariantEquals(expected, result); + } + + @Test + public void testPairFlatMap() { + List> inputData = Arrays.asList( + Arrays.asList("giants"), + Arrays.asList("dodgers"), + Arrays.asList("athletics")); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(6, "g"), + new Tuple2(6, "i"), + new Tuple2(6, "a"), + new Tuple2(6, "n"), + new Tuple2(6, "t"), + new Tuple2(6, "s")), + Arrays.asList( + new Tuple2(7, "d"), + new Tuple2(7, "o"), + new Tuple2(7, "d"), + new Tuple2(7, "g"), + new Tuple2(7, "e"), + new Tuple2(7, "r"), + new Tuple2(7, "s")), + Arrays.asList( + new Tuple2(9, "a"), + new Tuple2(9, "t"), + new Tuple2(9, "h"), + new Tuple2(9, "l"), + new Tuple2(9, "e"), + new Tuple2(9, "t"), + new Tuple2(9, "i"), + new Tuple2(9, "c"), + new Tuple2(9, "s"))); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream flatMapped = stream.flatMapToPair(s -> { + List> out = Lists.newArrayList(); + for (String letter : s.split("(?!^)")) { + out.add(new Tuple2(s.length(), letter)); + } + return out; + }); + + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + /* + * Performs an order-invariant comparison of lists representing two RDD streams. This allows + * us to account for ordering variation within individual RDD's which occurs during windowing. + */ + public static > void assertOrderInvariantEquals( + List> expected, List> actual) { + for (List list : expected) { + Collections.sort(list); + } + for (List list : actual) { + Collections.sort(list); + } + Assert.assertEquals(expected, actual); + } + + @Test + public void testPairFilter() { + List> inputData = Arrays.asList( + Arrays.asList("giants", "dodgers"), + Arrays.asList("yankees", "red socks")); + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2("giants", 6)), + Arrays.asList(new Tuple2("yankees", 7))); + + JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = + stream.mapToPair(x -> new Tuple2<>(x, x.length())); + JavaPairDStream filtered = pairStream.filter(x -> x._1().contains("a")); + JavaTestUtils.attachTestOutputStream(filtered); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + List>> stringStringKVStream = Arrays.asList( + Arrays.asList(new Tuple2("california", "dodgers"), + new Tuple2("california", "giants"), + new Tuple2("new york", "yankees"), + new Tuple2("new york", "mets")), + Arrays.asList(new Tuple2("california", "sharks"), + new Tuple2("california", "ducks"), + new Tuple2("new york", "rangers"), + new Tuple2("new york", "islanders"))); + + List>> stringIntKVStream = Arrays.asList( + Arrays.asList( + new Tuple2("california", 1), + new Tuple2("california", 3), + new Tuple2("new york", 4), + new Tuple2("new york", 1)), + Arrays.asList( + new Tuple2("california", 5), + new Tuple2("california", 5), + new Tuple2("new york", 3), + new Tuple2("new york", 1))); + + @Test + public void testPairMap() { // Maps pair -> pair of different type + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(1, "california"), + new Tuple2(3, "california"), + new Tuple2(4, "new york"), + new Tuple2(1, "new york")), + Arrays.asList( + new Tuple2(5, "california"), + new Tuple2(5, "california"), + new Tuple2(3, "new york"), + new Tuple2(1, "new york"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream reversed = pairStream.mapToPair(x -> x.swap()); + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairMapPartitions() { // Maps pair -> pair of different type + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(1, "california"), + new Tuple2(3, "california"), + new Tuple2(4, "new york"), + new Tuple2(1, "new york")), + Arrays.asList( + new Tuple2(5, "california"), + new Tuple2(5, "california"), + new Tuple2(3, "new york"), + new Tuple2(1, "new york"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream reversed = pairStream.mapPartitionsToPair(in -> { + LinkedList> out = new LinkedList>(); + while (in.hasNext()) { + Tuple2 next = in.next(); + out.add(next.swap()); + } + return out; + }); + + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairMap2() { // Maps pair -> single + List>> inputData = stringIntKVStream; + + List> expected = Arrays.asList( + Arrays.asList(1, 3, 4, 1), + Arrays.asList(5, 5, 3, 1)); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaDStream reversed = pairStream.map(in -> in._2()); + JavaTestUtils.attachTestOutputStream(reversed); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2("hi", 1), + new Tuple2("ho", 2)), + Arrays.asList( + new Tuple2("hi", 1), + new Tuple2("ho", 2))); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(1, "h"), + new Tuple2(1, "i"), + new Tuple2(2, "h"), + new Tuple2(2, "o")), + Arrays.asList( + new Tuple2(1, "h"), + new Tuple2(1, "i"), + new Tuple2(2, "h"), + new Tuple2(2, "o"))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaPairDStream flatMapped = pairStream.flatMapToPair(in -> { + List> out = new LinkedList>(); + for (Character s : in._1().toCharArray()) { + out.add(new Tuple2(in._2(), s.toString())); + } + return out; + }); + + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairReduceByKey() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2("california", 4), + new Tuple2("new york", 5)), + Arrays.asList( + new Tuple2("california", 10), + new Tuple2("new york", 4))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream reduced = pairStream.reduceByKey((x, y) -> x + y); + + JavaTestUtils.attachTestOutputStream(reduced); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testCombineByKey() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2("california", 4), + new Tuple2("new york", 5)), + Arrays.asList( + new Tuple2("california", 10), + new Tuple2("new york", 4))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream combined = pairStream.combineByKey(i -> i, + (x, y) -> x + y, (x, y) -> x + y, new HashPartitioner(2)); + + JavaTestUtils.attachTestOutputStream(combined); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testReduceByKeyAndWindow() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2("california", 4), + new Tuple2("new york", 5)), + Arrays.asList(new Tuple2("california", 14), + new Tuple2("new york", 9)), + Arrays.asList(new Tuple2("california", 10), + new Tuple2("new york", 4))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream reduceWindowed = + pairStream.reduceByKeyAndWindow((x, y) -> x + y, new Duration(2000), new Duration(1000)); + JavaTestUtils.attachTestOutputStream(reduceWindowed); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @Test + public void testUpdateStateByKey() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2("california", 4), + new Tuple2("new york", 5)), + Arrays.asList(new Tuple2("california", 14), + new Tuple2("new york", 9)), + Arrays.asList(new Tuple2("california", 14), + new Tuple2("new york", 9))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream updated = pairStream.updateStateByKey((values, state) -> { + int out = 0; + if (state.isPresent()) { + out = out + state.get(); + } + for (Integer v : values) { + out = out + v; + } + return Optional.of(out); + }); + + JavaTestUtils.attachTestOutputStream(updated); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @Test + public void testReduceByKeyAndWindowWithInverse() { + List>> inputData = stringIntKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2("california", 4), + new Tuple2("new york", 5)), + Arrays.asList(new Tuple2("california", 14), + new Tuple2("new york", 9)), + Arrays.asList(new Tuple2("california", 10), + new Tuple2("new york", 4))); + + JavaDStream> stream = + JavaTestUtils.attachTestInputStream(ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream reduceWindowed = + pairStream.reduceByKeyAndWindow((x, y) -> x + y, (x, y) -> x - y, new Duration(2000), + new Duration(1000)); + JavaTestUtils.attachTestOutputStream(reduceWindowed); + List>> result = JavaTestUtils.runStreams(ssc, 3, 3); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairTransform() { + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2(3, 5), + new Tuple2(1, 5), + new Tuple2(4, 5), + new Tuple2(2, 5)), + Arrays.asList( + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5), + new Tuple2(1, 5))); + + List>> expected = Arrays.asList( + Arrays.asList( + new Tuple2(1, 5), + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5)), + Arrays.asList( + new Tuple2(1, 5), + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream sorted = pairStream.transformToPair(in -> in.sortByKey()); + + JavaTestUtils.attachTestOutputStream(sorted); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testPairToNormalRDDTransform() { + List>> inputData = Arrays.asList( + Arrays.asList( + new Tuple2(3, 5), + new Tuple2(1, 5), + new Tuple2(4, 5), + new Tuple2(2, 5)), + Arrays.asList( + new Tuple2(2, 5), + new Tuple2(3, 5), + new Tuple2(4, 5), + new Tuple2(1, 5))); + + List> expected = Arrays.asList( + Arrays.asList(3, 1, 4, 2), + Arrays.asList(2, 3, 4, 1)); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + JavaDStream firstParts = pairStream.transform(in -> in.map(x -> x._1())); + JavaTestUtils.attachTestOutputStream(firstParts); + List> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testMapValues() { + List>> inputData = stringStringKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2("california", "DODGERS"), + new Tuple2("california", "GIANTS"), + new Tuple2("new york", "YANKEES"), + new Tuple2("new york", "METS")), + Arrays.asList(new Tuple2("california", "SHARKS"), + new Tuple2("california", "DUCKS"), + new Tuple2("new york", "RANGERS"), + new Tuple2("new york", "ISLANDERS"))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + JavaPairDStream mapped = pairStream.mapValues(s -> s.toUpperCase()); + JavaTestUtils.attachTestOutputStream(mapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected, result); + } + + @Test + public void testFlatMapValues() { + List>> inputData = stringStringKVStream; + + List>> expected = Arrays.asList( + Arrays.asList(new Tuple2("california", "dodgers1"), + new Tuple2("california", "dodgers2"), + new Tuple2("california", "giants1"), + new Tuple2("california", "giants2"), + new Tuple2("new york", "yankees1"), + new Tuple2("new york", "yankees2"), + new Tuple2("new york", "mets1"), + new Tuple2("new york", "mets2")), + Arrays.asList(new Tuple2("california", "sharks1"), + new Tuple2("california", "sharks2"), + new Tuple2("california", "ducks1"), + new Tuple2("california", "ducks2"), + new Tuple2("new york", "rangers1"), + new Tuple2("new york", "rangers2"), + new Tuple2("new york", "islanders1"), + new Tuple2("new york", "islanders2"))); + + JavaDStream> stream = JavaTestUtils.attachTestInputStream( + ssc, inputData, 1); + JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); + + + JavaPairDStream flatMapped = pairStream.flatMapValues(in -> { + List out = new ArrayList(); + out.add(in + "1"); + out.add(in + "2"); + return out; + }); + JavaTestUtils.attachTestOutputStream(flatMapped); + List>> result = JavaTestUtils.runStreams(ssc, 2, 2); + Assert.assertEquals(expected, result); + } + +} diff --git a/extras/java8-tests/src/test/resources/log4j.properties b/extras/java8-tests/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..180beaa8cc5a7 --- /dev/null +++ b/extras/java8-tests/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 target/unit-tests.log +log4j.rootCategory=INFO, file +log4j.appender.file=org.apache.log4j.FileAppender +log4j.appender.file.append=false +log4j.appender.file.file=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 diff --git a/pom.xml b/pom.xml index 7e28d7c1941bb..c59fada5cd4a0 100644 --- a/pom.xml +++ b/pom.xml @@ -710,6 +710,31 @@ yarn + + + java8-tests + + + + + org.apache.maven.plugins + maven-jar-plugin + 2.4 + + + + test-jar + + + + + + + + + extras/java8-tests + + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d45f6773fad78..aa1784897566b 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -90,6 +90,14 @@ object SparkBuild extends Build { } lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() + + // Conditionally include the java 8 sub-project + lazy val javaVersion = System.getProperty("java.specification.version") + lazy val isJava8Enabled = javaVersion.toDouble >= "1.8".toDouble + val maybeJava8Tests = if (isJava8Enabled) Seq[ProjectReference](java8Tests) else Seq[ProjectReference]() + lazy val java8Tests = Project("java8-tests", file("extras/java8-tests"), settings = java8TestsSettings). + dependsOn(core) dependsOn(streaming % "compile->compile;test->test") + // Conditionally include the yarn sub-project lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core) lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core) @@ -118,10 +126,11 @@ object SparkBuild extends Build { lazy val examples = Project("examples", file("examples"), settings = examplesSettings) .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) - // Everything except assembly, tools and examples belong to packageProjects + // Everything except assembly, tools, java8Tests and examples belong to packageProjects lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ maybeYarnRef - lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) + lazy val allProjects = packageProjects ++ allExternalRefs ++ + Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests def sharedSettings = Defaults.defaultSettings ++ Seq( organization := "org.apache.spark", @@ -132,6 +141,7 @@ object SparkBuild extends Build { javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, retrieveManaged := true, + javaHome := Properties.envOrNone("JAVA_HOME").map(file), // This is to add convenience of enabling sbt -Dsbt.offline=true for making the build offline. offline := "true".equalsIgnoreCase(sys.props("sbt.offline")), retrievePattern := "[type]s/[artifact](-[revision])(-[classifier]).[ext]", @@ -370,6 +380,12 @@ object SparkBuild extends Build { name := "spark-yarn" ) + def java8TestsSettings = sharedSettings ++ Seq( + name := "java8-tests", + javacOptions := Seq("-target", "1.8", "-source", "1.8"), + testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a") + ) + // Conditionally include the YARN dependencies because some tools look at all sub-projects and will complain // if we refer to nonexistent dependencies (e.g. hadoop-yarn-api from a Hadoop version without YARN). def extraYarnSettings = if(isYarnEnabled) yarnEnabledSettings else Seq() diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash index d65bbdc19ce03..00a6b41013e5f 100755 --- a/sbt/sbt-launch-lib.bash +++ b/sbt/sbt-launch-lib.bash @@ -16,7 +16,14 @@ declare -a residual_args declare -a java_args declare -a scalac_args declare -a sbt_commands -declare java_cmd=java + +if test -x "$JAVA_HOME/bin/java"; then + echo -e "Using $JAVA_HOME as default JAVA_HOME." + echo "Note, this will be overridden by -java-home if it is set." + declare java_cmd="$JAVA_HOME/bin/java" +else + declare java_cmd=java +fi echoerr () { echo 1>&2 "$@" @@ -131,7 +138,7 @@ process_args () { -sbt-jar) require_arg path "$1" "$2" && sbt_jar="$2" && shift 2 ;; -sbt-version) require_arg version "$1" "$2" && sbt_version="$2" && shift 2 ;; - -java-home) require_arg path "$1" "$2" && java_cmd="$2/bin/java" && shift 2 ;; + -java-home) require_arg path "$1" "$2" && java_cmd="$2/bin/java" && export JAVA_HOME=$2 && shift 2 ;; -D*) addJava "$1" && shift ;; -J*) addJava "${1:2}" && shift ;; diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index e23b725052864..721d50273259e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -41,7 +41,7 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T] /** Return a new DStream containing only the elements that satisfy a predicate. */ def filter(f: JFunction[T, java.lang.Boolean]): JavaDStream[T] = - dstream.filter((x => f(x).booleanValue())) + dstream.filter((x => f.call(x).booleanValue())) /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */ def cache(): JavaDStream[T] = dstream.cache() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index 7aa7ead29b469..a85cd04c9319c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -17,19 +17,20 @@ package org.apache.spark.streaming.api.java -import java.util.{List => JList} +import java.util import java.lang.{Long => JLong} +import java.util.{List => JList} import scala.collection.JavaConversions._ import scala.reflect.ClassTag -import org.apache.spark.streaming._ -import org.apache.spark.api.java.{JavaPairRDD, JavaRDDLike, JavaRDD} -import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} -import org.apache.spark.api.java.function.{Function3 => JFunction3, _} -import java.util +import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaRDDLike} +import org.apache.spark.api.java.JavaPairRDD._ +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, Function3 => JFunction3, _} import org.apache.spark.rdd.RDD -import JavaDStream._ +import org.apache.spark.streaming._ +import org.apache.spark.streaming.api.java.JavaDStream._ import org.apache.spark.streaming.dstream.DStream trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T, R]] @@ -123,23 +124,23 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * this DStream. Applying glom() to an RDD coalesces all elements within each partition into * an array. */ - def glom(): JavaDStream[JList[T]] = { + def glom(): JavaDStream[JList[T]] = new JavaDStream(dstream.glom().map(x => new java.util.ArrayList[T](x.toSeq))) - } + /** Return the [[org.apache.spark.streaming.StreamingContext]] associated with this DStream */ - def context(): StreamingContext = dstream.context() + def context(): StreamingContext = dstream.context /** Return a new DStream by applying a function to all elements of this DStream. */ def map[R](f: JFunction[T, R]): JavaDStream[R] = { - new JavaDStream(dstream.map(f)(f.returnType()))(f.returnType()) + new JavaDStream(dstream.map(f)(fakeClassTag))(fakeClassTag) } /** Return a new DStream by applying a function to all elements of this DStream. */ - def map[K2, V2](f: PairFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { - def cm = implicitly[ClassTag[Tuple2[_, _]]].asInstanceOf[ClassTag[Tuple2[K2, V2]]] - new JavaPairDStream(dstream.map(f)(cm))(f.keyType(), f.valueType()) + def mapToPair[K2, V2](f: PairFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { + def cm: ClassTag[(K2, V2)] = fakeClassTag + new JavaPairDStream(dstream.map[(K2, V2)](f)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } /** @@ -148,19 +149,19 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T */ def flatMap[U](f: FlatMapFunction[T, U]): JavaDStream[U] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.apply(x).asScala - new JavaDStream(dstream.flatMap(fn)(f.elementType()))(f.elementType()) + def fn = (x: T) => f.call(x).asScala + new JavaDStream(dstream.flatMap(fn)(fakeClassTag[U]))(fakeClassTag[U]) } /** * Return a new DStream by applying a function to all elements of this DStream, * and then flattening the results */ - def flatMap[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { + def flatMapToPair[K2, V2](f: PairFlatMapFunction[T, K2, V2]): JavaPairDStream[K2, V2] = { import scala.collection.JavaConverters._ - def fn = (x: T) => f.apply(x).asScala - def cm = implicitly[ClassTag[Tuple2[_, _]]].asInstanceOf[ClassTag[Tuple2[K2, V2]]] - new JavaPairDStream(dstream.flatMap(fn)(cm))(f.keyType(), f.valueType()) + def fn = (x: T) => f.call(x).asScala + def cm: ClassTag[(K2, V2)] = fakeClassTag + new JavaPairDStream(dstream.flatMap(fn)(cm))(fakeClassTag[K2], fakeClassTag[V2]) } /** @@ -169,8 +170,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * of the RDD. */ def mapPartitions[U](f: FlatMapFunction[java.util.Iterator[T], U]): JavaDStream[U] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) - new JavaDStream(dstream.mapPartitions(fn)(f.elementType()))(f.elementType()) + def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + new JavaDStream(dstream.mapPartitions(fn)(fakeClassTag[U]))(fakeClassTag[U]) } /** @@ -178,10 +179,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * of this DStream. Applying mapPartitions() to an RDD applies a function to each partition * of the RDD. */ - def mapPartitions[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]) + def mapPartitionsToPair[K2, V2](f: PairFlatMapFunction[java.util.Iterator[T], K2, V2]) : JavaPairDStream[K2, V2] = { - def fn = (x: Iterator[T]) => asScalaIterator(f.apply(asJavaIterator(x)).iterator()) - new JavaPairDStream(dstream.mapPartitions(fn))(f.keyType(), f.valueType()) + def fn = (x: Iterator[T]) => asScalaIterator(f.call(asJavaIterator(x)).iterator()) + new JavaPairDStream(dstream.mapPartitions(fn))(fakeClassTag[K2], fakeClassTag[V2]) } /** @@ -283,8 +284,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * on each RDD of 'this' DStream. */ def transform[U](transformFunc: JFunction[R, JavaRDD[U]]): JavaDStream[U] = { - implicit val cm: ClassTag[U] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] + implicit val cm: ClassTag[U] = fakeClassTag + def scalaTransform (in: RDD[T]): RDD[U] = transformFunc.call(wrapRDD(in)).rdd dstream.transform(scalaTransform(_)) @@ -295,8 +296,8 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * on each RDD of 'this' DStream. */ def transform[U](transformFunc: JFunction2[R, Time, JavaRDD[U]]): JavaDStream[U] = { - implicit val cm: ClassTag[U] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] + implicit val cm: ClassTag[U] = fakeClassTag + def scalaTransform (in: RDD[T], time: Time): RDD[U] = transformFunc.call(wrapRDD(in), time).rdd dstream.transform(scalaTransform(_, _)) @@ -306,12 +307,11 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Return a new DStream in which each RDD is generated by applying a function * on each RDD of 'this' DStream. */ - def transform[K2, V2](transformFunc: JFunction[R, JavaPairRDD[K2, V2]]): + def transformToPair[K2, V2](transformFunc: JFunction[R, JavaPairRDD[K2, V2]]): JavaPairDStream[K2, V2] = { - implicit val cmk: ClassTag[K2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]] - implicit val cmv: ClassTag[V2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]] + implicit val cmk: ClassTag[K2] = fakeClassTag + implicit val cmv: ClassTag[V2] = fakeClassTag + def scalaTransform (in: RDD[T]): RDD[(K2, V2)] = transformFunc.call(wrapRDD(in)).rdd dstream.transform(scalaTransform(_)) @@ -321,12 +321,11 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Return a new DStream in which each RDD is generated by applying a function * on each RDD of 'this' DStream. */ - def transform[K2, V2](transformFunc: JFunction2[R, Time, JavaPairRDD[K2, V2]]): + def transformToPair[K2, V2](transformFunc: JFunction2[R, Time, JavaPairRDD[K2, V2]]): JavaPairDStream[K2, V2] = { - implicit val cmk: ClassTag[K2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]] - implicit val cmv: ClassTag[V2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]] + implicit val cmk: ClassTag[K2] = fakeClassTag + implicit val cmv: ClassTag[V2] = fakeClassTag + def scalaTransform (in: RDD[T], time: Time): RDD[(K2, V2)] = transformFunc.call(wrapRDD(in), time).rdd dstream.transform(scalaTransform(_, _)) @@ -340,10 +339,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T other: JavaDStream[U], transformFunc: JFunction3[R, JavaRDD[U], Time, JavaRDD[W]] ): JavaDStream[W] = { - implicit val cmu: ClassTag[U] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] - implicit val cmv: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cmu: ClassTag[U] = fakeClassTag + implicit val cmv: ClassTag[W] = fakeClassTag + def scalaTransform (inThis: RDD[T], inThat: RDD[U], time: Time): RDD[W] = transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd dstream.transformWith[U, W](other.dstream, scalaTransform(_, _, _)) @@ -353,16 +351,13 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Return a new DStream in which each RDD is generated by applying a function * on each RDD of 'this' DStream and 'other' DStream. */ - def transformWith[U, K2, V2]( + def transformWithToPair[U, K2, V2]( other: JavaDStream[U], transformFunc: JFunction3[R, JavaRDD[U], Time, JavaPairRDD[K2, V2]] ): JavaPairDStream[K2, V2] = { - implicit val cmu: ClassTag[U] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] - implicit val cmk2: ClassTag[K2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]] - implicit val cmv2: ClassTag[V2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]] + implicit val cmu: ClassTag[U] = fakeClassTag + implicit val cmk2: ClassTag[K2] = fakeClassTag + implicit val cmv2: ClassTag[V2] = fakeClassTag def scalaTransform (inThis: RDD[T], inThat: RDD[U], time: Time): RDD[(K2, V2)] = transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd dstream.transformWith[U, (K2, V2)](other.dstream, scalaTransform(_, _, _)) @@ -376,12 +371,10 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T other: JavaPairDStream[K2, V2], transformFunc: JFunction3[R, JavaPairRDD[K2, V2], Time, JavaRDD[W]] ): JavaDStream[W] = { - implicit val cmk2: ClassTag[K2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]] - implicit val cmv2: ClassTag[V2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]] - implicit val cmw: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cmk2: ClassTag[K2] = fakeClassTag + implicit val cmv2: ClassTag[V2] = fakeClassTag + implicit val cmw: ClassTag[W] = fakeClassTag + def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[W] = transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd dstream.transformWith[(K2, V2), W](other.dstream, scalaTransform(_, _, _)) @@ -391,18 +384,14 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Return a new DStream in which each RDD is generated by applying a function * on each RDD of 'this' DStream and 'other' DStream. */ - def transformWith[K2, V2, K3, V3]( + def transformWithToPair[K2, V2, K3, V3]( other: JavaPairDStream[K2, V2], transformFunc: JFunction3[R, JavaPairRDD[K2, V2], Time, JavaPairRDD[K3, V3]] ): JavaPairDStream[K3, V3] = { - implicit val cmk2: ClassTag[K2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K2]] - implicit val cmv2: ClassTag[V2] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V2]] - implicit val cmk3: ClassTag[K3] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K3]] - implicit val cmv3: ClassTag[V3] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V3]] + implicit val cmk2: ClassTag[K2] = fakeClassTag + implicit val cmv2: ClassTag[V2] = fakeClassTag + implicit val cmk3: ClassTag[K3] = fakeClassTag + implicit val cmv3: ClassTag[V3] = fakeClassTag def scalaTransform (inThis: RDD[T], inThat: RDD[(K2, V2)], time: Time): RDD[(K3, V3)] = transformFunc.call(wrapRDD(inThis), other.wrapRDD(inThat), time).rdd dstream.transformWith[(K2, V2), (K3, V3)](other.dstream, scalaTransform(_, _, _)) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 2c7ff87744d7a..ac451d1913aaa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -17,24 +17,25 @@ package org.apache.spark.streaming.api.java -import java.util.{List => JList} import java.lang.{Long => JLong} +import java.util.{List => JList} import scala.collection.JavaConversions._ import scala.reflect.ClassTag -import org.apache.spark.streaming._ -import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, Function3 => JFunction3} -import org.apache.spark.Partitioner +import com.google.common.base.Optional +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.{JobConf, OutputFormat} import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} -import org.apache.hadoop.conf.Configuration -import org.apache.spark.api.java.{JavaUtils, JavaRDD, JavaPairRDD} -import org.apache.spark.storage.StorageLevel -import com.google.common.base.Optional +import org.apache.spark.Partitioner +import org.apache.spark.api.java.{JavaPairRDD, JavaUtils} +import org.apache.spark.api.java.JavaPairRDD._ +import org.apache.spark.api.java.JavaSparkContext.fakeClassTag +import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2} import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.PairRDDFunctions +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming._ +import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.DStream /** @@ -54,7 +55,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( /** Return a new DStream containing only the elements that satisfy a predicate. */ def filter(f: JFunction[(K, V), java.lang.Boolean]): JavaPairDStream[K, V] = - dstream.filter((x => f(x).booleanValue())) + dstream.filter((x => f.call(x).booleanValue())) /** Persist RDDs of this DStream with the default storage level (MEMORY_ONLY_SER) */ def cache(): JavaPairDStream[K, V] = dstream.cache() @@ -168,8 +169,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( mergeCombiners: JFunction2[C, C, C], partitioner: Partitioner ): JavaPairDStream[K, C] = { - implicit val cm: ClassTag[C] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] + implicit val cm: ClassTag[C] = fakeClassTag dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner) } @@ -184,8 +184,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( partitioner: Partitioner, mapSideCombine: Boolean ): JavaPairDStream[K, C] = { - implicit val cm: ClassTag[C] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[C]] + implicit val cm: ClassTag[C] = fakeClassTag dstream.combineByKey(createCombiner, mergeValue, mergeCombiners, partitioner, mapSideCombine) } @@ -279,7 +278,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * DStream's batching interval */ def reduceByKeyAndWindow( - reduceFunc: Function2[V, V, V], + reduceFunc: JFunction2[V, V, V], windowDuration: Duration, slideDuration: Duration ):JavaPairDStream[K, V] = { @@ -299,7 +298,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param numPartitions Number of partitions of each RDD in the new DStream. */ def reduceByKeyAndWindow( - reduceFunc: Function2[V, V, V], + reduceFunc: JFunction2[V, V, V], windowDuration: Duration, slideDuration: Duration, numPartitions: Int @@ -320,7 +319,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * DStream. */ def reduceByKeyAndWindow( - reduceFunc: Function2[V, V, V], + reduceFunc: JFunction2[V, V, V], windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner @@ -345,8 +344,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * DStream's batching interval */ def reduceByKeyAndWindow( - reduceFunc: Function2[V, V, V], - invReduceFunc: Function2[V, V, V], + reduceFunc: JFunction2[V, V, V], + invReduceFunc: JFunction2[V, V, V], windowDuration: Duration, slideDuration: Duration ): JavaPairDStream[K, V] = { @@ -374,8 +373,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * set this to null if you do not want to filter */ def reduceByKeyAndWindow( - reduceFunc: Function2[V, V, V], - invReduceFunc: Function2[V, V, V], + reduceFunc: JFunction2[V, V, V], + invReduceFunc: JFunction2[V, V, V], windowDuration: Duration, slideDuration: Duration, numPartitions: Int, @@ -412,8 +411,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * set this to null if you do not want to filter */ def reduceByKeyAndWindow( - reduceFunc: Function2[V, V, V], - invReduceFunc: Function2[V, V, V], + reduceFunc: JFunction2[V, V, V], + invReduceFunc: JFunction2[V, V, V], windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner, @@ -453,8 +452,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( */ def updateStateByKey[S](updateFunc: JFunction2[JList[V], Optional[S], Optional[S]]) : JavaPairDStream[K, S] = { - implicit val cm: ClassTag[S] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[S]] + implicit val cm: ClassTag[S] = fakeClassTag dstream.updateStateByKey(convertUpdateStateFunction(updateFunc)) } @@ -471,8 +469,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( updateFunc: JFunction2[JList[V], Optional[S], Optional[S]], numPartitions: Int) : JavaPairDStream[K, S] = { - implicit val cm: ClassTag[S] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[S]] + implicit val cm: ClassTag[S] = fakeClassTag dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), numPartitions) } @@ -490,8 +487,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( updateFunc: JFunction2[JList[V], Optional[S], Optional[S]], partitioner: Partitioner ): JavaPairDStream[K, S] = { - implicit val cm: ClassTag[S] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[S]] + implicit val cm: ClassTag[S] = fakeClassTag dstream.updateStateByKey(convertUpdateStateFunction(updateFunc), partitioner) } @@ -501,8 +497,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * 'this' DStream without changing the key. */ def mapValues[U](f: JFunction[V, U]): JavaPairDStream[K, U] = { - implicit val cm: ClassTag[U] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[U]] + implicit val cm: ClassTag[U] = fakeClassTag dstream.mapValues(f) } @@ -524,8 +519,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * of partitions. */ def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) } @@ -537,8 +531,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( other: JavaPairDStream[K, W], numPartitions: Int ): JavaPairDStream[K, (JList[V], JList[W])] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, numPartitions) .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) } @@ -551,8 +544,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( other: JavaPairDStream[K, W], partitioner: Partitioner ): JavaPairDStream[K, (JList[V], JList[W])] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, partitioner) .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) } @@ -562,8 +554,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Hash partitioning is used to generate the RDDs with Spark's default number of partitions. */ def join[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, W)] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag dstream.join(other.dstream) } @@ -572,8 +563,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ def join[W](other: JavaPairDStream[K, W], numPartitions: Int): JavaPairDStream[K, (V, W)] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag dstream.join(other.dstream, numPartitions) } @@ -585,8 +575,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( other: JavaPairDStream[K, W], partitioner: Partitioner ): JavaPairDStream[K, (V, W)] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag dstream.join(other.dstream, partitioner) } @@ -596,8 +585,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * number of partitions. */ def leftOuterJoin[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (V, Optional[W])] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag val joinResult = dstream.leftOuterJoin(other.dstream) joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))} } @@ -611,8 +599,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( other: JavaPairDStream[K, W], numPartitions: Int ): JavaPairDStream[K, (V, Optional[W])] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag val joinResult = dstream.leftOuterJoin(other.dstream, numPartitions) joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))} } @@ -625,8 +612,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( other: JavaPairDStream[K, W], partitioner: Partitioner ): JavaPairDStream[K, (V, Optional[W])] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag val joinResult = dstream.leftOuterJoin(other.dstream, partitioner) joinResult.mapValues{case (v, w) => (v, JavaUtils.optionToOptional(w))} } @@ -652,8 +638,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( other: JavaPairDStream[K, W], numPartitions: Int ): JavaPairDStream[K, (Optional[V], W)] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag val joinResult = dstream.rightOuterJoin(other.dstream, numPartitions) joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)} } @@ -667,8 +652,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( other: JavaPairDStream[K, W], partitioner: Partitioner ): JavaPairDStream[K, (Optional[V], W)] = { - implicit val cm: ClassTag[W] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[W]] + implicit val cm: ClassTag[W] = fakeClassTag val joinResult = dstream.rightOuterJoin(other.dstream, partitioner) joinResult.mapValues{case (v, w) => (JavaUtils.optionToOptional(v), w)} } @@ -748,8 +732,7 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( new JavaDStream[(K, V)](dstream) } - override val classTag: ClassTag[(K, V)] = - implicitly[ClassTag[Tuple2[_, _]]].asInstanceOf[ClassTag[Tuple2[K, V]]] + override val classTag: ClassTag[(K, V)] = fakeClassTag } object JavaPairDStream { @@ -758,10 +741,8 @@ object JavaPairDStream { } def fromJavaDStream[K, V](dstream: JavaDStream[(K, V)]): JavaPairDStream[K, V] = { - implicit val cmk: ClassTag[K] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] - implicit val cmv: ClassTag[V] = - implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[V]] + implicit val cmk: ClassTag[K] = fakeClassTag + implicit val cmv: ClassTag[V] = fakeClassTag new JavaPairDStream[K, V](dstream.dstream) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index b082bb058529b..c48d754e439e9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -187,7 +187,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { converter: JFunction[InputStream, java.lang.Iterable[T]], storageLevel: StorageLevel) : JavaDStream[T] = { - def fn = (x: InputStream) => converter.apply(x).toIterator + def fn = (x: InputStream) => converter.call(x).toIterator implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.socketStream(hostname, port, fn, storageLevel) @@ -431,7 +431,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * In the transform function, convert the JavaRDD corresponding to that JavaDStream to * a JavaPairRDD using org.apache.spark.api.java.JavaPairRDD.fromJavaRDD(). */ - def transform[K, V]( + def transformToPair[K, V]( dstreams: JList[JavaDStream[_]], transformFunc: JFunction2[JList[JavaRDD[_]], Time, JavaPairRDD[K, V]] ): JavaPairDStream[K, V] = { diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 54a0791d04ea4..e93bf18b6d0b9 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -247,14 +247,14 @@ public Iterable call(Iterator in) { Assert.assertEquals(expected, result); } - private class IntegerSum extends Function2 { + private class IntegerSum implements Function2 { @Override public Integer call(Integer i1, Integer i2) throws Exception { return i1 + i2; } } - private class IntegerDifference extends Function2 { + private class IntegerDifference implements Function2 { @Override public Integer call(Integer i1, Integer i2) throws Exception { return i1 - i2; @@ -392,7 +392,7 @@ public JavaRDD call(JavaRDD in) throws Exception { } ); - JavaPairDStream transformed3 = stream.transform( + JavaPairDStream transformed3 = stream.transformToPair( new Function, JavaPairRDD>() { @Override public JavaPairRDD call(JavaRDD in) throws Exception { return null; @@ -400,7 +400,7 @@ public JavaRDD call(JavaRDD in) throws Exception { } ); - JavaPairDStream transformed4 = stream.transform( + JavaPairDStream transformed4 = stream.transformToPair( new Function2, Time, JavaPairRDD>() { @Override public JavaPairRDD call(JavaRDD in, Time time) throws Exception { return null; @@ -424,7 +424,7 @@ public JavaRDD call(JavaRDD in) throws Exception { } ); - JavaPairDStream pairTransformed3 = pairStream.transform( + JavaPairDStream pairTransformed3 = pairStream.transformToPair( new Function, JavaPairRDD>() { @Override public JavaPairRDD call(JavaPairRDD in) throws Exception { return null; @@ -432,7 +432,7 @@ public JavaRDD call(JavaRDD in) throws Exception { } ); - JavaPairDStream pairTransformed4 = pairStream.transform( + JavaPairDStream pairTransformed4 = pairStream.transformToPair( new Function2, Time, JavaPairRDD>() { @Override public JavaPairRDD call(JavaPairRDD in, Time time) throws Exception { return null; @@ -482,7 +482,7 @@ public void testTransformWith() { ssc, stringStringKVStream2, 1); JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - JavaPairDStream> joined = pairStream1.transformWith( + JavaPairDStream> joined = pairStream1.transformWithToPair( pairStream2, new Function3< JavaPairRDD, @@ -551,7 +551,7 @@ public JavaRDD call(JavaRDD rdd1, JavaPairRDD } ); - JavaPairDStream transformed3 = stream1.transformWith( + JavaPairDStream transformed3 = stream1.transformWithToPair( stream2, new Function3, JavaRDD, Time, JavaPairRDD>() { @Override @@ -561,7 +561,7 @@ public JavaPairRDD call(JavaRDD rdd1, JavaRDD r } ); - JavaPairDStream transformed4 = stream1.transformWith( + JavaPairDStream transformed4 = stream1.transformWithToPair( pairStream1, new Function3, JavaPairRDD, Time, JavaPairRDD>() { @Override @@ -591,7 +591,7 @@ public JavaRDD call(JavaPairRDD rdd1, JavaPairRDD pairTransformed3 = pairStream1.transformWith( + JavaPairDStream pairTransformed3 = pairStream1.transformWithToPair( stream2, new Function3, JavaRDD, Time, JavaPairRDD>() { @Override @@ -601,7 +601,7 @@ public JavaPairRDD call(JavaPairRDD rdd1, JavaR } ); - JavaPairDStream pairTransformed4 = pairStream1.transformWith( + JavaPairDStream pairTransformed4 = pairStream1.transformWithToPair( pairStream2, new Function3, JavaPairRDD, Time, JavaPairRDD>() { @Override @@ -656,7 +656,7 @@ public JavaRDD call(List> listOfRDDs, Time time) { List> listOfDStreams2 = Arrays.>asList(stream1, stream2, pairStream1.toJavaDStream()); - JavaPairDStream> transformed2 = ssc.transform( + JavaPairDStream> transformed2 = ssc.transformToPair( listOfDStreams2, new Function2>, Time, JavaPairRDD>>() { public JavaPairRDD> call(List> listOfRDDs, Time time) { @@ -671,7 +671,7 @@ public Tuple2 call(Integer i) throws Exception { return new Tuple2(i, i); } }; - return rdd1.union(rdd2).map(mapToTuple).join(prdd3); + return rdd1.union(rdd2).mapToPair(mapToTuple).join(prdd3); } } ); @@ -742,17 +742,17 @@ public void testPairFlatMap() { new Tuple2(9, "s"))); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream flatMapped = stream.flatMap( - new PairFlatMapFunction() { - @Override - public Iterable> call(String in) throws Exception { - List> out = Lists.newArrayList(); - for (String letter: in.split("(?!^)")) { - out.add(new Tuple2(in.length(), letter)); - } - return out; + JavaPairDStream flatMapped = stream.flatMapToPair( + new PairFlatMapFunction() { + @Override + public Iterable> call(String in) throws Exception { + List> out = Lists.newArrayList(); + for (String letter: in.split("(?!^)")) { + out.add(new Tuple2(in.length(), letter)); } - }); + return out; + } + }); JavaTestUtils.attachTestOutputStream(flatMapped); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -816,7 +816,7 @@ public void testPairFilter() { Arrays.asList(new Tuple2("yankees", 7))); JavaDStream stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); - JavaPairDStream pairStream = stream.map( + JavaPairDStream pairStream = stream.mapToPair( new PairFunction() { @Override public Tuple2 call(String in) throws Exception { @@ -880,7 +880,7 @@ public void testPairMap() { // Maps pair -> pair of different type JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream reversed = pairStream.map( + JavaPairDStream reversed = pairStream.mapToPair( new PairFunction, Integer, String>() { @Override public Tuple2 call(Tuple2 in) throws Exception { @@ -913,7 +913,7 @@ public void testPairMapPartitions() { // Maps pair -> pair of different type JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream reversed = pairStream.mapPartitions( + JavaPairDStream reversed = pairStream.mapPartitionsToPair( new PairFlatMapFunction>, Integer, String>() { @Override public Iterable> call(Iterator> in) throws Exception { @@ -983,7 +983,7 @@ public void testPairToPairFlatMapWithChangingTypes() { // Maps pair -> pair JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream flatMapped = pairStream.flatMap( + JavaPairDStream flatMapped = pairStream.flatMapToPair( new PairFlatMapFunction, Integer, String>() { @Override public Iterable> call(Tuple2 in) throws Exception { @@ -1228,7 +1228,8 @@ public void testReduceByKeyAndWindowWithInverse() { JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); JavaPairDStream reduceWindowed = - pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(), new Duration(2000), new Duration(1000)); + pairStream.reduceByKeyAndWindow(new IntegerSum(), new IntegerDifference(), + new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(reduceWindowed); List>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1300,7 +1301,7 @@ public void testPairTransform() { ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream sorted = pairStream.transform( + JavaPairDStream sorted = pairStream.transformToPair( new Function, JavaPairRDD>() { @Override public JavaPairRDD call(JavaPairRDD in) throws Exception { @@ -1632,7 +1633,8 @@ public void testSocketTextStream() { @Test public void testSocketString() { - class Converter extends Function> { + + class Converter implements Function> { public Iterable call(InputStream in) throws IOException { BufferedReader reader = new BufferedReader(new InputStreamReader(in)); List out = new ArrayList(); From 2d8e0a062cef878bcc878e4825a5771d35f0509f Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 4 Mar 2014 10:27:02 -0800 Subject: [PATCH 046/397] SPARK-1164 Deprecated reduceByKeyToDriver as it is an alias for reduceByKeyLocally Author: Prashant Sharma Closes #72 from ScrapCodes/SPARK-1164/deprecate-reducebykeytodriver and squashes the following commits: ee521cd [Prashant Sharma] SPARK-1164 Deprecated reduceByKeyToDriver as it is an alias for reduceByKeyLocally --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 5aa0b030dbdd8..2384c8f2b6fd4 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -193,6 +193,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) } /** Alias for reduceByKeyLocally */ + @deprecated("Use reduceByKeyLocally", "1.0.0") def reduceByKeyToDriver(func: (V, V) => V): Map[K, V] = reduceByKeyLocally(func) /** Count the number of elements for each key, and return the result to the master as a Map. */ From 1865dd681bcb38f8f197b559d1bae3a3771a74e0 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 4 Mar 2014 10:28:17 -0800 Subject: [PATCH 047/397] SPARK-1178: missing document of spark.scheduler.revive.interval https://spark-project.atlassian.net/browse/SPARK-1178 The configuration on spark.scheduler.revive.interval is undocumented but actually used https://github.com/apache/spark/blob/master/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala#L64 Author: CodingCat Closes #74 from CodingCat/SPARK-1178 and squashes the following commits: 783ec69 [CodingCat] missing document of spark.scheduler.revive.interval --- docs/configuration.md | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/docs/configuration.md b/docs/configuration.md index 8e4c48c81f8be..dc5553f3da770 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -201,6 +201,13 @@ Apart from these, the following properties are also available, and may be useful multi-user services. + + + + + From 02836657cfec50bc6cc357541e40f8d36c90b352 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Tue, 4 Mar 2014 15:32:43 -0800 Subject: [PATCH 048/397] SPARK-1109 wrong API docs for pyspark map function Author: Prashant Sharma Closes #73 from ScrapCodes/SPARK-1109/wrong-API-docs and squashes the following commits: 1a55b58 [Prashant Sharma] SPARK-1109 wrong API docs for pyspark map function --- python/pyspark/rdd.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 1330e6146800c..be23f87f5ed2d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -163,7 +163,7 @@ def getCheckpointFile(self): def map(self, f, preservesPartitioning=False): """ - Return a new RDD containing the distinct elements in this RDD. + Return a new RDD by applying a function to each element of this RDD. """ def func(split, iterator): return imap(f, iterator) return PipelinedRDD(self, func, preservesPartitioning) From a3da5088195eea7d90b37feee5dd2a372fcd9ace Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 5 Mar 2014 14:00:28 -0800 Subject: [PATCH 049/397] SPARK-1171: when executor is removed, we should minus totalCores instead of just freeCores on that executor https://spark-project.atlassian.net/browse/SPARK-1171 When the executor is removed, the current implementation will only minus the freeCores of that executor. Actually we should minus the totalCores... Author: CodingCat Author: Nan Zhu Closes #63 from CodingCat/simplify_CoarseGrainedSchedulerBackend and squashes the following commits: f6bf93f [Nan Zhu] code clean 19c2bb4 [CodingCat] use copy idiom to reconstruct the workerOffers 43c13e9 [CodingCat] keep WorkerOffer immutable af470d3 [CodingCat] style fix 0c0e409 [CodingCat] simplify the implementation of CoarseGrainedSchedulerBackend --- .../scala/org/apache/spark/scheduler/WorkerOffer.scala | 2 +- .../scheduler/cluster/CoarseGrainedSchedulerBackend.scala | 8 ++++++-- 2 files changed, 7 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala index ba6bab3f91a65..810b36cddf835 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/WorkerOffer.scala @@ -21,4 +21,4 @@ package org.apache.spark.scheduler * Represents free resources available on an executor. */ private[spark] -class WorkerOffer(val executorId: String, val host: String, val cores: Int) +case class WorkerOffer(executorId: String, host: String, cores: Int) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 379e02eb9a437..fad03731572e7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -54,6 +54,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A private val executorAddress = new HashMap[String, Address] private val executorHost = new HashMap[String, String] private val freeCores = new HashMap[String, Int] + private val totalCores = new HashMap[String, Int] private val addressToExecutorId = new HashMap[Address, String] override def preStart() { @@ -76,6 +77,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A sender ! RegisteredExecutor(sparkProperties) executorActor(executorId) = sender executorHost(executorId) = Utils.parseHostPort(hostPort)._1 + totalCores(executorId) = cores freeCores(executorId) = cores executorAddress(executorId) = sender.path.address addressToExecutorId(sender.path.address) = executorId @@ -147,10 +149,12 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A def removeExecutor(executorId: String, reason: String) { if (executorActor.contains(executorId)) { logInfo("Executor " + executorId + " disconnected, so removing it") - val numCores = freeCores(executorId) - addressToExecutorId -= executorAddress(executorId) + val numCores = totalCores(executorId) executorActor -= executorId executorHost -= executorId + addressToExecutorId -= executorAddress(executorId) + executorAddress -= executorId + totalCores -= executorId freeCores -= executorId totalCoreCount.addAndGet(-numCores) scheduler.executorLost(executorId, SlaveLost(reason)) From 51ca7bd7038dd5f66327d5b15692a1ccaab42129 Mon Sep 17 00:00:00 2001 From: liguoqiang Date: Wed, 5 Mar 2014 16:38:43 -0800 Subject: [PATCH 050/397] Improve building with maven docs mvn -Dhadoop.version=... -Dsuites=spark.repl.ReplSuite test to mvn -Dhadoop.version=... -Dsuites=org.apache.spark.repl.ReplSuite test Author: liguoqiang Closes #70 from witgo/building_with_maven and squashes the following commits: 6ec8a54 [liguoqiang] spark.repl.ReplSuite to org.apache.spark.repl.ReplSuite --- docs/building-with-maven.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index a982c4dbac7d4..d3bc34e68b240 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -56,7 +56,7 @@ Tests are run by default via the [ScalaTest Maven plugin](http://www.scalatest.o The ScalaTest plugin also supports running only a specific test suite as follows: - $ mvn -Dhadoop.version=... -Dsuites=spark.repl.ReplSuite test + $ mvn -Dhadoop.version=... -Dsuites=org.apache.spark.repl.ReplSuite test ## Continuous Compilation ## From cda381f88cc03340fdf7b2d681699babbae2a56e Mon Sep 17 00:00:00 2001 From: Mark Grover Date: Wed, 5 Mar 2014 16:52:58 -0800 Subject: [PATCH 051/397] SPARK-1184: Update the distribution tar.gz to include spark-assembly jar See JIRA for details. Author: Mark Grover Closes #78 from markgrover/SPARK-1184 and squashes the following commits: 12b78e6 [Mark Grover] SPARK-1184: Update the distribution tar.gz to include spark-assembly jar --- assembly/pom.xml | 5 ++++- assembly/src/main/assembly/assembly.xml | 11 +++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 7ce30179e9ca2..82a5985504b4e 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -29,9 +29,12 @@ spark-assembly_2.10 Spark Project Assembly http://spark.apache.org/ + pom - ${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + scala-${scala.binary.version} + ${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} spark /usr/share/spark root diff --git a/assembly/src/main/assembly/assembly.xml b/assembly/src/main/assembly/assembly.xml index 6af383db65d47..711156337b7c3 100644 --- a/assembly/src/main/assembly/assembly.xml +++ b/assembly/src/main/assembly/assembly.xml @@ -55,6 +55,15 @@ **/* + + + ${project.parent.basedir}/assembly/target/${spark.jar.dir} + + / + + ${spark.jar.basename} + + @@ -75,6 +84,8 @@ org.apache.hadoop:*:jar org.apache.spark:*:jar + org.apache.zookeeper:*:jar + org.apache.avro:*:jar From 3eb009f362993dbe43028419c2d48011111a200d Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 5 Mar 2014 21:47:34 -0800 Subject: [PATCH 052/397] SPARK-1156: allow user to login into a cluster without slaves Reported in https://spark-project.atlassian.net/browse/SPARK-1156 The current spark-ec2 script doesn't allow user to login to a cluster without slaves. One of the issues brought by this behaviour is that when all the worker died, the user cannot even login to the cluster for debugging, etc. Author: CodingCat Closes #58 from CodingCat/SPARK-1156 and squashes the following commits: 104af07 [CodingCat] output ERROR to stderr 9a71769 [CodingCat] do not allow user to start 0-slave cluster 24a7c79 [CodingCat] allow user to login into a cluster without slaves --- ec2/spark_ec2.py | 11 ++++++----- 1 file changed, 6 insertions(+), 5 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index 25e85381896b0..d8840c94ac17c 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -398,15 +398,13 @@ def get_existing_cluster(conn, opts, cluster_name, die_on_error=True): if any((master_nodes, slave_nodes)): print ("Found %d master(s), %d slaves" % (len(master_nodes), len(slave_nodes))) - if (master_nodes != [] and slave_nodes != []) or not die_on_error: + if master_nodes != [] or not die_on_error: return (master_nodes, slave_nodes) else: if master_nodes == [] and slave_nodes != []: - print "ERROR: Could not find master in group " + cluster_name + "-master" - elif master_nodes != [] and slave_nodes == []: - print "ERROR: Could not find slaves in group " + cluster_name + "-slaves" + print >> sys.stderr, "ERROR: Could not find master in group " + cluster_name + "-master" else: - print "ERROR: Could not find any existing cluster" + print >> sys.stderr, "ERROR: Could not find any existing cluster" sys.exit(1) @@ -680,6 +678,9 @@ def real_main(): opts.zone = random.choice(conn.get_all_zones()).name if action == "launch": + if opts.slaves <= 0: + print >> sys.stderr, "ERROR: You have to start at least 1 slave" + sys.exit(1) if opts.resume: (master_nodes, slave_nodes) = get_existing_cluster( conn, opts, cluster_name) From 3d3acef0474b6dc21f1b470ea96079a491e58b75 Mon Sep 17 00:00:00 2001 From: Prabin Banka Date: Thu, 6 Mar 2014 12:45:27 -0800 Subject: [PATCH 053/397] SPARK-1187, Added missing Python APIs The following Python APIs are added, RDD.id() SparkContext.setJobGroup() SparkContext.setLocalProperty() SparkContext.getLocalProperty() SparkContext.sparkUser() was raised earlier as a part of apache/incubator-spark#486 Author: Prabin Banka Closes #75 from prabinb/python-api-backup and squashes the following commits: cc3c6cd [Prabin Banka] Added missing Python APIs --- python/pyspark/context.py | 31 +++++++++++++++++++++++++++++++ python/pyspark/rdd.py | 7 +++++++ 2 files changed, 38 insertions(+) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index 93faa2e3857ed..c9f42d3aacb58 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -372,6 +372,37 @@ def _getJavaStorageLevel(self, storageLevel): return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory, storageLevel.deserialized, storageLevel.replication) + def setJobGroup(self, groupId, description): + """ + Assigns a group ID to all the jobs started by this thread until the group ID is set to a + different value or cleared. + + Often, a unit of execution in an application consists of multiple Spark actions or jobs. + Application programmers can use this method to group all those jobs together and give a + group description. Once set, the Spark web UI will associate such jobs with this group. + """ + self._jsc.setJobGroup(groupId, description) + + def setLocalProperty(self, key, value): + """ + Set a local property that affects jobs submitted from this thread, such as the + Spark fair scheduler pool. + """ + self._jsc.setLocalProperty(key, value) + + def getLocalProperty(self, key): + """ + Get a local property set in this thread, or null if it is missing. See + L{setLocalProperty} + """ + return self._jsc.getLocalProperty(key) + + def sparkUser(self): + """ + Get SPARK_USER for user who is running SparkContext. + """ + return self._jsc.sc().sparkUser() + def _test(): import atexit import doctest diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index be23f87f5ed2d..097a0a236b145 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -95,6 +95,13 @@ def __init__(self, jrdd, ctx, jrdd_deserializer): self.is_checkpointed = False self.ctx = ctx self._jrdd_deserializer = jrdd_deserializer + self._id = jrdd.id() + + def id(self): + """ + A unique ID for this RDD (within its SparkContext). + """ + return self._id def __repr__(self): return self._jrdd.toString() From 40566e10aae4b21ffc71ea72702b8df118ac5c8e Mon Sep 17 00:00:00 2001 From: Kyle Ellrott Date: Thu, 6 Mar 2014 14:51:00 -0800 Subject: [PATCH 054/397] SPARK-942: Do not materialize partitions when DISK_ONLY storage level is used This is a port of a pull request original targeted at incubator-spark: https://github.com/apache/incubator-spark/pull/180 Essentially if a user returns a generative iterator (from a flatMap operation), when trying to persist the data, Spark would first unroll the iterator into an ArrayBuffer, and then try to figure out if it could store the data. In cases where the user provided an iterator that generated more data then available memory, this would case a crash. With this patch, if the user requests a persist with a 'StorageLevel.DISK_ONLY', the iterator will be unrolled as it is inputed into the serializer. To do this, two changes where made: 1) The type of the 'values' argument in the putValues method of the BlockStore interface was changed from ArrayBuffer to Iterator (and all code interfacing with this method was modified to connect correctly. 2) The JavaSerializer now calls the ObjectOutputStream 'reset' method every 1000 objects. This was done because the ObjectOutputStream caches objects (thus preventing them from being GC'd) to write more compact serialization. If reset is never called, eventually the memory fills up, if it is called too often then the serialization streams become much larger because of redundant class descriptions. Author: Kyle Ellrott Closes #50 from kellrott/iterator-to-disk and squashes the following commits: 9ef7cb8 [Kyle Ellrott] Fixing formatting issues. 60e0c57 [Kyle Ellrott] Fixing issues (formatting, variable names, etc.) from review comments 8aa31cd [Kyle Ellrott] Merge ../incubator-spark into iterator-to-disk 33ac390 [Kyle Ellrott] Merge branch 'iterator-to-disk' of github.com:kellrott/incubator-spark into iterator-to-disk 2f684ea [Kyle Ellrott] Refactoring the BlockManager to replace the Either[Either[A,B]] usage. Now using trait 'Values'. Also modified BlockStore.putBytes call to return PutResult, so that it behaves like putValues. f70d069 [Kyle Ellrott] Adding docs for spark.serializer.objectStreamReset configuration 7ccc74b [Kyle Ellrott] Moving the 'LargeIteratorSuite' to simply test persistance of iterators. It doesn't try to invoke a OOM error any more 16a4cea [Kyle Ellrott] Streamlined the LargeIteratorSuite unit test. It should now run in ~25 seconds. Confirmed that it still crashes an unpatched copy of Spark. c2fb430 [Kyle Ellrott] Removing more un-needed array-buffer to iterator conversions 627a8b7 [Kyle Ellrott] Wrapping a few long lines 0f28ec7 [Kyle Ellrott] Adding second putValues to BlockStore interface that accepts an ArrayBuffer (rather then an Iterator). This will allow BlockStores to have slightly different behaviors dependent on whether they get an Iterator or ArrayBuffer. In the case of the MemoryStore, it needs to duplicate and cache an Iterator into an ArrayBuffer, but if handed a ArrayBuffer, it can skip the duplication. 656c33e [Kyle Ellrott] Fixing the JavaSerializer to read from the SparkConf rather then the System property. 8644ee8 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk 00c98e0 [Kyle Ellrott] Making the Java ObjectStreamSerializer reset rate configurable by the system variable 'spark.serializer.objectStreamReset', default is not 10000. 40fe1d7 [Kyle Ellrott] Removing rouge space 31fe08e [Kyle Ellrott] Removing un-needed semi-colons 9df0276 [Kyle Ellrott] Added check to make sure that streamed-to-dist RDD actually returns good data in the LargeIteratorSuite a6424ba [Kyle Ellrott] Wrapping long line 2eeda75 [Kyle Ellrott] Fixing dumb mistake ("||" instead of "&&") 0e6f808 [Kyle Ellrott] Deleting temp output directory when done 95c7f67 [Kyle Ellrott] Simplifying StorageLevel checks 56f71cd [Kyle Ellrott] Merge branch 'master' into iterator-to-disk 44ec35a [Kyle Ellrott] Adding some comments. 5eb2b7e [Kyle Ellrott] Changing the JavaSerializer reset to occur every 1000 objects. f403826 [Kyle Ellrott] Merge branch 'master' into iterator-to-disk 81d670c [Kyle Ellrott] Adding unit test for straight to disk iterator methods. d32992f [Kyle Ellrott] Merge remote-tracking branch 'origin/master' into iterator-to-disk cac1fad [Kyle Ellrott] Fixing MemoryStore, so that it converts incoming iterators to ArrayBuffer objects. This was previously done higher up the stack. efe1102 [Kyle Ellrott] Changing CacheManager and BlockManager to pass iterators directly to the serializer when a 'DISK_ONLY' persist is called. This is in response to SPARK-942. --- .../scala/org/apache/spark/CacheManager.scala | 28 +++++- .../spark/serializer/JavaSerializer.scala | 29 +++++-- .../apache/spark/storage/BlockManager.scala | 87 +++++++++++-------- .../org/apache/spark/storage/BlockStore.scala | 5 +- .../org/apache/spark/storage/DiskStore.scala | 14 ++- .../apache/spark/storage/MemoryStore.scala | 31 +++++-- .../spark/storage/FlatmapIteratorSuite.scala | 74 ++++++++++++++++ docs/configuration.md | 11 +++ 8 files changed, 226 insertions(+), 53 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 1daabecf23292..872e892c04fe6 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -71,10 +71,30 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val computedValues = rdd.computeOrReadCheckpoint(split, context) // Persist the result, so long as the task is not running locally if (context.runningLocally) { return computedValues } - val elements = new ArrayBuffer[Any] - elements ++= computedValues - blockManager.put(key, elements, storageLevel, tellMaster = true) - elements.iterator.asInstanceOf[Iterator[T]] + if (storageLevel.useDisk && !storageLevel.useMemory) { + // In the case that this RDD is to be persisted using DISK_ONLY + // the iterator will be passed directly to the blockManager (rather then + // caching it to an ArrayBuffer first), then the resulting block data iterator + // will be passed back to the user. If the iterator generates a lot of data, + // this means that it doesn't all have to be held in memory at one time. + // This could also apply to MEMORY_ONLY_SER storage, but we need to make sure + // blocks aren't dropped by the block store before enabling that. + blockManager.put(key, computedValues, storageLevel, tellMaster = true) + return blockManager.get(key) match { + case Some(values) => + return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) + case None => + logInfo("Failure to store %s".format(key)) + throw new Exception("Block manager failed to return persisted valued") + } + } else { + // In this case the RDD is cached to an array buffer. This will save the results + // if we're dealing with a 'one-time' iterator + val elements = new ArrayBuffer[Any] + elements ++= computedValues + blockManager.put(key, elements, storageLevel, tellMaster = true) + return elements.iterator.asInstanceOf[Iterator[T]] + } } finally { loading.synchronized { loading.remove(key) diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 33c1705ad7c58..bfa647f7f0516 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -23,9 +23,28 @@ import java.nio.ByteBuffer import org.apache.spark.SparkConf import org.apache.spark.util.ByteBufferInputStream -private[spark] class JavaSerializationStream(out: OutputStream) extends SerializationStream { +private[spark] class JavaSerializationStream(out: OutputStream, conf: SparkConf) + extends SerializationStream { val objOut = new ObjectOutputStream(out) - def writeObject[T](t: T): SerializationStream = { objOut.writeObject(t); this } + var counter = 0 + val counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) + + /** + * Calling reset to avoid memory leak: + * http://stackoverflow.com/questions/1281549/memory-leak-traps-in-the-java-standard-api + * But only call it every 10,000th time to avoid bloated serialization streams (when + * the stream 'resets' object class descriptions have to be re-written) + */ + def writeObject[T](t: T): SerializationStream = { + objOut.writeObject(t) + if (counterReset > 0 && counter >= counterReset) { + objOut.reset() + counter = 0 + } else { + counter += 1 + } + this + } def flush() { objOut.flush() } def close() { objOut.close() } } @@ -41,7 +60,7 @@ extends DeserializationStream { def close() { objIn.close() } } -private[spark] class JavaSerializerInstance extends SerializerInstance { +private[spark] class JavaSerializerInstance(conf: SparkConf) extends SerializerInstance { def serialize[T](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) @@ -63,7 +82,7 @@ private[spark] class JavaSerializerInstance extends SerializerInstance { } def serializeStream(s: OutputStream): SerializationStream = { - new JavaSerializationStream(s) + new JavaSerializationStream(s, conf) } def deserializeStream(s: InputStream): DeserializationStream = { @@ -79,5 +98,5 @@ private[spark] class JavaSerializerInstance extends SerializerInstance { * A Spark serializer that uses Java's built-in serialization. */ class JavaSerializer(conf: SparkConf) extends Serializer { - def newInstance(): SerializerInstance = new JavaSerializerInstance + def newInstance(): SerializerInstance = new JavaSerializerInstance(conf) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a734ddc1ef702..977c24687cc5f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -35,6 +35,12 @@ import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ +sealed trait Values + +case class ByteBufferValues(buffer: ByteBuffer) extends Values +case class IteratorValues(iterator: Iterator[Any]) extends Values +case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values + private[spark] class BlockManager( executorId: String, actorSystem: ActorSystem, @@ -455,9 +461,7 @@ private[spark] class BlockManager( def put(blockId: BlockId, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) : Long = { - val elements = new ArrayBuffer[Any] - elements ++= values - put(blockId, elements, level, tellMaster) + doPut(blockId, IteratorValues(values), level, tellMaster) } /** @@ -479,7 +483,7 @@ private[spark] class BlockManager( def put(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, tellMaster: Boolean = true) : Long = { require(values != null, "Values is null") - doPut(blockId, Left(values), level, tellMaster) + doPut(blockId, ArrayBufferValues(values), level, tellMaster) } /** @@ -488,10 +492,11 @@ private[spark] class BlockManager( def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, tellMaster: Boolean = true) { require(bytes != null, "Bytes is null") - doPut(blockId, Right(bytes), level, tellMaster) + doPut(blockId, ByteBufferValues(bytes), level, tellMaster) } - private def doPut(blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer], + private def doPut(blockId: BlockId, + data: Values, level: StorageLevel, tellMaster: Boolean = true): Long = { require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") @@ -534,8 +539,9 @@ private[spark] class BlockManager( // If we're storing bytes, then initiate the replication before storing them locally. // This is faster as data is already serialized and ready to send. - val replicationFuture = if (data.isRight && level.replication > 1) { - val bufferView = data.right.get.duplicate() // Doesn't copy the bytes, just creates a wrapper + val replicationFuture = if (data.isInstanceOf[ByteBufferValues] && level.replication > 1) { + // Duplicate doesn't copy the bytes, just creates a wrapper + val bufferView = data.asInstanceOf[ByteBufferValues].buffer.duplicate() Future { replicate(blockId, bufferView, level) } @@ -549,34 +555,43 @@ private[spark] class BlockManager( var marked = false try { - data match { - case Left(values) => { - if (level.useMemory) { - // Save it just to memory first, even if it also has useDisk set to true; we will - // drop it to disk later if the memory store can't hold it. - val res = memoryStore.putValues(blockId, values, level, true) - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case Left(newIterator) => valuesAfterPut = newIterator - } - } else { - // Save directly to disk. - // Don't get back the bytes unless we replicate them. - val askForBytes = level.replication > 1 - val res = diskStore.putValues(blockId, values, level, askForBytes) - size = res.size - res.data match { - case Right(newBytes) => bytesAfterPut = newBytes - case _ => - } + if (level.useMemory) { + // Save it just to memory first, even if it also has useDisk set to true; we will + // drop it to disk later if the memory store can't hold it. + val res = data match { + case IteratorValues(iterator) => + memoryStore.putValues(blockId, iterator, level, true) + case ArrayBufferValues(array) => + memoryStore.putValues(blockId, array, level, true) + case ByteBufferValues(bytes) => { + bytes.rewind(); + memoryStore.putBytes(blockId, bytes, level) + } + } + size = res.size + res.data match { + case Right(newBytes) => bytesAfterPut = newBytes + case Left(newIterator) => valuesAfterPut = newIterator + } + } else { + // Save directly to disk. + // Don't get back the bytes unless we replicate them. + val askForBytes = level.replication > 1 + + val res = data match { + case IteratorValues(iterator) => + diskStore.putValues(blockId, iterator, level, askForBytes) + case ArrayBufferValues(array) => + diskStore.putValues(blockId, array, level, askForBytes) + case ByteBufferValues(bytes) => { + bytes.rewind(); + diskStore.putBytes(blockId, bytes, level) } } - case Right(bytes) => { - bytes.rewind() - // Store it only in memory at first, even if useDisk is also set to true - (if (level.useMemory) memoryStore else diskStore).putBytes(blockId, bytes, level) - size = bytes.limit + size = res.size + res.data match { + case Right(newBytes) => bytesAfterPut = newBytes + case _ => } } @@ -605,8 +620,8 @@ private[spark] class BlockManager( // values and need to serialize and replicate them now: if (level.replication > 1) { data match { - case Right(bytes) => Await.ready(replicationFuture, Duration.Inf) - case Left(values) => { + case ByteBufferValues(bytes) => Await.ready(replicationFuture, Duration.Inf) + case _ => { val remoteStartTime = System.currentTimeMillis // Serialize the block if not already done if (bytesAfterPut == null) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala index b047644b88f48..9a9be047c7245 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockStore.scala @@ -28,7 +28,7 @@ import org.apache.spark.Logging */ private[spark] abstract class BlockStore(val blockManager: BlockManager) extends Logging { - def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel) + def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel) : PutResult /** * Put in a block and, possibly, also return its content as either bytes or another Iterator. @@ -37,6 +37,9 @@ abstract class BlockStore(val blockManager: BlockManager) extends Logging { * @return a PutResult that contains the size of the data, as well as the values put if * returnValues is true (if not, the result's data field can be null) */ + def putValues(blockId: BlockId, values: Iterator[Any], level: StorageLevel, + returnValues: Boolean) : PutResult + def putValues(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, returnValues: Boolean) : PutResult diff --git a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala index d1f07ddb24bb2..36ee4bcc41c66 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskStore.scala @@ -37,7 +37,7 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage diskManager.getBlockLocation(blockId).length } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) { + override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) : PutResult = { // So that we do not modify the input offsets ! // duplicate does not copy buffer, so inexpensive val bytes = _bytes.duplicate() @@ -52,6 +52,7 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage val finishTime = System.currentTimeMillis logDebug("Block %s stored as %s file on disk in %d ms".format( file.getName, Utils.bytesToString(bytes.limit), (finishTime - startTime))) + return PutResult(bytes.limit(), Right(bytes.duplicate())) } override def putValues( @@ -59,13 +60,22 @@ private class DiskStore(blockManager: BlockManager, diskManager: DiskBlockManage values: ArrayBuffer[Any], level: StorageLevel, returnValues: Boolean) + : PutResult = { + return putValues(blockId, values.toIterator, level, returnValues) + } + + override def putValues( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean) : PutResult = { logDebug("Attempting to write values for block " + blockId) val startTime = System.currentTimeMillis val file = diskManager.getFile(blockId) val outputStream = new FileOutputStream(file) - blockManager.dataSerializeStream(blockId, outputStream, values.iterator) + blockManager.dataSerializeStream(blockId, outputStream, values) val length = file.length val timeTaken = System.currentTimeMillis - startTime diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 18141756518c5..b89212eaabf6c 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -49,7 +49,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) { + override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) : PutResult = { // Work on a duplicate - since the original input might be used elsewhere. val bytes = _bytes.duplicate() bytes.rewind() @@ -59,8 +59,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) elements ++= values val sizeEstimate = SizeEstimator.estimate(elements.asInstanceOf[AnyRef]) tryToPut(blockId, elements, sizeEstimate, true) + PutResult(sizeEstimate, Left(values.toIterator)) } else { tryToPut(blockId, bytes, bytes.limit, false) + PutResult(bytes.limit(), Right(bytes.duplicate())) } } @@ -69,14 +71,33 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) values: ArrayBuffer[Any], level: StorageLevel, returnValues: Boolean) - : PutResult = { - + : PutResult = { if (level.deserialized) { val sizeEstimate = SizeEstimator.estimate(values.asInstanceOf[AnyRef]) tryToPut(blockId, values, sizeEstimate, true) - PutResult(sizeEstimate, Left(values.iterator)) + PutResult(sizeEstimate, Left(values.toIterator)) + } else { + val bytes = blockManager.dataSerialize(blockId, values.toIterator) + tryToPut(blockId, bytes, bytes.limit, false) + PutResult(bytes.limit(), Right(bytes.duplicate())) + } + } + + override def putValues( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean) + : PutResult = { + + if (level.deserialized) { + val valueEntries = new ArrayBuffer[Any]() + valueEntries ++= values + val sizeEstimate = SizeEstimator.estimate(valueEntries.asInstanceOf[AnyRef]) + tryToPut(blockId, valueEntries, sizeEstimate, true) + PutResult(sizeEstimate, Left(valueEntries.toIterator)) } else { - val bytes = blockManager.dataSerialize(blockId, values.iterator) + val bytes = blockManager.dataSerialize(blockId, values) tryToPut(blockId, bytes, bytes.limit, false) PutResult(bytes.limit(), Right(bytes.duplicate())) } diff --git a/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala new file mode 100644 index 0000000000000..b843b4c629e84 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala @@ -0,0 +1,74 @@ +/* + * 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.storage + +import org.scalatest.FunSuite +import org.apache.spark.{SharedSparkContext, SparkConf, LocalSparkContext, SparkContext} + + +class FlatmapIteratorSuite extends FunSuite with LocalSparkContext { + /* Tests the ability of Spark to deal with user provided iterators from flatMap + * calls, that may generate more data then available memory. In any + * memory based persistance Spark will unroll the iterator into an ArrayBuffer + * for caching, however in the case that the use defines DISK_ONLY persistance, + * the iterator will be fed directly to the serializer and written to disk. + * + * This also tests the ObjectOutputStream reset rate. When serializing using the + * Java serialization system, the serializer caches objects to prevent writing redundant + * data, however that stops GC of those objects. By calling 'reset' you flush that + * info from the serializer, and allow old objects to be GC'd + */ + test("Flatmap Iterator to Disk") { + val sconf = new SparkConf().setMaster("local-cluster[1,1,512]") + .setAppName("iterator_to_disk_test") + sc = new SparkContext(sconf) + val expand_size = 100 + val data = sc.parallelize((1 to 5).toSeq). + flatMap( x => Stream.range(0, expand_size)) + var persisted = data.persist(StorageLevel.DISK_ONLY) + println(persisted.count()) + assert(persisted.count()===500) + assert(persisted.filter(_==1).count()===5) + } + + test("Flatmap Iterator to Memory") { + val sconf = new SparkConf().setMaster("local-cluster[1,1,512]") + .setAppName("iterator_to_disk_test") + sc = new SparkContext(sconf) + val expand_size = 100 + val data = sc.parallelize((1 to 5).toSeq). + flatMap(x => Stream.range(0, expand_size)) + var persisted = data.persist(StorageLevel.MEMORY_ONLY) + println(persisted.count()) + assert(persisted.count()===500) + assert(persisted.filter(_==1).count()===5) + } + + test("Serializer Reset") { + val sconf = new SparkConf().setMaster("local-cluster[1,1,512]") + .setAppName("serializer_reset_test") + .set("spark.serializer.objectStreamReset", "10") + sc = new SparkContext(sconf) + val expand_size = 500 + val data = sc.parallelize(Seq(1,2)). + flatMap(x => Stream.range(1, expand_size). + map(y => "%d: string test %d".format(y,x))) + var persisted = data.persist(StorageLevel.MEMORY_ONLY_SER) + assert(persisted.filter(_.startsWith("1:")).count()===2) + } + +} diff --git a/docs/configuration.md b/docs/configuration.md index dc5553f3da770..017d509854b42 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -244,6 +244,17 @@ Apart from these, the following properties are also available, and may be useful exceeded" exception inside Kryo. Note that there will be one buffer per core on each worker. + + + + + From 7edbea41b43e0dc11a2de156be220db8b7952d01 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 6 Mar 2014 18:27:50 -0600 Subject: [PATCH 055/397] SPARK-1189: Add Security to Spark - Akka, Http, ConnectionManager, UI use servlets resubmit pull request. was https://github.com/apache/incubator-spark/pull/332. Author: Thomas Graves Closes #33 from tgravescs/security-branch-0.9-with-client-rebase and squashes the following commits: dfe3918 [Thomas Graves] Fix merge conflict since startUserClass now using runAsUser 05eebed [Thomas Graves] Fix dependency lost in upmerge d1040ec [Thomas Graves] Fix up various imports 05ff5e0 [Thomas Graves] Fix up imports after upmerging to master ac046b3 [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase 13733e1 [Thomas Graves] Pass securityManager and SparkConf around where we can. Switch to use sparkConf for reading config whereever possible. Added ConnectionManagerSuite unit tests. 4a57acc [Thomas Graves] Change UI createHandler routines to createServlet since they now return servlets 2f77147 [Thomas Graves] Rework from comments 50dd9f2 [Thomas Graves] fix header in SecurityManager ecbfb65 [Thomas Graves] Fix spacing and formatting b514bec [Thomas Graves] Fix reference to config ed3d1c1 [Thomas Graves] Add security.md 6f7ddf3 [Thomas Graves] Convert SaslClient and SaslServer to scala, change spark.authenticate.ui to spark.ui.acls.enable, and fix up various other things from review comments 2d9e23e [Thomas Graves] Merge remote-tracking branch 'upstream/master' into security-branch-0.9-with-client-rebase_rework 5721c5a [Thomas Graves] update AkkaUtilsSuite test for the actorSelection changes, fix typos based on comments, and remove extra lines I missed in rebase from AkkaUtils f351763 [Thomas Graves] Add Security to Spark - Akka, Http, ConnectionManager, UI to use servlets --- core/pom.xml | 16 ++ .../org/apache/spark/HttpFileServer.scala | 5 +- .../scala/org/apache/spark/HttpServer.scala | 60 +++- .../org/apache/spark/SecurityManager.scala | 253 +++++++++++++++++ .../scala/org/apache/spark/SparkContext.scala | 4 +- .../scala/org/apache/spark/SparkEnv.scala | 24 +- .../org/apache/spark/SparkSaslClient.scala | 146 ++++++++++ .../org/apache/spark/SparkSaslServer.scala | 174 ++++++++++++ .../apache/spark/broadcast/Broadcast.scala | 5 +- .../spark/broadcast/BroadcastFactory.scala | 3 +- .../spark/broadcast/HttpBroadcast.scala | 32 ++- .../spark/broadcast/TorrentBroadcast.scala | 4 +- .../org/apache/spark/deploy/Client.scala | 4 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 10 + .../spark/deploy/client/TestClient.scala | 5 +- .../apache/spark/deploy/master/Master.scala | 17 +- .../spark/deploy/master/ui/MasterWebUI.scala | 25 +- .../spark/deploy/worker/DriverWrapper.scala | 5 +- .../apache/spark/deploy/worker/Worker.scala | 12 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 26 +- .../CoarseGrainedExecutorBackend.scala | 5 +- .../org/apache/spark/executor/Executor.scala | 15 +- .../apache/spark/metrics/MetricsSystem.scala | 13 +- .../spark/metrics/sink/ConsoleSink.scala | 4 +- .../apache/spark/metrics/sink/CsvSink.scala | 4 +- .../spark/metrics/sink/GangliaSink.scala | 4 +- .../spark/metrics/sink/GraphiteSink.scala | 4 +- .../apache/spark/metrics/sink/JmxSink.scala | 5 +- .../spark/metrics/sink/MetricsServlet.scala | 14 +- .../apache/spark/network/BufferMessage.scala | 8 +- .../org/apache/spark/network/Connection.scala | 61 +++- .../apache/spark/network/ConnectionId.scala | 34 +++ .../spark/network/ConnectionManager.scala | 266 +++++++++++++++++- .../org/apache/spark/network/Message.scala | 1 + .../spark/network/MessageChunkHeader.scala | 11 +- .../apache/spark/network/ReceiverTest.scala | 6 +- .../spark/network/SecurityMessage.scala | 163 +++++++++++ .../org/apache/spark/network/SenderTest.scala | 7 +- .../apache/spark/storage/BlockManager.scala | 12 +- .../apache/spark/storage/ThreadingTest.scala | 4 +- .../org/apache/spark/ui/JettyUtils.scala | 138 ++++++--- .../scala/org/apache/spark/ui/SparkUI.scala | 15 +- .../apache/spark/ui/env/EnvironmentUI.scala | 7 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 7 +- .../apache/spark/ui/jobs/JobProgressUI.scala | 15 +- .../spark/ui/storage/BlockManagerUI.scala | 12 +- .../org/apache/spark/util/AkkaUtils.scala | 17 +- .../scala/org/apache/spark/util/Utils.scala | 37 ++- .../org/apache/spark/AkkaUtilsSuite.scala | 215 ++++++++++++++ .../org/apache/spark/BroadcastSuite.scala | 1 + .../apache/spark/ConnectionManagerSuite.scala | 230 +++++++++++++++ .../scala/org/apache/spark/DriverSuite.scala | 1 + .../org/apache/spark/FileServerSuite.scala | 26 ++ .../apache/spark/MapOutputTrackerSuite.scala | 6 +- .../spark/metrics/MetricsSystemSuite.scala | 9 +- .../spark/storage/BlockManagerSuite.scala | 67 +++-- .../scala/org/apache/spark/ui/UISuite.scala | 10 +- docs/configuration.md | 51 ++++ docs/index.md | 1 + docs/security.md | 18 ++ .../streaming/examples/ActorWordCount.scala | 7 +- pom.xml | 20 ++ project/SparkBuild.scala | 4 + .../spark/repl/ExecutorClassLoader.scala | 13 +- .../org/apache/spark/repl/SparkILoop.scala | 22 +- .../org/apache/spark/repl/SparkIMain.scala | 13 +- .../spark/deploy/yarn/ApplicationMaster.scala | 44 +-- .../spark/deploy/yarn/WorkerLauncher.scala | 6 +- .../spark/deploy/yarn/ClientArguments.scala | 2 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 24 +- .../spark/deploy/yarn/ApplicationMaster.scala | 28 +- .../spark/deploy/yarn/WorkerLauncher.scala | 6 +- 72 files changed, 2251 insertions(+), 292 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/SecurityManager.scala create mode 100644 core/src/main/scala/org/apache/spark/SparkSaslClient.scala create mode 100644 core/src/main/scala/org/apache/spark/SparkSaslServer.scala create mode 100644 core/src/main/scala/org/apache/spark/network/ConnectionId.scala create mode 100644 core/src/main/scala/org/apache/spark/network/SecurityMessage.scala create mode 100644 core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala create mode 100644 docs/security.md diff --git a/core/pom.xml b/core/pom.xml index 99c841472b3eb..4c1c2d4da5f77 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -64,6 +64,18 @@ org.apache.curator curator-recipes + + org.eclipse.jetty + jetty-plus + + + org.eclipse.jetty + jetty-security + + + org.eclipse.jetty + jetty-util + org.eclipse.jetty jetty-server @@ -118,6 +130,10 @@ chill-java 0.3.1 + + commons-net + commons-net + ${akka.group} akka-remote_${scala.binary.version} diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index d3264a4bb3c81..3d7692ea8a49e 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -23,7 +23,7 @@ import com.google.common.io.Files import org.apache.spark.util.Utils -private[spark] class HttpFileServer extends Logging { +private[spark] class HttpFileServer(securityManager: SecurityManager) extends Logging { var baseDir : File = null var fileDir : File = null @@ -38,9 +38,10 @@ private[spark] class HttpFileServer extends Logging { fileDir.mkdir() jarDir.mkdir() logInfo("HTTP File server directory is " + baseDir) - httpServer = new HttpServer(baseDir) + httpServer = new HttpServer(baseDir, securityManager) httpServer.start() serverUri = httpServer.uri + logDebug("HTTP file server started at: " + serverUri) } def stop() { diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index 759e68ee0cc61..cb5df25fa48df 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -19,15 +19,18 @@ package org.apache.spark import java.io.File +import org.eclipse.jetty.util.security.{Constraint, Password} +import org.eclipse.jetty.security.authentication.DigestAuthenticator +import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService, SecurityHandler} + import org.eclipse.jetty.server.Server import org.eclipse.jetty.server.bio.SocketConnector -import org.eclipse.jetty.server.handler.DefaultHandler -import org.eclipse.jetty.server.handler.HandlerList -import org.eclipse.jetty.server.handler.ResourceHandler +import org.eclipse.jetty.server.handler.{DefaultHandler, HandlerList, ResourceHandler} import org.eclipse.jetty.util.thread.QueuedThreadPool import org.apache.spark.util.Utils + /** * Exception type thrown by HttpServer when it is in the wrong state for an operation. */ @@ -38,7 +41,8 @@ private[spark] class ServerStateException(message: String) extends Exception(mes * as well as classes created by the interpreter when the user types in code. This is just a wrapper * around a Jetty server. */ -private[spark] class HttpServer(resourceBase: File) extends Logging { +private[spark] class HttpServer(resourceBase: File, securityManager: SecurityManager) + extends Logging { private var server: Server = null private var port: Int = -1 @@ -59,14 +63,60 @@ private[spark] class HttpServer(resourceBase: File) extends Logging { server.setThreadPool(threadPool) val resHandler = new ResourceHandler resHandler.setResourceBase(resourceBase.getAbsolutePath) + val handlerList = new HandlerList handlerList.setHandlers(Array(resHandler, new DefaultHandler)) - server.setHandler(handlerList) + + if (securityManager.isAuthenticationEnabled()) { + logDebug("HttpServer is using security") + val sh = setupSecurityHandler(securityManager) + // make sure we go through security handler to get resources + sh.setHandler(handlerList) + server.setHandler(sh) + } else { + logDebug("HttpServer is not using security") + server.setHandler(handlerList) + } + server.start() port = server.getConnectors()(0).getLocalPort() } } + /** + * Setup Jetty to the HashLoginService using a single user with our + * shared secret. Configure it to use DIGEST-MD5 authentication so that the password + * isn't passed in plaintext. + */ + private def setupSecurityHandler(securityMgr: SecurityManager): ConstraintSecurityHandler = { + val constraint = new Constraint() + // use DIGEST-MD5 as the authentication mechanism + constraint.setName(Constraint.__DIGEST_AUTH) + constraint.setRoles(Array("user")) + constraint.setAuthenticate(true) + constraint.setDataConstraint(Constraint.DC_NONE) + + val cm = new ConstraintMapping() + cm.setConstraint(constraint) + cm.setPathSpec("/*") + val sh = new ConstraintSecurityHandler() + + // the hashLoginService lets us do a single user and + // secret right now. This could be changed to use the + // JAASLoginService for other options. + val hashLogin = new HashLoginService() + + val userCred = new Password(securityMgr.getSecretKey()) + if (userCred == null) { + throw new Exception("Error: secret key is null with authentication on") + } + hashLogin.putUser(securityMgr.getHttpUser(), userCred, Array("user")) + sh.setLoginService(hashLogin) + sh.setAuthenticator(new DigestAuthenticator()); + sh.setConstraintMappings(Array(cm)) + sh + } + def stop() { if (server == null) { throw new ServerStateException("Server is already stopped") diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala new file mode 100644 index 0000000000000..591978c1d3630 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -0,0 +1,253 @@ +/* + * 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 + +import java.net.{Authenticator, PasswordAuthentication} +import org.apache.hadoop.io.Text +import org.apache.hadoop.security.Credentials +import org.apache.hadoop.security.UserGroupInformation +import org.apache.spark.deploy.SparkHadoopUtil + +import scala.collection.mutable.ArrayBuffer + +/** + * Spark class responsible for security. + * + * In general this class should be instantiated by the SparkEnv and most components + * should access it from that. There are some cases where the SparkEnv hasn't been + * initialized yet and this class must be instantiated directly. + * + * Spark currently supports authentication via a shared secret. + * Authentication can be configured to be on via the 'spark.authenticate' configuration + * parameter. This parameter controls whether the Spark communication protocols do + * authentication using the shared secret. This authentication is a basic handshake to + * make sure both sides have the same shared secret and are allowed to communicate. + * If the shared secret is not identical they will not be allowed to communicate. + * + * The Spark UI can also be secured by using javax servlet filters. A user may want to + * secure the UI if it has data that other users should not be allowed to see. The javax + * servlet filter specified by the user can authenticate the user and then once the user + * is logged in, Spark can compare that user versus the view acls to make sure they are + * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' + * control the behavior of the acls. Note that the person who started the application + * always has view access to the UI. + * + * Spark does not currently support encryption after authentication. + * + * At this point spark has multiple communication protocols that need to be secured and + * different underlying mechanisms are used depending on the protocol: + * + * - Akka -> The only option here is to use the Akka Remote secure-cookie functionality. + * Akka remoting allows you to specify a secure cookie that will be exchanged + * and ensured to be identical in the connection handshake between the client + * and the server. If they are not identical then the client will be refused + * to connect to the server. There is no control of the underlying + * authentication mechanism so its not clear if the password is passed in + * plaintext or uses DIGEST-MD5 or some other mechanism. + * Akka also has an option to turn on SSL, this option is not currently supported + * but we could add a configuration option in the future. + * + * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty + * for the HttpServer. Jetty supports multiple authentication mechanisms - + * Basic, Digest, Form, Spengo, etc. It also supports multiple different login + * services - Hash, JAAS, Spnego, JDBC, etc. Spark currently uses the HashLoginService + * to authenticate using DIGEST-MD5 via a single user and the shared secret. + * Since we are using DIGEST-MD5, the shared secret is not passed on the wire + * in plaintext. + * We currently do not support SSL (https), but Jetty can be configured to use it + * so we could add a configuration option for this in the future. + * + * The Spark HttpServer installs the HashLoginServer and configures it to DIGEST-MD5. + * Any clients must specify the user and password. There is a default + * Authenticator installed in the SecurityManager to how it does the authentication + * and in this case gets the user name and password from the request. + * + * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously + * exchange messages. For this we use the Java SASL + * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 + * as the authentication mechanism. This means the shared secret is not passed + * over the wire in plaintext. + * Note that SASL is pluggable as to what mechanism it uses. We currently use + * DIGEST-MD5 but this could be changed to use Kerberos or other in the future. + * Spark currently supports "auth" for the quality of protection, which means + * the connection is not supporting integrity or privacy protection (encryption) + * after authentication. SASL also supports "auth-int" and "auth-conf" which + * SPARK could be support in the future to allow the user to specify the quality + * of protection they want. If we support those, the messages will also have to + * be wrapped and unwrapped via the SaslServer/SaslClient.wrap/unwrap API's. + * + * Since the connectionManager does asynchronous messages passing, the SASL + * authentication is a bit more complex. A ConnectionManager can be both a client + * and a Server, so for a particular connection is has to determine what to do. + * A ConnectionId was added to be able to track connections and is used to + * match up incoming messages with connections waiting for authentication. + * If its acting as a client and trying to send a message to another ConnectionManager, + * it blocks the thread calling sendMessage until the SASL negotiation has occurred. + * The ConnectionManager tracks all the sendingConnections using the ConnectionId + * and waits for the response from the server and does the handshake. + * + * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters + * can be used. Yarn requires a specific AmIpFilter be installed for security to work + * properly. For non-Yarn deployments, users can write a filter to go through a + * companies normal login service. If an authentication filter is in place then the + * SparkUI can be configured to check the logged in user against the list of users who + * have view acls to see if that user is authorized. + * The filters can also be used for many different purposes. For instance filters + * could be used for logging, encryption, or compression. + * + * The exact mechanisms used to generate/distributed the shared secret is deployment specific. + * + * For Yarn deployments, the secret is automatically generated using the Akka remote + * Crypt.generateSecureCookie() API. The secret is placed in the Hadoop UGI which gets passed + * around via the Hadoop RPC mechanism. Hadoop RPC can be configured to support different levels + * of protection. See the Hadoop documentation for more details. Each Spark application on Yarn + * gets a different shared secret. On Yarn, the Spark UI gets configured to use the Hadoop Yarn + * AmIpFilter which requires the user to go through the ResourceManager Proxy. That Proxy is there + * to reduce the possibility of web based attacks through YARN. Hadoop can be configured to use + * filters to do authentication. That authentication then happens via the ResourceManager Proxy + * and Spark will use that to do authorization against the view acls. + * + * For other Spark deployments, the shared secret must be specified via the + * spark.authenticate.secret config. + * All the nodes (Master and Workers) and the applications need to have the same shared secret. + * This again is not ideal as one user could potentially affect another users application. + * This should be enhanced in the future to provide better protection. + * If the UI needs to be secured the user needs to install a javax servlet filter to do the + * authentication. Spark will then use that user to compare against the view acls to do + * authorization. If not filter is in place the user is generally null and no authorization + * can take place. + */ + +private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { + + // key used to store the spark secret in the Hadoop UGI + private val sparkSecretLookupKey = "sparkCookie" + + private val authOn = sparkConf.getBoolean("spark.authenticate", false) + private val uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + + // always add the current user and SPARK_USER to the viewAcls + private val aclUsers = ArrayBuffer[String](System.getProperty("user.name", ""), + Option(System.getenv("SPARK_USER")).getOrElse("")) + aclUsers ++= sparkConf.get("spark.ui.view.acls", "").split(',') + private val viewAcls = aclUsers.map(_.trim()).filter(!_.isEmpty).toSet + + private val secretKey = generateSecretKey() + logInfo("SecurityManager, is authentication enabled: " + authOn + + " are ui acls enabled: " + uiAclsOn + " users with view permissions: " + viewAcls.toString()) + + // Set our own authenticator to properly negotiate user/password for HTTP connections. + // This is needed by the HTTP client fetching from the HttpServer. Put here so its + // only set once. + if (authOn) { + Authenticator.setDefault( + new Authenticator() { + override def getPasswordAuthentication(): PasswordAuthentication = { + var passAuth: PasswordAuthentication = null + val userInfo = getRequestingURL().getUserInfo() + if (userInfo != null) { + val parts = userInfo.split(":", 2) + passAuth = new PasswordAuthentication(parts(0), parts(1).toCharArray()) + } + return passAuth + } + } + ) + } + + /** + * Generates or looks up the secret key. + * + * The way the key is stored depends on the Spark deployment mode. Yarn + * uses the Hadoop UGI. + * + * For non-Yarn deployments, If the config variable is not set + * we throw an exception. + */ + private def generateSecretKey(): String = { + if (!isAuthenticationEnabled) return null + // first check to see if the secret is already set, else generate a new one if on yarn + val sCookie = if (SparkHadoopUtil.get.isYarnMode) { + val secretKey = SparkHadoopUtil.get.getSecretKeyFromUserCredentials(sparkSecretLookupKey) + if (secretKey != null) { + logDebug("in yarn mode, getting secret from credentials") + return new Text(secretKey).toString + } else { + logDebug("getSecretKey: yarn mode, secret key from credentials is null") + } + val cookie = akka.util.Crypt.generateSecureCookie + // if we generated the secret then we must be the first so lets set it so t + // gets used by everyone else + SparkHadoopUtil.get.addSecretKeyToUserCredentials(sparkSecretLookupKey, cookie) + logInfo("adding secret to credentials in yarn mode") + cookie + } else { + // user must have set spark.authenticate.secret config + sparkConf.getOption("spark.authenticate.secret") match { + case Some(value) => value + case None => throw new Exception("Error: a secret key must be specified via the " + + "spark.authenticate.secret config") + } + } + sCookie + } + + /** + * Check to see if Acls for the UI are enabled + * @return true if UI authentication is enabled, otherwise false + */ + def uiAclsEnabled(): Boolean = uiAclsOn + + /** + * Checks the given user against the view acl list to see if they have + * authorization to view the UI. If the UI acls must are disabled + * via spark.ui.acls.enable, all users have view access. + * + * @param user to see if is authorized + * @return true is the user has permission, otherwise false + */ + def checkUIViewPermissions(user: String): Boolean = { + if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true + } + + /** + * Check to see if authentication for the Spark communication protocols is enabled + * @return true if authentication is enabled, otherwise false + */ + def isAuthenticationEnabled(): Boolean = authOn + + /** + * Gets the user used for authenticating HTTP connections. + * For now use a single hardcoded user. + * @return the HTTP user as a String + */ + def getHttpUser(): String = "sparkHttpUser" + + /** + * Gets the user used for authenticating SASL connections. + * For now use a single hardcoded user. + * @return the SASL user as a String + */ + def getSaslUser(): String = "sparkSaslUser" + + /** + * Gets the secret key. + * @return the secret key as a String if authentication is enabled, otherwise returns null + */ + def getSecretKey(): String = secretKey +} diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index da778aa851cd2..24731ad706a49 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -130,6 +130,8 @@ class SparkContext( val isLocal = (master == "local" || master.startsWith("local[")) + if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.create( conf, @@ -634,7 +636,7 @@ class SparkContext( addedFiles(key) = System.currentTimeMillis // Fetch the file locally in case a job is executed using DAGScheduler.runLocally(). - Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf) + Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf, env.securityManager) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) } diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 7ac65828f670f..5e43b5198422c 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -53,7 +53,8 @@ class SparkEnv private[spark] ( val httpFileServer: HttpFileServer, val sparkFilesDir: String, val metricsSystem: MetricsSystem, - val conf: SparkConf) extends Logging { + val conf: SparkConf, + val securityManager: SecurityManager) extends Logging { // A mapping of thread ID to amount of memory used for shuffle in bytes // All accesses should be manually synchronized @@ -122,8 +123,9 @@ object SparkEnv extends Logging { isDriver: Boolean, isLocal: Boolean): SparkEnv = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, - conf = conf) + val securityManager = new SecurityManager(conf) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, conf = conf, + securityManager = securityManager) // Bit of a hack: If this is the driver and our port was 0 (meaning bind to any free port), // figure out which port number Akka actually bound to and set spark.driver.port to it. @@ -139,7 +141,6 @@ object SparkEnv extends Logging { val name = conf.get(propertyName, defaultClassName) Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] } - val serializerManager = new SerializerManager val serializer = serializerManager.setDefault( @@ -167,12 +168,12 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf)), conf) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf) + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, + serializer, conf, securityManager) val connectionManager = blockManager.connectionManager - val broadcastManager = new BroadcastManager(isDriver, conf) + val broadcastManager = new BroadcastManager(isDriver, conf, securityManager) val cacheManager = new CacheManager(blockManager) @@ -190,14 +191,14 @@ object SparkEnv extends Logging { val shuffleFetcher = instantiateClass[ShuffleFetcher]( "spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher") - val httpFileServer = new HttpFileServer() + val httpFileServer = new HttpFileServer(securityManager) httpFileServer.initialize() conf.set("spark.fileserver.uri", httpFileServer.serverUri) val metricsSystem = if (isDriver) { - MetricsSystem.createMetricsSystem("driver", conf) + MetricsSystem.createMetricsSystem("driver", conf, securityManager) } else { - MetricsSystem.createMetricsSystem("executor", conf) + MetricsSystem.createMetricsSystem("executor", conf, securityManager) } metricsSystem.start() @@ -231,6 +232,7 @@ object SparkEnv extends Logging { httpFileServer, sparkFilesDir, metricsSystem, - conf) + conf, + securityManager) } } diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala new file mode 100644 index 0000000000000..a2a871cbd3c31 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -0,0 +1,146 @@ +/* + * 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 + +import java.io.IOException +import javax.security.auth.callback.Callback +import javax.security.auth.callback.CallbackHandler +import javax.security.auth.callback.NameCallback +import javax.security.auth.callback.PasswordCallback +import javax.security.auth.callback.UnsupportedCallbackException +import javax.security.sasl.RealmCallback +import javax.security.sasl.RealmChoiceCallback +import javax.security.sasl.Sasl +import javax.security.sasl.SaslClient +import javax.security.sasl.SaslException + +import scala.collection.JavaConversions.mapAsJavaMap + +/** + * Implements SASL Client logic for Spark + */ +private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logging { + + /** + * Used to respond to server's counterpart, SaslServer with SASL tokens + * represented as byte arrays. + * + * The authentication mechanism used here is DIGEST-MD5. This could be changed to be + * configurable in the future. + */ + private var saslClient: SaslClient = Sasl.createSaslClient(Array[String](SparkSaslServer.DIGEST), + null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, + new SparkSaslClientCallbackHandler(securityMgr)) + + /** + * Used to initiate SASL handshake with server. + * @return response to challenge if needed + */ + def firstToken(): Array[Byte] = { + synchronized { + val saslToken: Array[Byte] = + if (saslClient != null && saslClient.hasInitialResponse()) { + logDebug("has initial response") + saslClient.evaluateChallenge(new Array[Byte](0)) + } else { + new Array[Byte](0) + } + saslToken + } + } + + /** + * Determines whether the authentication exchange has completed. + * @return true is complete, otherwise false + */ + def isComplete(): Boolean = { + synchronized { + if (saslClient != null) saslClient.isComplete() else false + } + } + + /** + * Respond to server's SASL token. + * @param saslTokenMessage contains server's SASL token + * @return client's response SASL token + */ + def saslResponse(saslTokenMessage: Array[Byte]): Array[Byte] = { + synchronized { + if (saslClient != null) saslClient.evaluateChallenge(saslTokenMessage) else new Array[Byte](0) + } + } + + /** + * Disposes of any system resources or security-sensitive information the + * SaslClient might be using. + */ + def dispose() { + synchronized { + if (saslClient != null) { + try { + saslClient.dispose() + } catch { + case e: SaslException => // ignored + } finally { + saslClient = null + } + } + } + } + + /** + * Implementation of javax.security.auth.callback.CallbackHandler + * that works with share secrets. + */ + private class SparkSaslClientCallbackHandler(securityMgr: SecurityManager) extends + CallbackHandler { + + private val userName: String = + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + private val secretKey = securityMgr.getSecretKey() + private val userPassword: Array[Char] = + SparkSaslServer.encodePassword(if (secretKey != null) secretKey.getBytes() else "".getBytes()) + + /** + * Implementation used to respond to SASL request from the server. + * + * @param callbacks objects that indicate what credential information the + * server's SaslServer requires from the client. + */ + override def handle(callbacks: Array[Callback]) { + logDebug("in the sasl client callback handler") + callbacks foreach { + case nc: NameCallback => { + logDebug("handle: SASL client callback: setting username: " + userName) + nc.setName(userName) + } + case pc: PasswordCallback => { + logDebug("handle: SASL client callback: setting userPassword") + pc.setPassword(userPassword) + } + case rc: RealmCallback => { + logDebug("handle: SASL client callback: setting realm: " + rc.getDefaultText()) + rc.setText(rc.getDefaultText()) + } + case cb: RealmChoiceCallback => {} + case cb: Callback => throw + new UnsupportedCallbackException(cb, "handle: Unrecognized SASL client callback") + } + } + } +} diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala new file mode 100644 index 0000000000000..11fcb2ae3a5c5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -0,0 +1,174 @@ +/* + * 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 + +import javax.security.auth.callback.Callback +import javax.security.auth.callback.CallbackHandler +import javax.security.auth.callback.NameCallback +import javax.security.auth.callback.PasswordCallback +import javax.security.auth.callback.UnsupportedCallbackException +import javax.security.sasl.AuthorizeCallback +import javax.security.sasl.RealmCallback +import javax.security.sasl.Sasl +import javax.security.sasl.SaslException +import javax.security.sasl.SaslServer +import scala.collection.JavaConversions.mapAsJavaMap +import org.apache.commons.net.util.Base64 + +/** + * Encapsulates SASL server logic + */ +private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Logging { + + /** + * Actual SASL work done by this object from javax.security.sasl. + */ + private var saslServer: SaslServer = Sasl.createSaslServer(SparkSaslServer.DIGEST, null, + SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, + new SparkSaslDigestCallbackHandler(securityMgr)) + + /** + * Determines whether the authentication exchange has completed. + * @return true is complete, otherwise false + */ + def isComplete(): Boolean = { + synchronized { + if (saslServer != null) saslServer.isComplete() else false + } + } + + /** + * Used to respond to server SASL tokens. + * @param token Server's SASL token + * @return response to send back to the server. + */ + def response(token: Array[Byte]): Array[Byte] = { + synchronized { + if (saslServer != null) saslServer.evaluateResponse(token) else new Array[Byte](0) + } + } + + /** + * Disposes of any system resources or security-sensitive information the + * SaslServer might be using. + */ + def dispose() { + synchronized { + if (saslServer != null) { + try { + saslServer.dispose() + } catch { + case e: SaslException => // ignore + } finally { + saslServer = null + } + } + } + } + + /** + * Implementation of javax.security.auth.callback.CallbackHandler + * for SASL DIGEST-MD5 mechanism + */ + private class SparkSaslDigestCallbackHandler(securityMgr: SecurityManager) + extends CallbackHandler { + + private val userName: String = + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + + override def handle(callbacks: Array[Callback]) { + logDebug("In the sasl server callback handler") + callbacks foreach { + case nc: NameCallback => { + logDebug("handle: SASL server callback: setting username") + nc.setName(userName) + } + case pc: PasswordCallback => { + logDebug("handle: SASL server callback: setting userPassword") + val password: Array[Char] = + SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes()) + pc.setPassword(password) + } + case rc: RealmCallback => { + logDebug("handle: SASL server callback: setting realm: " + rc.getDefaultText()) + rc.setText(rc.getDefaultText()) + } + case ac: AuthorizeCallback => { + val authid = ac.getAuthenticationID() + val authzid = ac.getAuthorizationID() + if (authid.equals(authzid)) { + logDebug("set auth to true") + ac.setAuthorized(true) + } else { + logDebug("set auth to false") + ac.setAuthorized(false) + } + if (ac.isAuthorized()) { + logDebug("sasl server is authorized") + ac.setAuthorizedID(authzid) + } + } + case cb: Callback => throw + new UnsupportedCallbackException(cb, "handle: Unrecognized SASL DIGEST-MD5 Callback") + } + } + } +} + +private[spark] object SparkSaslServer { + + /** + * This is passed as the server name when creating the sasl client/server. + * This could be changed to be configurable in the future. + */ + val SASL_DEFAULT_REALM = "default" + + /** + * The authentication mechanism used here is DIGEST-MD5. This could be changed to be + * configurable in the future. + */ + val DIGEST = "DIGEST-MD5" + + /** + * The quality of protection is just "auth". This means that we are doing + * authentication only, we are not supporting integrity or privacy protection of the + * communication channel after authentication. This could be changed to be configurable + * in the future. + */ + val SASL_PROPS = Map(Sasl.QOP -> "auth", Sasl.SERVER_AUTH ->"true") + + /** + * Encode a byte[] identifier as a Base64-encoded string. + * + * @param identifier identifier to encode + * @return Base64-encoded string + */ + def encodeIdentifier(identifier: Array[Byte]): String = { + new String(Base64.encodeBase64(identifier)) + } + + /** + * Encode a password as a base64-encoded char[] array. + * @param password as a byte array. + * @return password as a char array. + */ + def encodePassword(password: Array[Byte]): Array[Char] = { + new String(Base64.encodeBase64(password)).toCharArray() + } +} + diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index d113d4040594d..e3c3a12d16f2a 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -60,7 +60,8 @@ abstract class Broadcast[T](val id: Long) extends Serializable { } private[spark] -class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging with Serializable { +class BroadcastManager(val _isDriver: Boolean, conf: SparkConf, securityManager: SecurityManager) + extends Logging with Serializable { private var initialized = false private var broadcastFactory: BroadcastFactory = null @@ -78,7 +79,7 @@ class BroadcastManager(val _isDriver: Boolean, conf: SparkConf) extends Logging Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] // Initialize appropriate BroadcastFactory and BroadcastObject - broadcastFactory.initialize(isDriver, conf) + broadcastFactory.initialize(isDriver, conf, securityManager) initialized = true } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 940e5ab805100..6beecaeced5be 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -16,6 +16,7 @@ */ package org.apache.spark.broadcast +import org.apache.spark.SecurityManager import org.apache.spark.SparkConf @@ -26,7 +27,7 @@ import org.apache.spark.SparkConf * entire Spark job. */ trait BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf): Unit + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index 20207c261320b..e8eb04bb10469 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -18,13 +18,13 @@ package org.apache.spark.broadcast import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream} -import java.net.URL +import java.net.{URL, URLConnection, URI} import java.util.concurrent.TimeUnit import it.unimi.dsi.fastutil.io.FastBufferedInputStream import it.unimi.dsi.fastutil.io.FastBufferedOutputStream -import org.apache.spark.{HttpServer, Logging, SparkConf, SparkEnv} +import org.apache.spark.{SparkConf, HttpServer, Logging, SecurityManager, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils} @@ -67,7 +67,9 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea * A [[BroadcastFactory]] implementation that uses a HTTP server as the broadcast medium. */ class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf) { HttpBroadcast.initialize(isDriver, conf) } + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + HttpBroadcast.initialize(isDriver, conf, securityMgr) + } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new HttpBroadcast[T](value_, isLocal, id) @@ -83,6 +85,7 @@ private object HttpBroadcast extends Logging { private var bufferSize: Int = 65536 private var serverUri: String = null private var server: HttpServer = null + private var securityManager: SecurityManager = null // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist private val files = new TimeStampedHashSet[String] @@ -92,11 +95,12 @@ private object HttpBroadcast extends Logging { private var compressionCodec: CompressionCodec = null - def initialize(isDriver: Boolean, conf: SparkConf) { + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { synchronized { if (!initialized) { bufferSize = conf.getInt("spark.buffer.size", 65536) compress = conf.getBoolean("spark.broadcast.compress", true) + securityManager = securityMgr if (isDriver) { createServer(conf) conf.set("spark.httpBroadcast.uri", serverUri) @@ -126,7 +130,7 @@ private object HttpBroadcast extends Logging { private def createServer(conf: SparkConf) { broadcastDir = Utils.createTempDir(Utils.getLocalDir(conf)) - server = new HttpServer(broadcastDir) + server = new HttpServer(broadcastDir, securityManager) server.start() serverUri = server.uri logInfo("Broadcast server started at " + serverUri) @@ -149,11 +153,23 @@ private object HttpBroadcast extends Logging { } def read[T](id: Long): T = { + logDebug("broadcast read server: " + serverUri + " id: broadcast-" + id) val url = serverUri + "/" + BroadcastBlockId(id).name + + var uc: URLConnection = null + if (securityManager.isAuthenticationEnabled()) { + logDebug("broadcast security enabled") + val newuri = Utils.constructURIForAuthentication(new URI(url), securityManager) + uc = newuri.toURL().openConnection() + uc.setAllowUserInteraction(false) + } else { + logDebug("broadcast not using security") + uc = new URL(url).openConnection() + } + val in = { - val httpConnection = new URL(url).openConnection() - httpConnection.setReadTimeout(httpReadTimeout) - val inputStream = httpConnection.getInputStream + uc.setReadTimeout(httpReadTimeout) + val inputStream = uc.getInputStream(); if (compress) { compressionCodec.compressedInputStream(inputStream) } else { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 22d783c8590c6..3cd71213769b7 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -241,7 +241,9 @@ private[spark] case class TorrentInfo( */ class TorrentBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.initialize(isDriver, conf) } + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + TorrentBroadcast.initialize(isDriver, conf) + } def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = new TorrentBroadcast[T](value_, isLocal, id) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index eb5676b51d836..d9e3035e1ab59 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -26,7 +26,7 @@ import akka.pattern.ask import akka.remote.{AssociationErrorEvent, DisassociatedEvent, RemotingLifecycleEvent} import org.apache.log4j.{Level, Logger} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} import org.apache.spark.util.{AkkaUtils, Utils} @@ -141,7 +141,7 @@ object Client { // TODO: See if we can initialize akka so return messages are sent back using the same TCP // flow. Else, this (sadly) requires the DriverClient be routable from the Master. val (actorSystem, _) = AkkaUtils.createActorSystem( - "driverClient", Utils.localHostName(), 0, false, conf) + "driverClient", Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[ClientActor], driverArgs, conf)) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index ec15647e1d9eb..d2d8d6d662d55 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -21,6 +21,7 @@ import java.security.PrivilegedExceptionAction import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation import org.apache.spark.{SparkContext, SparkException} @@ -65,6 +66,15 @@ class SparkHadoopUtil { def addCredentials(conf: JobConf) {} def isYarnMode(): Boolean = { false } + + def getCurrentUserCredentials(): Credentials = { null } + + def addCurrentUserCredentials(creds: Credentials) {} + + def addSecretKeyToUserCredentials(key: String, secret: String) {} + + def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { null } + } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 1550c3eb4286b..63f166d401059 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy.client -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf, Logging} import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.util.{AkkaUtils, Utils} @@ -45,8 +45,9 @@ private[spark] object TestClient { def main(args: Array[String]) { val url = args(0) + val conf = new SparkConf val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, - conf = new SparkConf) + conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription( "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), Some("dummy-spark-home"), "ignored") diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 51794ce40cb45..2d6d0c33fac7e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -30,7 +30,7 @@ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.DriverState.DriverState @@ -39,7 +39,8 @@ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem import org.apache.spark.util.{AkkaUtils, Utils} -private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Actor with Logging { +private[spark] class Master(host: String, port: Int, webUiPort: Int, + val securityMgr: SecurityManager) extends Actor with Logging { import context.dispatcher // to use Akka's scheduler.schedule() val conf = new SparkConf @@ -70,8 +71,9 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int) extends Act Utils.checkHost(host, "Expected hostname") - val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf) - val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf) + val masterMetricsSystem = MetricsSystem.createMetricsSystem("master", conf, securityMgr) + val applicationMetricsSystem = MetricsSystem.createMetricsSystem("applications", conf, + securityMgr) val masterSource = new MasterSource(this) val webUi = new MasterWebUI(this, webUiPort) @@ -711,8 +713,11 @@ private[spark] object Master { def startSystemAndActor(host: String, port: Int, webUiPort: Int, conf: SparkConf) : (ActorSystem, Int, Int) = { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf) - val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort), actorName) + val securityMgr = new SecurityManager(conf) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf, + securityManager = securityMgr) + val actor = actorSystem.actorOf(Props(classOf[Master], host, boundPort, webUiPort, + securityMgr), actorName) val timeout = AkkaUtils.askTimeout(conf) val respFuture = actor.ask(RequestWebUIPort)(timeout) val resp = Await.result(respFuture, timeout).asInstanceOf[WebUIPortResponse] diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 5ab13e7aa6b1f..a7bd01e284c8e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -18,8 +18,8 @@ package org.apache.spark.deploy.master.ui import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.server.{Handler, Server} +import org.eclipse.jetty.server.Server +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.master.Master @@ -46,7 +46,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { def start() { try { - val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers, master.conf) server = Some(srv) boundPort = Some(bPort) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort.get)) @@ -60,12 +60,17 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { val metricsHandlers = master.masterMetricsSystem.getServletHandlers ++ master.applicationMetricsSystem.getServletHandlers - val handlers = metricsHandlers ++ Array[(String, Handler)]( - ("/static", createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR)), - ("/app/json", (request: HttpServletRequest) => applicationPage.renderJson(request)), - ("/app", (request: HttpServletRequest) => applicationPage.render(request)), - ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), - ("*", (request: HttpServletRequest) => indexPage.render(request)) + val handlers = metricsHandlers ++ Seq[ServletContextHandler]( + createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static/*"), + createServletHandler("/app/json", + createServlet((request: HttpServletRequest) => applicationPage.renderJson(request), + master.securityMgr)), + createServletHandler("/app", createServlet((request: HttpServletRequest) => applicationPage + .render(request), master.securityMgr)), + createServletHandler("/json", createServlet((request: HttpServletRequest) => indexPage + .renderJson(request), master.securityMgr)), + createServletHandler("*", createServlet((request: HttpServletRequest) => indexPage.render + (request), master.securityMgr)) ) def stop() { @@ -74,5 +79,5 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { } private[spark] object MasterWebUI { - val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + val STATIC_RESOURCE_DIR = "org/apache/spark/ui" } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala index a26e47950a0ec..be15138f62406 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverWrapper.scala @@ -19,7 +19,7 @@ package org.apache.spark.deploy.worker import akka.actor._ -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.util.{AkkaUtils, Utils} /** @@ -29,8 +29,9 @@ object DriverWrapper { def main(args: Array[String]) { args.toList match { case workerUrl :: mainClass :: extraArgs => + val conf = new SparkConf() val (actorSystem, _) = AkkaUtils.createActorSystem("Driver", - Utils.localHostName(), 0, false, new SparkConf()) + Utils.localHostName(), 0, false, conf, new SecurityManager(conf)) actorSystem.actorOf(Props(classOf[WorkerWatcher], workerUrl), name = "workerWatcher") // Delegate to supplied main class diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 7b0b7861b76e1..afaabedffefea 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -27,7 +27,7 @@ import scala.concurrent.duration._ import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ExecutorDescription, ExecutorState} import org.apache.spark.deploy.DeployMessages._ import org.apache.spark.deploy.master.{DriverState, Master} @@ -48,7 +48,8 @@ private[spark] class Worker( actorSystemName: String, actorName: String, workDirPath: String = null, - val conf: SparkConf) + val conf: SparkConf, + val securityMgr: SecurityManager) extends Actor with Logging { import context.dispatcher @@ -91,7 +92,7 @@ private[spark] class Worker( var coresUsed = 0 var memoryUsed = 0 - val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf) + val metricsSystem = MetricsSystem.createMetricsSystem("worker", conf, securityMgr) val workerSource = new WorkerSource(this) def coresFree: Int = cores - coresUsed @@ -347,10 +348,11 @@ private[spark] object Worker { val conf = new SparkConf val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") val actorName = "Worker" + val securityMgr = new SecurityManager(conf) val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, - conf = conf) + conf = conf, securityManager = securityMgr) actorSystem.actorOf(Props(classOf[Worker], host, boundPort, webUiPort, cores, memory, - masterUrls, systemName, actorName, workDir, conf), name = actorName) + masterUrls, systemName, actorName, workDir, conf, securityMgr), name = actorName) (actorSystem, boundPort) } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index bdf126f93abc8..ffc05bd30687a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -19,8 +19,8 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.server.{Handler, Server} +import org.eclipse.jetty.server.Server +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker @@ -33,7 +33,7 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends Logging { + extends Logging { val timeout = AkkaUtils.askTimeout(worker.conf) val host = Utils.localHostName() val port = requestedPort.getOrElse( @@ -46,17 +46,21 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val metricsHandlers = worker.metricsSystem.getServletHandlers - val handlers = metricsHandlers ++ Array[(String, Handler)]( - ("/static", createStaticHandler(WorkerWebUI.STATIC_RESOURCE_DIR)), - ("/log", (request: HttpServletRequest) => log(request)), - ("/logPage", (request: HttpServletRequest) => logPage(request)), - ("/json", (request: HttpServletRequest) => indexPage.renderJson(request)), - ("*", (request: HttpServletRequest) => indexPage.render(request)) + val handlers = metricsHandlers ++ Seq[ServletContextHandler]( + createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static/*"), + createServletHandler("/log", createServlet((request: HttpServletRequest) => log(request), + worker.securityMgr)), + createServletHandler("/logPage", createServlet((request: HttpServletRequest) => logPage + (request), worker.securityMgr)), + createServletHandler("/json", createServlet((request: HttpServletRequest) => indexPage + .renderJson(request), worker.securityMgr)), + createServletHandler("*", createServlet((request: HttpServletRequest) => indexPage.render + (request), worker.securityMgr)) ) def start() { try { - val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers) + val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers, worker.conf) server = Some(srv) boundPort = Some(bPort) logInfo("Started Worker web UI at http://%s:%d".format(host, bPort)) @@ -198,6 +202,6 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } private[spark] object WorkerWebUI { - val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + val STATIC_RESOURCE_BASE = "org/apache/spark/ui" val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 0aae569b17272..3486092a140fb 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -22,7 +22,7 @@ import java.nio.ByteBuffer import akka.actor._ import akka.remote._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf, Logging} import org.apache.spark.TaskState.TaskState import org.apache.spark.deploy.worker.WorkerWatcher import org.apache.spark.scheduler.cluster.CoarseGrainedClusterMessages._ @@ -97,10 +97,11 @@ private[spark] object CoarseGrainedExecutorBackend { // Debug code Utils.checkHost(hostname) + val conf = new SparkConf // Create a new ActorSystem to run the backend, because we can't create a SparkEnv / Executor // before getting started with all our system properties, etc val (actorSystem, boundPort) = AkkaUtils.createActorSystem("sparkExecutor", hostname, 0, - indestructible = true, conf = new SparkConf) + indestructible = true, conf = conf, new SecurityManager(conf)) // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 989d666f15600..e69f6f72d3275 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -69,11 +69,6 @@ private[spark] class Executor( conf.set("spark.local.dir", getYarnLocalDirs()) } - // Create our ClassLoader and set it on this thread - private val urlClassLoader = createClassLoader() - private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) - Thread.currentThread.setContextClassLoader(replClassLoader) - if (!isLocal) { // Setup an uncaught exception handler for non-local mode. // Make any thread terminations due to uncaught exceptions kill the entire @@ -117,6 +112,12 @@ private[spark] class Executor( } } + // Create our ClassLoader and set it on this thread + // do this after SparkEnv creation so can access the SecurityManager + private val urlClassLoader = createClassLoader() + private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) + Thread.currentThread.setContextClassLoader(replClassLoader) + // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. private val akkaFrameSize = { @@ -338,12 +339,12 @@ private[spark] class Executor( // Fetch missing dependencies for ((name, timestamp) <- newFiles if currentFiles.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, env.securityManager) currentFiles(name) = timestamp } for ((name, timestamp) <- newJars if currentJars.getOrElse(name, -1L) < timestamp) { logInfo("Fetching " + name + " with timestamp " + timestamp) - Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf) + Utils.fetchFile(name, new File(SparkFiles.getRootDirectory), conf, env.securityManager) currentJars(name) = timestamp // Add it to our class loader val localName = name.split("/").last diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index 966c092124266..c5bda2078fc14 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -24,7 +24,7 @@ import scala.collection.mutable import com.codahale.metrics.{Metric, MetricFilter, MetricRegistry} -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SecurityManager, SparkConf} import org.apache.spark.metrics.sink.{MetricsServlet, Sink} import org.apache.spark.metrics.source.Source @@ -64,7 +64,7 @@ import org.apache.spark.metrics.source.Source * [options] is the specific property of this source or sink. */ private[spark] class MetricsSystem private (val instance: String, - conf: SparkConf) extends Logging { + conf: SparkConf, securityMgr: SecurityManager) extends Logging { val confFile = conf.get("spark.metrics.conf", null) val metricsConfig = new MetricsConfig(Option(confFile)) @@ -131,8 +131,8 @@ private[spark] class MetricsSystem private (val instance: String, val classPath = kv._2.getProperty("class") try { val sink = Class.forName(classPath) - .getConstructor(classOf[Properties], classOf[MetricRegistry]) - .newInstance(kv._2, registry) + .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) + .newInstance(kv._2, registry, securityMgr) if (kv._1 == "servlet") { metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) } else { @@ -160,6 +160,7 @@ private[spark] object MetricsSystem { } } - def createMetricsSystem(instance: String, conf: SparkConf): MetricsSystem = - new MetricsSystem(instance, conf) + def createMetricsSystem(instance: String, conf: SparkConf, + securityMgr: SecurityManager): MetricsSystem = + new MetricsSystem(instance, conf, securityMgr) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 98fa1dbd7c6ab..4d2ffc54d8983 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -22,9 +22,11 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{ConsoleReporter, MetricRegistry} +import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class ConsoleSink(val property: Properties, val registry: MetricRegistry) extends Sink { +class ConsoleSink(val property: Properties, val registry: MetricRegistry, + securityMgr: SecurityManager) extends Sink { val CONSOLE_DEFAULT_PERIOD = 10 val CONSOLE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 40f64768e6885..319f40815d65f 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -23,9 +23,11 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.{CsvReporter, MetricRegistry} +import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class CsvSink(val property: Properties, val registry: MetricRegistry) extends Sink { +class CsvSink(val property: Properties, val registry: MetricRegistry, + securityMgr: SecurityManager) extends Sink { val CSV_KEY_PERIOD = "period" val CSV_KEY_UNIT = "unit" val CSV_KEY_DIR = "directory" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala index 410ca0704b5c4..cd37317da77de 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala @@ -24,9 +24,11 @@ import com.codahale.metrics.MetricRegistry import com.codahale.metrics.ganglia.GangliaReporter import info.ganglia.gmetric4j.gmetric.GMetric +import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class GangliaSink(val property: Properties, val registry: MetricRegistry) extends Sink { +class GangliaSink(val property: Properties, val registry: MetricRegistry, + securityMgr: SecurityManager) extends Sink { val GANGLIA_KEY_PERIOD = "period" val GANGLIA_DEFAULT_PERIOD = 10 diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index e09be001421fc..0ffdf3846dc4a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -24,9 +24,11 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.MetricRegistry import com.codahale.metrics.graphite.{Graphite, GraphiteReporter} +import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class GraphiteSink(val property: Properties, val registry: MetricRegistry) extends Sink { +class GraphiteSink(val property: Properties, val registry: MetricRegistry, + securityMgr: SecurityManager) extends Sink { val GRAPHITE_DEFAULT_PERIOD = 10 val GRAPHITE_DEFAULT_UNIT = "SECONDS" val GRAPHITE_DEFAULT_PREFIX = "" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index b5cf210af2119..3b5edd5c376f0 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -20,8 +20,11 @@ package org.apache.spark.metrics.sink import java.util.Properties import com.codahale.metrics.{JmxReporter, MetricRegistry} +import org.apache.spark.SecurityManager + +class JmxSink(val property: Properties, val registry: MetricRegistry, + securityMgr: SecurityManager) extends Sink { -class JmxSink(val property: Properties, val registry: MetricRegistry) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() override def start() { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 3cdfe26d40f66..3110eccdee4fc 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -19,16 +19,19 @@ package org.apache.spark.metrics.sink import java.util.Properties import java.util.concurrent.TimeUnit + import javax.servlet.http.HttpServletRequest import com.codahale.metrics.MetricRegistry import com.codahale.metrics.json.MetricsModule import com.fasterxml.jackson.databind.ObjectMapper -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.servlet.ServletContextHandler +import org.apache.spark.SecurityManager import org.apache.spark.ui.JettyUtils -class MetricsServlet(val property: Properties, val registry: MetricRegistry) extends Sink { +class MetricsServlet(val property: Properties, val registry: MetricRegistry, + securityMgr: SecurityManager) extends Sink { val SERVLET_KEY_PATH = "path" val SERVLET_KEY_SAMPLE = "sample" @@ -42,8 +45,11 @@ class MetricsServlet(val property: Properties, val registry: MetricRegistry) ext val mapper = new ObjectMapper().registerModule( new MetricsModule(TimeUnit.SECONDS, TimeUnit.MILLISECONDS, servletShowSample)) - def getHandlers = Array[(String, Handler)]( - (servletPath, JettyUtils.createHandler(request => getMetricsSnapshot(request), "text/json")) + def getHandlers = Array[ServletContextHandler]( + JettyUtils.createServletHandler(servletPath, + JettyUtils.createServlet( + new JettyUtils.ServletParams(request => getMetricsSnapshot(request), "text/json"), + securityMgr) ) ) def getMetricsSnapshot(request: HttpServletRequest): String = { diff --git a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala index d3c09b16063d6..04df2f3b0d696 100644 --- a/core/src/main/scala/org/apache/spark/network/BufferMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/BufferMessage.scala @@ -45,9 +45,10 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: throw new Exception("Max chunk size is " + maxChunkSize) } + val security = if (isSecurityNeg) 1 else 0 if (size == 0 && !gotChunkForSendingOnce) { val newChunk = new MessageChunk( - new MessageChunkHeader(typ, id, 0, 0, ackId, senderAddress), null) + new MessageChunkHeader(typ, id, 0, 0, ackId, security, senderAddress), null) gotChunkForSendingOnce = true return Some(newChunk) } @@ -65,7 +66,7 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: } buffer.position(buffer.position + newBuffer.remaining) val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) + typ, id, size, newBuffer.remaining, ackId, security, senderAddress), newBuffer) gotChunkForSendingOnce = true return Some(newChunk) } @@ -79,6 +80,7 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: throw new Exception("Attempting to get chunk from message with multiple data buffers") } val buffer = buffers(0) + val security = if (isSecurityNeg) 1 else 0 if (buffer.remaining > 0) { if (buffer.remaining < chunkSize) { throw new Exception("Not enough space in data buffer for receiving chunk") @@ -86,7 +88,7 @@ class BufferMessage(id_ : Int, val buffers: ArrayBuffer[ByteBuffer], var ackId: val newBuffer = buffer.slice().limit(chunkSize).asInstanceOf[ByteBuffer] buffer.position(buffer.position + newBuffer.remaining) val newChunk = new MessageChunk(new MessageChunkHeader( - typ, id, size, newBuffer.remaining, ackId, senderAddress), newBuffer) + typ, id, size, newBuffer.remaining, ackId, security, senderAddress), newBuffer) return Some(newChunk) } None diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 8219a185ea983..8fd9c2b87d256 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -17,6 +17,11 @@ package org.apache.spark.network +import org.apache.spark._ +import org.apache.spark.SparkSaslServer + +import scala.collection.mutable.{HashMap, Queue, ArrayBuffer} + import java.net._ import java.nio._ import java.nio.channels._ @@ -27,13 +32,16 @@ import org.apache.spark._ private[spark] abstract class Connection(val channel: SocketChannel, val selector: Selector, - val socketRemoteConnectionManagerId: ConnectionManagerId) + val socketRemoteConnectionManagerId: ConnectionManagerId, val connectionId: ConnectionId) extends Logging { - def this(channel_ : SocketChannel, selector_ : Selector) = { + var sparkSaslServer: SparkSaslServer = null + var sparkSaslClient: SparkSaslClient = null + + def this(channel_ : SocketChannel, selector_ : Selector, id_ : ConnectionId) = { this(channel_, selector_, ConnectionManagerId.fromSocketAddress( - channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress])) + channel_.socket.getRemoteSocketAddress().asInstanceOf[InetSocketAddress]), id_) } channel.configureBlocking(false) @@ -49,6 +57,16 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, val remoteAddress = getRemoteAddress() + /** + * Used to synchronize client requests: client's work-related requests must + * wait until SASL authentication completes. + */ + private val authenticated = new Object() + + def getAuthenticated(): Object = authenticated + + def isSaslComplete(): Boolean + def resetForceReregister(): Boolean // Read channels typically do not register for write and write does not for read @@ -69,6 +87,16 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, // Will be true for ReceivingConnection, false for SendingConnection. def changeInterestForRead(): Boolean + private def disposeSasl() { + if (sparkSaslServer != null) { + sparkSaslServer.dispose(); + } + + if (sparkSaslClient != null) { + sparkSaslClient.dispose() + } + } + // On receiving a write event, should we change the interest for this channel or not ? // Will be false for ReceivingConnection, true for SendingConnection. // Actually, for now, should not get triggered for ReceivingConnection @@ -101,6 +129,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, k.cancel() } channel.close() + disposeSasl() callOnCloseCallback() } @@ -168,8 +197,12 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, private[spark] class SendingConnection(val address: InetSocketAddress, selector_ : Selector, - remoteId_ : ConnectionManagerId) - extends Connection(SocketChannel.open, selector_, remoteId_) { + remoteId_ : ConnectionManagerId, id_ : ConnectionId) + extends Connection(SocketChannel.open, selector_, remoteId_, id_) { + + def isSaslComplete(): Boolean = { + if (sparkSaslClient != null) sparkSaslClient.isComplete() else false + } private class Outbox { val messages = new Queue[Message]() @@ -226,6 +259,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, data as detailed in https://github.com/mesos/spark/pull/791 */ private var needForceReregister = false + val currentBuffers = new ArrayBuffer[ByteBuffer]() /*channel.socket.setSendBufferSize(256 * 1024)*/ @@ -316,6 +350,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, // If we have 'seen' pending messages, then reset flag - since we handle that as // normal registering of event (below) if (needForceReregister && buffers.exists(_.remaining() > 0)) resetForceReregister() + currentBuffers ++= buffers } case None => { @@ -384,8 +419,15 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, // Must be created within selector loop - else deadlock -private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : Selector) - extends Connection(channel_, selector_) { +private[spark] class ReceivingConnection( + channel_ : SocketChannel, + selector_ : Selector, + id_ : ConnectionId) + extends Connection(channel_, selector_, id_) { + + def isSaslComplete(): Boolean = { + if (sparkSaslServer != null) sparkSaslServer.isComplete() else false + } class Inbox() { val messages = new HashMap[Int, BufferMessage]() @@ -396,6 +438,7 @@ private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : S val newMessage = Message.create(header).asInstanceOf[BufferMessage] newMessage.started = true newMessage.startTime = System.currentTimeMillis + newMessage.isSecurityNeg = header.securityNeg == 1 logDebug( "Starting to receive [" + newMessage + "] from [" + getRemoteConnectionManagerId() + "]") messages += ((newMessage.id, newMessage)) @@ -441,7 +484,7 @@ private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : S val inbox = new Inbox() val headerBuffer: ByteBuffer = ByteBuffer.allocate(MessageChunkHeader.HEADER_SIZE) - var onReceiveCallback: (Connection , Message) => Unit = null + var onReceiveCallback: (Connection, Message) => Unit = null var currentChunk: MessageChunk = null channel.register(selector, SelectionKey.OP_READ) @@ -516,7 +559,7 @@ private[spark] class ReceivingConnection(channel_ : SocketChannel, selector_ : S } } } catch { - case e: Exception => { + case e: Exception => { logWarning("Error reading from connection to " + getRemoteConnectionManagerId(), e) callOnExceptionCallback(e) close() diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionId.scala new file mode 100644 index 0000000000000..ffaab677d411a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/ConnectionId.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.network + +private[spark] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { + override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId +} + +private[spark] object ConnectionId { + + def createConnectionIdFromString(connectionIdString: String): ConnectionId = { + val res = connectionIdString.split("_").map(_.trim()) + if (res.size != 3) { + throw new Exception("Error converting ConnectionId string: " + connectionIdString + + " to a ConnectionId Object") + } + new ConnectionId(new ConnectionManagerId(res(0), res(1).toInt), res(2).toInt) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index a7f20f8c51a5a..a75130cba2a2e 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -21,6 +21,9 @@ import java.net._ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ +import java.net._ +import java.util.concurrent.atomic.AtomicInteger + import java.util.concurrent.{LinkedBlockingDeque, TimeUnit, ThreadPoolExecutor} import scala.collection.mutable.ArrayBuffer @@ -28,13 +31,15 @@ import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet import scala.collection.mutable.SynchronizedMap import scala.collection.mutable.SynchronizedQueue + import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.concurrent.duration._ import org.apache.spark._ -import org.apache.spark.util.Utils +import org.apache.spark.util.{SystemClock, Utils} -private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Logging { +private[spark] class ConnectionManager(port: Int, conf: SparkConf, + securityManager: SecurityManager) extends Logging { class MessageStatus( val message: Message, @@ -50,6 +55,9 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private val selector = SelectorProvider.provider.openSelector() + // default to 30 second timeout waiting for authentication + private val authTimeout = conf.getInt("spark.core.connection.auth.wait.timeout", 30) + private val handleMessageExecutor = new ThreadPoolExecutor( conf.getInt("spark.core.connection.handler.threads.min", 20), conf.getInt("spark.core.connection.handler.threads.max", 60), @@ -71,6 +79,9 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi new LinkedBlockingDeque[Runnable]()) private val serverChannel = ServerSocketChannel.open() + // used to track the SendingConnections waiting to do SASL negotiation + private val connectionsAwaitingSasl = new HashMap[ConnectionId, SendingConnection] + with SynchronizedMap[ConnectionId, SendingConnection] private val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] private val connectionsById = new HashMap[ConnectionManagerId, SendingConnection] @@ -84,6 +95,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private var onReceiveCallback: (BufferMessage, ConnectionManagerId) => Option[Message]= null + private val authEnabled = securityManager.isAuthenticationEnabled() + serverChannel.configureBlocking(false) serverChannel.socket.setReuseAddress(true) serverChannel.socket.setReceiveBufferSize(256 * 1024) @@ -94,6 +107,10 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi val id = new ConnectionManagerId(Utils.localHostName, serverChannel.socket.getLocalPort) logInfo("Bound socket to port " + serverChannel.socket.getLocalPort() + " with id = " + id) + // used in combination with the ConnectionManagerId to create unique Connection ids + // to be able to track asynchronous messages + private val idCount: AtomicInteger = new AtomicInteger(1) + private val selectorThread = new Thread("connection-manager-thread") { override def run() = ConnectionManager.this.run() } @@ -125,7 +142,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi } finally { writeRunnableStarted.synchronized { writeRunnableStarted -= key - val needReregister = register || conn.resetForceReregister() + val needReregister = register || conn.resetForceReregister() if (needReregister && conn.changeInterestForWrite()) { conn.registerInterest() } @@ -372,7 +389,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi // accept them all in a tight loop. non blocking accept with no processing, should be fine while (newChannel != null) { try { - val newConnection = new ReceivingConnection(newChannel, selector) + val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) + val newConnection = new ReceivingConnection(newChannel, selector, newConnectionId) newConnection.onReceive(receiveMessage) addListeners(newConnection) addConnection(newConnection) @@ -406,6 +424,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi logInfo("Removing SendingConnection to " + sendingConnectionManagerId) connectionsById -= sendingConnectionManagerId + connectionsAwaitingSasl -= connection.connectionId messageStatuses.synchronized { messageStatuses @@ -481,7 +500,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi val creationTime = System.currentTimeMillis def run() { logDebug("Handler thread delay is " + (System.currentTimeMillis - creationTime) + " ms") - handleMessage(connectionManagerId, message) + handleMessage(connectionManagerId, message, connection) logDebug("Handling delay is " + (System.currentTimeMillis - creationTime) + " ms") } } @@ -489,10 +508,133 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi /*handleMessage(connection, message)*/ } - private def handleMessage(connectionManagerId: ConnectionManagerId, message: Message) { + private def handleClientAuthentication( + waitingConn: SendingConnection, + securityMsg: SecurityMessage, + connectionId : ConnectionId) { + if (waitingConn.isSaslComplete()) { + logDebug("Client sasl completed for id: " + waitingConn.connectionId) + connectionsAwaitingSasl -= waitingConn.connectionId + waitingConn.getAuthenticated().synchronized { + waitingConn.getAuthenticated().notifyAll(); + } + return + } else { + var replyToken : Array[Byte] = null + try { + replyToken = waitingConn.sparkSaslClient.saslResponse(securityMsg.getToken); + if (waitingConn.isSaslComplete()) { + logDebug("Client sasl completed after evaluate for id: " + waitingConn.connectionId) + connectionsAwaitingSasl -= waitingConn.connectionId + waitingConn.getAuthenticated().synchronized { + waitingConn.getAuthenticated().notifyAll() + } + return + } + var securityMsgResp = SecurityMessage.fromResponse(replyToken, + securityMsg.getConnectionId.toString()) + var message = securityMsgResp.toBufferMessage + if (message == null) throw new Exception("Error creating security message") + sendSecurityMessage(waitingConn.getRemoteConnectionManagerId(), message) + } catch { + case e: Exception => { + logError("Error handling sasl client authentication", e) + waitingConn.close() + throw new Exception("Error evaluating sasl response: " + e) + } + } + } + } + + private def handleServerAuthentication( + connection: Connection, + securityMsg: SecurityMessage, + connectionId: ConnectionId) { + if (!connection.isSaslComplete()) { + logDebug("saslContext not established") + var replyToken : Array[Byte] = null + try { + connection.synchronized { + if (connection.sparkSaslServer == null) { + logDebug("Creating sasl Server") + connection.sparkSaslServer = new SparkSaslServer(securityManager) + } + } + replyToken = connection.sparkSaslServer.response(securityMsg.getToken) + if (connection.isSaslComplete()) { + logDebug("Server sasl completed: " + connection.connectionId) + } else { + logDebug("Server sasl not completed: " + connection.connectionId) + } + if (replyToken != null) { + var securityMsgResp = SecurityMessage.fromResponse(replyToken, + securityMsg.getConnectionId) + var message = securityMsgResp.toBufferMessage + if (message == null) throw new Exception("Error creating security Message") + sendSecurityMessage(connection.getRemoteConnectionManagerId(), message) + } + } catch { + case e: Exception => { + logError("Error in server auth negotiation: " + e) + // It would probably be better to send an error message telling other side auth failed + // but for now just close + connection.close() + } + } + } else { + logDebug("connection already established for this connection id: " + connection.connectionId) + } + } + + + private def handleAuthentication(conn: Connection, bufferMessage: BufferMessage): Boolean = { + if (bufferMessage.isSecurityNeg) { + logDebug("This is security neg message") + + // parse as SecurityMessage + val securityMsg = SecurityMessage.fromBufferMessage(bufferMessage) + val connectionId = ConnectionId.createConnectionIdFromString(securityMsg.getConnectionId) + + connectionsAwaitingSasl.get(connectionId) match { + case Some(waitingConn) => { + // Client - this must be in response to us doing Send + logDebug("Client handleAuth for id: " + waitingConn.connectionId) + handleClientAuthentication(waitingConn, securityMsg, connectionId) + } + case None => { + // Server - someone sent us something and we haven't authenticated yet + logDebug("Server handleAuth for id: " + connectionId) + handleServerAuthentication(conn, securityMsg, connectionId) + } + } + return true + } else { + if (!conn.isSaslComplete()) { + // We could handle this better and tell the client we need to do authentication + // negotiation, but for now just ignore them. + logError("message sent that is not security negotiation message on connection " + + "not authenticated yet, ignoring it!!") + return true + } + } + return false + } + + private def handleMessage( + connectionManagerId: ConnectionManagerId, + message: Message, + connection: Connection) { logDebug("Handling [" + message + "] from [" + connectionManagerId + "]") message match { case bufferMessage: BufferMessage => { + if (authEnabled) { + val res = handleAuthentication(connection, bufferMessage) + if (res == true) { + // message was security negotiation so skip the rest + logDebug("After handleAuth result was true, returning") + return + } + } if (bufferMessage.hasAckId) { val sentMessageStatus = messageStatuses.synchronized { messageStatuses.get(bufferMessage.ackId) match { @@ -541,17 +683,124 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi } } + private def checkSendAuthFirst(connManagerId: ConnectionManagerId, conn: SendingConnection) { + // see if we need to do sasl before writing + // this should only be the first negotiation as the Client!!! + if (!conn.isSaslComplete()) { + conn.synchronized { + if (conn.sparkSaslClient == null) { + conn.sparkSaslClient = new SparkSaslClient(securityManager) + var firstResponse: Array[Byte] = null + try { + firstResponse = conn.sparkSaslClient.firstToken() + var securityMsg = SecurityMessage.fromResponse(firstResponse, + conn.connectionId.toString()) + var message = securityMsg.toBufferMessage + if (message == null) throw new Exception("Error creating security message") + connectionsAwaitingSasl += ((conn.connectionId, conn)) + sendSecurityMessage(connManagerId, message) + logDebug("adding connectionsAwaitingSasl id: " + conn.connectionId) + } catch { + case e: Exception => { + logError("Error getting first response from the SaslClient.", e) + conn.close() + throw new Exception("Error getting first response from the SaslClient") + } + } + } + } + } else { + logDebug("Sasl already established ") + } + } + + // allow us to add messages to the inbox for doing sasl negotiating + private def sendSecurityMessage(connManagerId: ConnectionManagerId, message: Message) { + def startNewConnection(): SendingConnection = { + val inetSocketAddress = new InetSocketAddress(connManagerId.host, connManagerId.port) + val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) + val newConnection = new SendingConnection(inetSocketAddress, selector, connManagerId, + newConnectionId) + logInfo("creating new sending connection for security! " + newConnectionId ) + registerRequests.enqueue(newConnection) + + newConnection + } + // I removed the lookupKey stuff as part of merge ... should I re-add it ? + // We did not find it useful in our test-env ... + // If we do re-add it, we should consistently use it everywhere I guess ? + message.senderAddress = id.toSocketAddress() + logTrace("Sending Security [" + message + "] to [" + connManagerId + "]") + val connection = connectionsById.getOrElseUpdate(connManagerId, startNewConnection()) + + //send security message until going connection has been authenticated + connection.send(message) + + wakeupSelector() + } + private def sendMessage(connectionManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { val inetSocketAddress = new InetSocketAddress(connectionManagerId.host, connectionManagerId.port) - val newConnection = new SendingConnection(inetSocketAddress, selector, connectionManagerId) + val newConnectionId = new ConnectionId(id, idCount.getAndIncrement.intValue) + val newConnection = new SendingConnection(inetSocketAddress, selector, connectionManagerId, + newConnectionId) + logTrace("creating new sending connection: " + newConnectionId) registerRequests.enqueue(newConnection) newConnection } val connection = connectionsById.getOrElseUpdate(connectionManagerId, startNewConnection()) + if (authEnabled) { + checkSendAuthFirst(connectionManagerId, connection) + } message.senderAddress = id.toSocketAddress() + logDebug("Before Sending [" + message + "] to [" + connectionManagerId + "]" + " " + + "connectionid: " + connection.connectionId) + + if (authEnabled) { + // if we aren't authenticated yet lets block the senders until authentication completes + try { + connection.getAuthenticated().synchronized { + val clock = SystemClock + val startTime = clock.getTime() + + while (!connection.isSaslComplete()) { + logDebug("getAuthenticated wait connectionid: " + connection.connectionId) + // have timeout in case remote side never responds + connection.getAuthenticated().wait(500) + if (((clock.getTime() - startTime) >= (authTimeout * 1000)) + && (!connection.isSaslComplete())) { + // took to long to authenticate the connection, something probably went wrong + throw new Exception("Took to long for authentication to " + connectionManagerId + + ", waited " + authTimeout + "seconds, failing.") + } + } + } + } catch { + case e: Exception => logError("Exception while waiting for authentication.", e) + + // need to tell sender it failed + messageStatuses.synchronized { + val s = messageStatuses.get(message.id) + s match { + case Some(msgStatus) => { + messageStatuses -= message.id + logInfo("Notifying " + msgStatus.connectionManagerId) + msgStatus.synchronized { + msgStatus.attempted = true + msgStatus.acked = false + msgStatus.markDone() + } + } + case None => { + logError("no messageStatus for failed message id: " + message.id) + } + } + } + } + } logDebug("Sending [" + message + "] to [" + connectionManagerId + "]") connection.send(message) @@ -603,7 +852,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf) extends Loggi private[spark] object ConnectionManager { def main(args: Array[String]) { - val manager = new ConnectionManager(9999, new SparkConf) + val conf = new SparkConf + val manager = new ConnectionManager(9999, conf, new SecurityManager(conf)) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { println("Received [" + msg + "] from [" + id + "]") None diff --git a/core/src/main/scala/org/apache/spark/network/Message.scala b/core/src/main/scala/org/apache/spark/network/Message.scala index 20fe67661844f..7caccfdbb44f9 100644 --- a/core/src/main/scala/org/apache/spark/network/Message.scala +++ b/core/src/main/scala/org/apache/spark/network/Message.scala @@ -27,6 +27,7 @@ private[spark] abstract class Message(val typ: Long, val id: Int) { var started = false var startTime = -1L var finishTime = -1L + var isSecurityNeg = false def size: Int diff --git a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala index 9bcbc6141a502..ead663ede7a1c 100644 --- a/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/MessageChunkHeader.scala @@ -27,6 +27,7 @@ private[spark] class MessageChunkHeader( val totalSize: Int, val chunkSize: Int, val other: Int, + val securityNeg: Int, val address: InetSocketAddress) { lazy val buffer = { // No need to change this, at 'use' time, we do a reverse lookup of the hostname. @@ -40,6 +41,7 @@ private[spark] class MessageChunkHeader( putInt(totalSize). putInt(chunkSize). putInt(other). + putInt(securityNeg). putInt(ip.size). put(ip). putInt(port). @@ -48,12 +50,13 @@ private[spark] class MessageChunkHeader( } override def toString = "" + this.getClass.getSimpleName + ":" + id + " of type " + typ + - " and sizes " + totalSize + " / " + chunkSize + " bytes" + " and sizes " + totalSize + " / " + chunkSize + " bytes, securityNeg: " + securityNeg + } private[spark] object MessageChunkHeader { - val HEADER_SIZE = 40 + val HEADER_SIZE = 44 def create(buffer: ByteBuffer): MessageChunkHeader = { if (buffer.remaining != HEADER_SIZE) { @@ -64,11 +67,13 @@ private[spark] object MessageChunkHeader { val totalSize = buffer.getInt() val chunkSize = buffer.getInt() val other = buffer.getInt() + val securityNeg = buffer.getInt() val ipSize = buffer.getInt() val ipBytes = new Array[Byte](ipSize) buffer.get(ipBytes) val ip = InetAddress.getByAddress(ipBytes) val port = buffer.getInt() - new MessageChunkHeader(typ, id, totalSize, chunkSize, other, new InetSocketAddress(ip, port)) + new MessageChunkHeader(typ, id, totalSize, chunkSize, other, securityNeg, + new InetSocketAddress(ip, port)) } } diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 9976255c7e251..3c09a713c6fe0 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -18,12 +18,12 @@ package org.apache.spark.network import java.nio.ByteBuffer - -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} private[spark] object ReceiverTest { def main(args: Array[String]) { - val manager = new ConnectionManager(9999, new SparkConf) + val conf = new SparkConf + val manager = new ConnectionManager(9999, conf, new SecurityManager(conf)) println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala new file mode 100644 index 0000000000000..0d9f743b3624b --- /dev/null +++ b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala @@ -0,0 +1,163 @@ +/* + * 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.network + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.StringBuilder + +import org.apache.spark._ +import org.apache.spark.network._ + +/** + * SecurityMessage is class that contains the connectionId and sasl token + * used in SASL negotiation. SecurityMessage has routines for converting + * it to and from a BufferMessage so that it can be sent by the ConnectionManager + * and easily consumed by users when received. + * The api was modeled after BlockMessage. + * + * The connectionId is the connectionId of the client side. Since + * message passing is asynchronous and its possible for the server side (receiving) + * to get multiple different types of messages on the same connection the connectionId + * is used to know which connnection the security message is intended for. + * + * For instance, lets say we are node_0. We need to send data to node_1. The node_0 side + * is acting as a client and connecting to node_1. SASL negotiation has to occur + * between node_0 and node_1 before node_1 trusts node_0 so node_0 sends a security message. + * node_1 receives the message from node_0 but before it can process it and send a response, + * some thread on node_1 decides it needs to send data to node_0 so it connects to node_0 + * and sends a security message of its own to authenticate as a client. Now node_0 gets + * the message and it needs to decide if this message is in response to it being a client + * (from the first send) or if its just node_1 trying to connect to it to send data. This + * is where the connectionId field is used. node_0 can lookup the connectionId to see if + * it is in response to it being a client or if its in response to someone sending other data. + * + * The format of a SecurityMessage as its sent is: + * - Length of the ConnectionId + * - ConnectionId + * - Length of the token + * - Token + */ +private[spark] class SecurityMessage() extends Logging { + + private var connectionId: String = null + private var token: Array[Byte] = null + + def set(byteArr: Array[Byte], newconnectionId: String) { + if (byteArr == null) { + token = new Array[Byte](0) + } else { + token = byteArr + } + connectionId = newconnectionId + } + + /** + * Read the given buffer and set the members of this class. + */ + def set(buffer: ByteBuffer) { + val idLength = buffer.getInt() + val idBuilder = new StringBuilder(idLength) + for (i <- 1 to idLength) { + idBuilder += buffer.getChar() + } + connectionId = idBuilder.toString() + + val tokenLength = buffer.getInt() + token = new Array[Byte](tokenLength) + if (tokenLength > 0) { + buffer.get(token, 0, tokenLength) + } + } + + def set(bufferMsg: BufferMessage) { + val buffer = bufferMsg.buffers.apply(0) + buffer.clear() + set(buffer) + } + + def getConnectionId: String = { + return connectionId + } + + def getToken: Array[Byte] = { + return token + } + + /** + * Create a BufferMessage that can be sent by the ConnectionManager containing + * the security information from this class. + * @return BufferMessage + */ + def toBufferMessage: BufferMessage = { + val startTime = System.currentTimeMillis + val buffers = new ArrayBuffer[ByteBuffer]() + + // 4 bytes for the length of the connectionId + // connectionId is of type char so multiple the length by 2 to get number of bytes + // 4 bytes for the length of token + // token is a byte buffer so just take the length + var buffer = ByteBuffer.allocate(4 + connectionId.length() * 2 + 4 + token.length) + buffer.putInt(connectionId.length()) + connectionId.foreach((x: Char) => buffer.putChar(x)) + buffer.putInt(token.length) + + if (token.length > 0) { + buffer.put(token) + } + buffer.flip() + buffers += buffer + + var message = Message.createBufferMessage(buffers) + logDebug("message total size is : " + message.size) + message.isSecurityNeg = true + return message + } + + override def toString: String = { + "SecurityMessage [connId= " + connectionId + ", Token = " + token + "]" + } +} + +private[spark] object SecurityMessage { + + /** + * Convert the given BufferMessage to a SecurityMessage by parsing the contents + * of the BufferMessage and populating the SecurityMessage fields. + * @param bufferMessage is a BufferMessage that was received + * @return new SecurityMessage + */ + def fromBufferMessage(bufferMessage: BufferMessage): SecurityMessage = { + val newSecurityMessage = new SecurityMessage() + newSecurityMessage.set(bufferMessage) + newSecurityMessage + } + + /** + * Create a SecurityMessage to send from a given saslResponse. + * @param response is the response to a challenge from the SaslClient or Saslserver + * @param connectionId the client connectionId we are negotiation authentication for + * @return a new SecurityMessage + */ + def fromResponse(response : Array[Byte], connectionId : String) : SecurityMessage = { + val newSecurityMessage = new SecurityMessage() + newSecurityMessage.set(response, connectionId) + newSecurityMessage + } +} diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 646f8425d9551..aac2c24a46faa 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -18,8 +18,7 @@ package org.apache.spark.network import java.nio.ByteBuffer - -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} private[spark] object SenderTest { def main(args: Array[String]) { @@ -32,8 +31,8 @@ private[spark] object SenderTest { val targetHost = args(0) val targetPort = args(1).toInt val targetConnectionManagerId = new ConnectionManagerId(targetHost, targetPort) - - val manager = new ConnectionManager(0, new SparkConf) + val conf = new SparkConf + val manager = new ConnectionManager(0, conf, new SecurityManager(conf)) println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 977c24687cc5f..1bf3f4db32ea7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,7 +29,7 @@ import akka.actor.{ActorSystem, Cancellable, Props} import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer -import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} +import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException, SecurityManager} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -47,7 +47,8 @@ private[spark] class BlockManager( val master: BlockManagerMaster, val defaultSerializer: Serializer, maxMemory: Long, - val conf: SparkConf) + val conf: SparkConf, + securityManager: SecurityManager) extends Logging { val shuffleBlockManager = new ShuffleBlockManager(this) @@ -66,7 +67,7 @@ private[spark] class BlockManager( if (useNetty) diskBlockManager.startShuffleBlockSender(nettyPortConfig) else 0 } - val connectionManager = new ConnectionManager(0, conf) + val connectionManager = new ConnectionManager(0, conf, securityManager) implicit val futureExecContext = connectionManager.futureExecContext val blockManagerId = BlockManagerId( @@ -122,8 +123,9 @@ private[spark] class BlockManager( * Construct a BlockManager with a memory limit set based on system properties. */ def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, conf: SparkConf) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf) + serializer: Serializer, conf: SparkConf, securityManager: SecurityManager) = { + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf, + securityManager) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 1d81d006c0b29..36f2a0fd02724 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -24,6 +24,7 @@ import util.Random import org.apache.spark.SparkConf import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.{SecurityManager, SparkConf} /** * This class tests the BlockManager and MemoryStore for thread safety and @@ -98,7 +99,8 @@ private[spark] object ThreadingTest { val blockManagerMaster = new BlockManagerMaster( actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) val blockManager = new BlockManager( - "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf) + "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, + new SecurityManager(conf)) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 1b78c52ff6077..7c35cd165ad7c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -18,7 +18,8 @@ package org.apache.spark.ui import java.net.InetSocketAddress -import javax.servlet.http.{HttpServletResponse, HttpServletRequest} +import java.net.URL +import javax.servlet.http.{HttpServlet, HttpServletResponse, HttpServletRequest} import scala.annotation.tailrec import scala.util.{Failure, Success, Try} @@ -26,11 +27,14 @@ import scala.xml.Node import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} -import org.eclipse.jetty.server.{Handler, Request, Server} -import org.eclipse.jetty.server.handler.{AbstractHandler, ContextHandler, HandlerList, ResourceHandler} + +import org.eclipse.jetty.server.{DispatcherType, Server} +import org.eclipse.jetty.server.handler.HandlerList +import org.eclipse.jetty.servlet.{DefaultServlet, FilterHolder, ServletContextHandler, ServletHolder} import org.eclipse.jetty.util.thread.QueuedThreadPool -import org.apache.spark.Logging +import org.apache.spark.{Logging, SecurityManager, SparkConf} + /** Utilities for launching a web server using Jetty's HTTP Server class */ private[spark] object JettyUtils extends Logging { @@ -39,57 +43,104 @@ private[spark] object JettyUtils extends Logging { type Responder[T] = HttpServletRequest => T - // Conversions from various types of Responder's to jetty Handlers - implicit def jsonResponderToHandler(responder: Responder[JValue]): Handler = - createHandler(responder, "text/json", (in: JValue) => pretty(render(in))) + class ServletParams[T <% AnyRef](val responder: Responder[T], + val contentType: String, + val extractFn: T => String = (in: Any) => in.toString) {} + + // Conversions from various types of Responder's to appropriate servlet parameters + implicit def jsonResponderToServlet(responder: Responder[JValue]): ServletParams[JValue] = + new ServletParams(responder, "text/json", (in: JValue) => pretty(render(in))) - implicit def htmlResponderToHandler(responder: Responder[Seq[Node]]): Handler = - createHandler(responder, "text/html", (in: Seq[Node]) => "" + in.toString) + implicit def htmlResponderToServlet(responder: Responder[Seq[Node]]): ServletParams[Seq[Node]] = + new ServletParams(responder, "text/html", (in: Seq[Node]) => "" + in.toString) - implicit def textResponderToHandler(responder: Responder[String]): Handler = - createHandler(responder, "text/plain") + implicit def textResponderToServlet(responder: Responder[String]): ServletParams[String] = + new ServletParams(responder, "text/plain") - def createHandler[T <% AnyRef](responder: Responder[T], contentType: String, - extractFn: T => String = (in: Any) => in.toString): Handler = { - new AbstractHandler { - def handle(target: String, - baseRequest: Request, - request: HttpServletRequest, + def createServlet[T <% AnyRef](servletParams: ServletParams[T], + securityMgr: SecurityManager): HttpServlet = { + new HttpServlet { + override def doGet(request: HttpServletRequest, response: HttpServletResponse) { - response.setContentType("%s;charset=utf-8".format(contentType)) - response.setStatus(HttpServletResponse.SC_OK) - baseRequest.setHandled(true) - val result = responder(request) - response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") - response.getWriter().println(extractFn(result)) + if (securityMgr.checkUIViewPermissions(request.getRemoteUser())) { + response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) + response.setStatus(HttpServletResponse.SC_OK) + val result = servletParams.responder(request) + response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") + response.getWriter().println(servletParams.extractFn(result)) + } else { + response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) + response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") + response.sendError(HttpServletResponse.SC_UNAUTHORIZED, + "User is not authorized to access this page."); + } } } } + def createServletHandler(path: String, servlet: HttpServlet): ServletContextHandler = { + val contextHandler = new ServletContextHandler() + val holder = new ServletHolder(servlet) + contextHandler.setContextPath(path) + contextHandler.addServlet(holder, "/") + contextHandler + } + /** Creates a handler that always redirects the user to a given path */ - def createRedirectHandler(newPath: String): Handler = { - new AbstractHandler { - def handle(target: String, - baseRequest: Request, - request: HttpServletRequest, + def createRedirectHandler(newPath: String, path: String): ServletContextHandler = { + val servlet = new HttpServlet { + override def doGet(request: HttpServletRequest, response: HttpServletResponse) { - response.setStatus(302) - response.setHeader("Location", baseRequest.getRootURL + newPath) - baseRequest.setHandled(true) + // make sure we don't end up with // in the middle + val newUri = new URL(new URL(request.getRequestURL.toString), newPath).toURI + response.sendRedirect(newUri.toString) } } + val contextHandler = new ServletContextHandler() + val holder = new ServletHolder(servlet) + contextHandler.setContextPath(path) + contextHandler.addServlet(holder, "/") + contextHandler } /** Creates a handler for serving files from a static directory */ - def createStaticHandler(resourceBase: String): ResourceHandler = { - val staticHandler = new ResourceHandler + def createStaticHandler(resourceBase: String, path: String): ServletContextHandler = { + val contextHandler = new ServletContextHandler() + val staticHandler = new DefaultServlet + val holder = new ServletHolder(staticHandler) Option(getClass.getClassLoader.getResource(resourceBase)) match { case Some(res) => - staticHandler.setResourceBase(res.toString) + holder.setInitParameter("resourceBase", res.toString) case None => throw new Exception("Could not find resource path for Web UI: " + resourceBase) } - staticHandler + contextHandler.addServlet(holder, path) + contextHandler + } + + private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { + val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) + filters.foreach { + case filter : String => + if (!filter.isEmpty) { + logInfo("Adding filter: " + filter) + val holder : FilterHolder = new FilterHolder() + holder.setClassName(filter) + // get any parameters for each filter + val paramName = "spark." + filter + ".params" + val params = conf.get(paramName, "").split(',').map(_.trim()).toSet + params.foreach { + case param : String => + if (!param.isEmpty) { + val parts = param.split("=") + if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) + } + } + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, + DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) + handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } + } + } } /** @@ -99,17 +150,12 @@ private[spark] object JettyUtils extends Logging { * If the desired port number is contented, continues incrementing ports until a free port is * found. Returns the chosen port and the jetty Server object. */ - def startJettyServer(hostName: String, port: Int, handlers: Seq[(String, Handler)]): (Server, Int) - = { - - val handlersToRegister = handlers.map { case(path, handler) => - val contextHandler = new ContextHandler(path) - contextHandler.setHandler(handler) - contextHandler.asInstanceOf[org.eclipse.jetty.server.Handler] - } + def startJettyServer(hostName: String, port: Int, handlers: Seq[ServletContextHandler], + conf: SparkConf): (Server, Int) = { + addFilters(handlers, conf) val handlerList = new HandlerList - handlerList.setHandlers(handlersToRegister.toArray) + handlerList.setHandlers(handlers.toArray) @tailrec def connect(currentPort: Int): (Server, Int) = { @@ -119,7 +165,9 @@ private[spark] object JettyUtils extends Logging { server.setThreadPool(pool) server.setHandler(handlerList) - Try { server.start() } match { + Try { + server.start() + } match { case s: Success[_] => (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index af6b65860e006..ca82c3da2fc24 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,7 +17,10 @@ package org.apache.spark.ui -import org.eclipse.jetty.server.{Handler, Server} +import javax.servlet.http.HttpServletRequest + +import org.eclipse.jetty.server.Server +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SparkContext, SparkEnv} import org.apache.spark.ui.JettyUtils._ @@ -34,9 +37,9 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { var boundPort: Option[Int] = None var server: Option[Server] = None - val handlers = Seq[(String, Handler)]( - ("/static", createStaticHandler(SparkUI.STATIC_RESOURCE_DIR)), - ("/", createRedirectHandler("/stages")) + val handlers = Seq[ServletContextHandler] ( + createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static/*"), + createRedirectHandler("/stages", "/") ) val storage = new BlockManagerUI(sc) val jobs = new JobProgressUI(sc) @@ -52,7 +55,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { /** Bind the HTTP server which backs this web interface */ def bind() { try { - val (srv, usedPort) = JettyUtils.startJettyServer(host, port, allHandlers) + val (srv, usedPort) = JettyUtils.startJettyServer(host, port, allHandlers, sc.conf) logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) server = Some(srv) boundPort = Some(usedPort) @@ -83,5 +86,5 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { private[spark] object SparkUI { val DEFAULT_PORT = "4040" - val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + val STATIC_RESOURCE_DIR = "org/apache/spark/ui" } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 9e7cdc88162e8..14333476c0e31 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -23,7 +23,7 @@ import scala.collection.JavaConversions._ import scala.util.Properties import scala.xml.Node -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SparkContext import org.apache.spark.ui.JettyUtils._ @@ -32,8 +32,9 @@ import org.apache.spark.ui.UIUtils private[spark] class EnvironmentUI(sc: SparkContext) { - def getHandlers = Seq[(String, Handler)]( - ("/environment", (request: HttpServletRequest) => envDetails(request)) + def getHandlers = Seq[ServletContextHandler]( + createServletHandler("/environment", + createServlet((request: HttpServletRequest) => envDetails(request), sc.env.securityManager)) ) def envDetails(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 1f3b7a4c231b6..4235cfeff9fa2 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -22,7 +22,7 @@ import javax.servlet.http.HttpServletRequest import scala.collection.mutable.{HashMap, HashSet} import scala.xml.Node -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{ExceptionFailure, Logging, SparkContext} import org.apache.spark.executor.TaskMetrics @@ -43,8 +43,9 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { sc.addSparkListener(listener) } - def getHandlers = Seq[(String, Handler)]( - ("/executors", (request: HttpServletRequest) => render(request)) + def getHandlers = Seq[ServletContextHandler]( + createServletHandler("/executors", createServlet((request: HttpServletRequest) => render + (request), sc.env.securityManager)) ) def render(request: HttpServletRequest): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 557bce6b66353..2d95d47e154cd 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -23,6 +23,7 @@ import javax.servlet.http.HttpServletRequest import scala.Seq import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SparkContext import org.apache.spark.ui.JettyUtils._ @@ -45,9 +46,15 @@ private[spark] class JobProgressUI(val sc: SparkContext) { def formatDuration(ms: Long) = Utils.msDurationToString(ms) - def getHandlers = Seq[(String, Handler)]( - ("/stages/stage", (request: HttpServletRequest) => stagePage.render(request)), - ("/stages/pool", (request: HttpServletRequest) => poolPage.render(request)), - ("/stages", (request: HttpServletRequest) => indexPage.render(request)) + def getHandlers = Seq[ServletContextHandler]( + createServletHandler("/stages/stage", + createServlet((request: HttpServletRequest) => stagePage.render(request), + sc.env.securityManager)), + createServletHandler("/stages/pool", + createServlet((request: HttpServletRequest) => poolPage.render(request), + sc.env.securityManager)), + createServletHandler("/stages", + createServlet((request: HttpServletRequest) => indexPage.render(request), + sc.env.securityManager)) ) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index dc18eab74e0da..cb2083eb019bf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -19,7 +19,7 @@ package org.apache.spark.ui.storage import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Handler +import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SparkContext} import org.apache.spark.ui.JettyUtils._ @@ -29,8 +29,12 @@ private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { val indexPage = new IndexPage(this) val rddPage = new RDDPage(this) - def getHandlers = Seq[(String, Handler)]( - ("/storage/rdd", (request: HttpServletRequest) => rddPage.render(request)), - ("/storage", (request: HttpServletRequest) => indexPage.render(request)) + def getHandlers = Seq[ServletContextHandler]( + createServletHandler("/storage/rdd", + createServlet((request: HttpServletRequest) => rddPage.render(request), + sc.env.securityManager)), + createServletHandler("/storage", + createServlet((request: HttpServletRequest) => indexPage.render(request), + sc.env.securityManager)) ) } diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index f26ed47e58046..a6c9a9aaba8eb 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -24,12 +24,12 @@ import akka.actor.{ActorSystem, ExtendedActorSystem, IndestructibleActorSystem} import com.typesafe.config.ConfigFactory import org.apache.log4j.{Level, Logger} -import org.apache.spark.SparkConf +import org.apache.spark.{Logging, SecurityManager, SparkConf} /** * Various utility classes for working with Akka. */ -private[spark] object AkkaUtils { +private[spark] object AkkaUtils extends Logging { /** * Creates an ActorSystem ready for remoting, with various Spark features. Returns both the @@ -42,7 +42,7 @@ private[spark] object AkkaUtils { * of a fatal exception. This is used by [[org.apache.spark.executor.Executor]]. */ def createActorSystem(name: String, host: String, port: Int, indestructible: Boolean = false, - conf: SparkConf): (ActorSystem, Int) = { + conf: SparkConf, securityManager: SecurityManager): (ActorSystem, Int) = { val akkaThreads = conf.getInt("spark.akka.threads", 4) val akkaBatchSize = conf.getInt("spark.akka.batchSize", 15) @@ -65,6 +65,15 @@ private[spark] object AkkaUtils { conf.getDouble("spark.akka.failure-detector.threshold", 300.0) val akkaHeartBeatInterval = conf.getInt("spark.akka.heartbeat.interval", 1000) + val secretKey = securityManager.getSecretKey() + val isAuthOn = securityManager.isAuthenticationEnabled() + if (isAuthOn && secretKey == null) { + throw new Exception("Secret key is null with authentication on") + } + val requireCookie = if (isAuthOn) "on" else "off" + val secureCookie = if (isAuthOn) secretKey else "" + logDebug("In createActorSystem, requireCookie is: " + requireCookie) + val akkaConf = ConfigFactory.parseMap(conf.getAkkaConf.toMap[String, String]).withFallback( ConfigFactory.parseString( s""" @@ -72,6 +81,8 @@ private[spark] object AkkaUtils { |akka.loggers = [""akka.event.slf4j.Slf4jLogger""] |akka.stdout-loglevel = "ERROR" |akka.jvm-exit-on-fatal-error = off + |akka.remote.require-cookie = "$requireCookie" + |akka.remote.secure-cookie = "$secureCookie" |akka.remote.transport-failure-detector.heartbeat-interval = $akkaHeartBeatInterval s |akka.remote.transport-failure-detector.acceptable-heartbeat-pause = $akkaHeartBeatPauses s |akka.remote.transport-failure-detector.threshold = $akkaFailureDetector diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8e69f1d3351b5..0eb2f78b730f6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -18,7 +18,7 @@ package org.apache.spark.util import java.io._ -import java.net.{InetAddress, Inet4Address, NetworkInterface, URI, URL} +import java.net.{InetAddress, Inet4Address, NetworkInterface, URI, URL, URLConnection} import java.nio.ByteBuffer import java.util.{Locale, Random, UUID} import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} @@ -33,10 +33,11 @@ import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} -import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil + /** * Various utility methods used by Spark. */ @@ -232,6 +233,22 @@ private[spark] object Utils extends Logging { } } + /** + * Construct a URI container information used for authentication. + * This also sets the default authenticator to properly negotiation the + * user/password based on the URI. + * + * Note this relies on the Authenticator.setDefault being set properly to decode + * the user name and password. This is currently set in the SecurityManager. + */ + def constructURIForAuthentication(uri: URI, securityMgr: SecurityManager): URI = { + val userCred = securityMgr.getSecretKey() + if (userCred == null) throw new Exception("Secret key is null with authentication on") + val userInfo = securityMgr.getHttpUser() + ":" + userCred + new URI(uri.getScheme(), userInfo, uri.getHost(), uri.getPort(), uri.getPath(), + uri.getQuery(), uri.getFragment()) + } + /** * Download a file requested by the executor. Supports fetching the file in a variety of ways, * including HTTP, HDFS and files on a standard filesystem, based on the URL parameter. @@ -239,7 +256,7 @@ private[spark] object Utils extends Logging { * Throws SparkException if the target file already exists and has different contents than * the requested file. */ - def fetchFile(url: String, targetDir: File, conf: SparkConf) { + def fetchFile(url: String, targetDir: File, conf: SparkConf, securityMgr: SecurityManager) { val filename = url.split("/").last val tempDir = getLocalDir(conf) val tempFile = File.createTempFile("fetchFileTemp", null, new File(tempDir)) @@ -249,7 +266,19 @@ private[spark] object Utils extends Logging { uri.getScheme match { case "http" | "https" | "ftp" => logInfo("Fetching " + url + " to " + tempFile) - val in = new URL(url).openStream() + + var uc: URLConnection = null + if (securityMgr.isAuthenticationEnabled()) { + logDebug("fetchFile with security enabled") + val newuri = constructURIForAuthentication(uri, securityMgr) + uc = newuri.toURL().openConnection() + uc.setAllowUserInteraction(false) + } else { + logDebug("fetchFile not using security") + uc = new URL(url).openConnection() + } + + val in = uc.getInputStream(); val out = new FileOutputStream(tempFile) Utils.copyStream(in, out, true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala new file mode 100644 index 0000000000000..cd054c1f684ab --- /dev/null +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -0,0 +1,215 @@ +/* + * 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 + +import org.scalatest.FunSuite + +import akka.actor._ +import org.apache.spark.scheduler.MapStatus +import org.apache.spark.storage.BlockManagerId +import org.apache.spark.util.AkkaUtils +import scala.concurrent.Await + +/** + * Test the AkkaUtils with various security settings. + */ +class AkkaUtilsSuite extends FunSuite with LocalSparkContext { + + test("remote fetch security bad password") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + + val securityManager = new SecurityManager(conf); + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + assert(securityManager.isAuthenticationEnabled() === true) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + + val badconf = new SparkConf + badconf.set("spark.authenticate", "true") + badconf.set("spark.authenticate.secret", "bad") + val securityManagerBad = new SecurityManager(badconf); + + assert(securityManagerBad.isAuthenticationEnabled() === true) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = conf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTracker(conf) + val selection = slaveSystem.actorSelection( + s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + val timeout = AkkaUtils.lookupTimeout(conf) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + } + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + test("remote fetch security off") { + val conf = new SparkConf + conf.set("spark.authenticate", "false") + conf.set("spark.authenticate.secret", "bad") + val securityManager = new SecurityManager(conf); + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === false) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + + val badconf = new SparkConf + badconf.set("spark.authenticate", "false") + badconf.set("spark.authenticate.secret", "good") + val securityManagerBad = new SecurityManager(badconf); + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = badconf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTracker(conf) + val selection = slaveSystem.actorSelection( + s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + val timeout = AkkaUtils.lookupTimeout(conf) + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + + assert(securityManagerBad.isAuthenticationEnabled() === false) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val size1000 = MapOutputTracker.decompressSize(compressedSize1000) + masterTracker.registerMapOutput(10, 0, new MapStatus( + BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000))) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + // this should succeed since security off + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000, 0), size1000))) + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + test("remote fetch security pass") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + val securityManager = new SecurityManager(conf); + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === true) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + + val goodconf = new SparkConf + goodconf.set("spark.authenticate", "true") + goodconf.set("spark.authenticate.secret", "good") + val securityManagerGood = new SecurityManager(goodconf); + + assert(securityManagerGood.isAuthenticationEnabled() === true) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = goodconf, securityManager = securityManagerGood) + val slaveTracker = new MapOutputTracker(conf) + val selection = slaveSystem.actorSelection( + s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + val timeout = AkkaUtils.lookupTimeout(conf) + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + + masterTracker.registerShuffle(10, 1) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val size1000 = MapOutputTracker.decompressSize(compressedSize1000) + masterTracker.registerMapOutput(10, 0, new MapStatus( + BlockManagerId("a", "hostA", 1000, 0), Array(compressedSize1000))) + masterTracker.incrementEpoch() + slaveTracker.updateEpoch(masterTracker.getEpoch) + + // this should succeed since security on and passwords match + assert(slaveTracker.getServerStatuses(10, 0).toSeq === + Seq((BlockManagerId("a", "hostA", 1000, 0), size1000))) + + actorSystem.shutdown() + slaveSystem.shutdown() + } + + test("remote fetch security off client") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + + val securityManager = new SecurityManager(conf); + + val hostname = "localhost" + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + conf = conf, securityManager = securityManager) + System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext + System.setProperty("spark.hostPort", hostname + ":" + boundPort) + + assert(securityManager.isAuthenticationEnabled() === true) + + val masterTracker = new MapOutputTrackerMaster(conf) + masterTracker.trackerActor = actorSystem.actorOf( + Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + + val badconf = new SparkConf + badconf.set("spark.authenticate", "false") + badconf.set("spark.authenticate.secret", "bad") + val securityManagerBad = new SecurityManager(badconf); + + assert(securityManagerBad.isAuthenticationEnabled() === false) + + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + conf = badconf, securityManager = securityManagerBad) + val slaveTracker = new MapOutputTracker(conf) + val selection = slaveSystem.actorSelection( + s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") + val timeout = AkkaUtils.lookupTimeout(conf) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + } + + actorSystem.shutdown() + slaveSystem.shutdown() + } + +} diff --git a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala index e022accee6d08..96ba3929c1685 100644 --- a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala @@ -21,6 +21,7 @@ import org.scalatest.FunSuite class BroadcastSuite extends FunSuite with LocalSparkContext { + override def afterEach() { super.afterEach() System.clearProperty("spark.broadcast.factory") diff --git a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala new file mode 100644 index 0000000000000..80f7ec00c74b2 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala @@ -0,0 +1,230 @@ +/* + * 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 + +import org.scalatest.FunSuite + +import java.nio._ + +import org.apache.spark.network.{ConnectionManager, Message, ConnectionManagerId} +import scala.concurrent.Await +import scala.concurrent.TimeoutException +import scala.concurrent.duration._ + + +/** + * Test the ConnectionManager with various security settings. + */ +class ConnectionManagerSuite extends FunSuite { + + test("security default off") { + val conf = new SparkConf + val securityManager = new SecurityManager(conf) + val manager = new ConnectionManager(0, conf, securityManager) + var receivedMessage = false + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + receivedMessage = true + None + }) + + val size = 10 * 1024 * 1024 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + manager.sendMessageReliablySync(manager.id, bufferMessage) + + assert(receivedMessage == true) + + manager.stop() + } + + test("security on same password") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + val securityManager = new SecurityManager(conf) + val manager = new ConnectionManager(0, conf, securityManager) + var numReceivedMessages = 0 + + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + numReceivedMessages += 1 + None + }) + val managerServer = new ConnectionManager(0, conf, securityManager) + var numReceivedServerMessages = 0 + managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + numReceivedServerMessages += 1 + None + }) + + val size = 10 * 1024 * 1024 + val count = 10 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + + (0 until count).map(i => { + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + manager.sendMessageReliablySync(managerServer.id, bufferMessage) + }) + + assert(numReceivedServerMessages == 10) + assert(numReceivedMessages == 0) + + manager.stop() + managerServer.stop() + } + + test("security mismatch password") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + val securityManager = new SecurityManager(conf) + val manager = new ConnectionManager(0, conf, securityManager) + var numReceivedMessages = 0 + + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + numReceivedMessages += 1 + None + }) + + val badconf = new SparkConf + badconf.set("spark.authenticate", "true") + badconf.set("spark.authenticate.secret", "bad") + val badsecurityManager = new SecurityManager(badconf) + val managerServer = new ConnectionManager(0, badconf, badsecurityManager) + var numReceivedServerMessages = 0 + + managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + numReceivedServerMessages += 1 + None + }) + + val size = 10 * 1024 * 1024 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + manager.sendMessageReliablySync(managerServer.id, bufferMessage) + + assert(numReceivedServerMessages == 0) + assert(numReceivedMessages == 0) + + manager.stop() + managerServer.stop() + } + + test("security mismatch auth off") { + val conf = new SparkConf + conf.set("spark.authenticate", "false") + conf.set("spark.authenticate.secret", "good") + val securityManager = new SecurityManager(conf) + val manager = new ConnectionManager(0, conf, securityManager) + var numReceivedMessages = 0 + + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + numReceivedMessages += 1 + None + }) + + val badconf = new SparkConf + badconf.set("spark.authenticate", "true") + badconf.set("spark.authenticate.secret", "good") + val badsecurityManager = new SecurityManager(badconf) + val managerServer = new ConnectionManager(0, badconf, badsecurityManager) + var numReceivedServerMessages = 0 + managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + numReceivedServerMessages += 1 + None + }) + + val size = 10 * 1024 * 1024 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + (0 until 1).map(i => { + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + manager.sendMessageReliably(managerServer.id, bufferMessage) + }).foreach(f => { + try { + val g = Await.result(f, 1 second) + assert(false) + } catch { + case e: TimeoutException => { + // we should timeout here since the client can't do the negotiation + assert(true) + } + } + }) + + assert(numReceivedServerMessages == 0) + assert(numReceivedMessages == 0) + manager.stop() + managerServer.stop() + } + + test("security auth off") { + val conf = new SparkConf + conf.set("spark.authenticate", "false") + val securityManager = new SecurityManager(conf) + val manager = new ConnectionManager(0, conf, securityManager) + var numReceivedMessages = 0 + + manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + numReceivedMessages += 1 + None + }) + + val badconf = new SparkConf + badconf.set("spark.authenticate", "false") + val badsecurityManager = new SecurityManager(badconf) + val managerServer = new ConnectionManager(0, badconf, badsecurityManager) + var numReceivedServerMessages = 0 + + managerServer.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + numReceivedServerMessages += 1 + None + }) + + val size = 10 * 1024 * 1024 + val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) + buffer.flip + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + (0 until 10).map(i => { + val bufferMessage = Message.createBufferMessage(buffer.duplicate) + manager.sendMessageReliably(managerServer.id, bufferMessage) + }).foreach(f => { + try { + val g = Await.result(f, 1 second) + if (!g.isDefined) assert(false) else assert(true) + } catch { + case e: Exception => { + assert(false) + } + } + }) + assert(numReceivedServerMessages == 10) + assert(numReceivedMessages == 0) + + manager.stop() + managerServer.stop() + } + + + +} + diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index e0e8011278649..9cbdfc54a3dc8 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -30,6 +30,7 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.Utils class DriverSuite extends FunSuite with Timeouts { + test("driver should exit after finishing") { val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get // Regression test for SPARK-530: "Spark driver process doesn't exit after finishing" diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index 9be67b3c95abd..aee9ab9091dac 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -30,6 +30,12 @@ class FileServerSuite extends FunSuite with LocalSparkContext { @transient var tmpFile: File = _ @transient var tmpJarUrl: String = _ + override def beforeEach() { + super.beforeEach() + resetSparkContext() + System.setProperty("spark.authenticate", "false") + } + override def beforeAll() { super.beforeAll() val tmpDir = new File(Files.createTempDir(), "test") @@ -43,6 +49,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val jarFile = new File(tmpDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) + System.setProperty("spark.authenticate", "false") val jarEntry = new JarEntry(textFile.getName) jar.putNextEntry(jarEntry) @@ -77,6 +84,25 @@ class FileServerSuite extends FunSuite with LocalSparkContext { assert(result.toSet === Set((1,200), (2,300), (3,500))) } + test("Distributing files locally security On") { + val sparkConf = new SparkConf(false) + sparkConf.set("spark.authenticate", "true") + sparkConf.set("spark.authenticate.secret", "good") + sc = new SparkContext("local[4]", "test", sparkConf) + + sc.addFile(tmpFile.toString) + assert(sc.env.securityManager.isAuthenticationEnabled() === true) + val testData = Array((1,1), (1,1), (2,1), (3,5), (2,2), (3,0)) + val result = sc.parallelize(testData).reduceByKey { + val path = SparkFiles.get("FileServerSuite.txt") + val in = new BufferedReader(new FileReader(path)) + val fileVal = in.readLine().toInt + in.close() + _ * fileVal + _ * fileVal + }.collect() + assert(result.toSet === Set((1,200), (2,300), (3,500))) + } + test("Distributing files locally using URL as input") { // addFile("file:///....") sc = new SparkContext("local[4]", "test") diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 6c1e325f6f348..8efa072a97911 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -98,14 +98,16 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("remote fetch") { val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, + securityManager = new SecurityManager(conf)) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf) + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, + securityManager = new SecurityManager(conf)) val slaveTracker = new MapOutputTracker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") diff --git a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala index c1e8b295dfe3b..96a5a1231813e 100644 --- a/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala +++ b/core/src/test/scala/org/apache/spark/metrics/MetricsSystemSuite.scala @@ -18,21 +18,22 @@ package org.apache.spark.metrics import org.scalatest.{BeforeAndAfter, FunSuite} - -import org.apache.spark.SparkConf +import org.apache.spark.{SecurityManager, SparkConf} import org.apache.spark.deploy.master.MasterSource class MetricsSystemSuite extends FunSuite with BeforeAndAfter { var filePath: String = _ var conf: SparkConf = null + var securityMgr: SecurityManager = null before { filePath = getClass.getClassLoader.getResource("test_metrics_system.properties").getFile() conf = new SparkConf(false).set("spark.metrics.conf", filePath) + securityMgr = new SecurityManager(conf) } test("MetricsSystem with default config") { - val metricsSystem = MetricsSystem.createMetricsSystem("default", conf) + val metricsSystem = MetricsSystem.createMetricsSystem("default", conf, securityMgr) val sources = metricsSystem.sources val sinks = metricsSystem.sinks @@ -42,7 +43,7 @@ class MetricsSystemSuite extends FunSuite with BeforeAndAfter { } test("MetricsSystem with sources add") { - val metricsSystem = MetricsSystem.createMetricsSystem("test", conf) + val metricsSystem = MetricsSystem.createMetricsSystem("test", conf, securityMgr) val sources = metricsSystem.sources val sinks = metricsSystem.sinks diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 9f011d9c8d132..121e47c7b1b41 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf, SparkContext} import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} @@ -39,6 +39,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT var actorSystem: ActorSystem = null var master: BlockManagerMaster = null var oldArch: String = null + conf.set("spark.authenticate", "false") + val securityMgr = new SecurityManager(conf) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") @@ -49,7 +51,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT def rdd(rddId: Int, splitId: Int) = RDDBlockId(rddId, splitId) before { - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("test", "localhost", 0, conf = conf, + securityManager = securityMgr) this.actorSystem = actorSystem conf.set("spark.driver.port", boundPort.toString) @@ -125,7 +128,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -155,8 +158,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) - store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) + store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, + securityMgr) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -171,7 +175,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -219,7 +223,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -253,7 +257,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -269,7 +273,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("reregistration on block update") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -288,7 +292,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -325,7 +329,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -344,7 +348,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -363,7 +367,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -382,7 +386,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) @@ -405,7 +409,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -418,7 +422,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -433,7 +437,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -448,7 +452,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -463,7 +467,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -478,7 +482,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -503,7 +507,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -527,7 +531,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -573,7 +577,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500, conf) + store = new BlockManager("", actorSystem, master, serializer, 500, conf, securityMgr) store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) @@ -584,7 +588,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") @@ -592,7 +596,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, securityMgr) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, "shuffle_0_0_0 was compressed") @@ -600,7 +604,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, securityMgr) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, "broadcast_0 was not compressed") @@ -608,28 +612,28 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, securityMgr) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") store.stop() store = null conf.set("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, securityMgr) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") store.stop() store = null conf.set("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, securityMgr) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf) + store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, securityMgr) store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") store.stop() @@ -643,7 +647,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block store put failure") { // Use Java serializer so we can create an unserializable error. - store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf) + store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, + securityMgr) // The put should fail since a1 is not serializable. class UnserializableClass diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 20ebb1897e6ba..30415814adbba 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -24,6 +24,8 @@ import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server import org.scalatest.FunSuite +import org.apache.spark.SparkConf + class UISuite extends FunSuite { test("jetty port increases under contention") { val startPort = 4040 @@ -34,15 +36,17 @@ class UISuite extends FunSuite { case Failure(e) => // Either case server port is busy hence setup for test complete } - val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq()) - val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq()) + val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq(), + new SparkConf) + val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq(), + new SparkConf) // Allow some wiggle room in case ports on the machine are under contention assert(boundPort1 > startPort && boundPort1 < startPort + 10) assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10) } test("jetty binds to port 0 correctly") { - val (jettyServer, boundPort) = JettyUtils.startJettyServer("0.0.0.0", 0, Seq()) + val (jettyServer, boundPort) = JettyUtils.startJettyServer("0.0.0.0", 0, Seq(), new SparkConf) assert(jettyServer.getState === "STARTED") assert(boundPort != 0) Try {new ServerSocket(boundPort)} match { diff --git a/docs/configuration.md b/docs/configuration.md index 017d509854b42..913c653b0dac4 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -147,6 +147,34 @@ Apart from these, the following properties are also available, and may be useful How many stages the Spark UI remembers before garbage collecting. + + + + + + + + + + + + + + + @@ -495,6 +523,29 @@ Apart from these, the following properties are also available, and may be useful + + + + + + + + + + + + + +
    ClassFunction Type
    spark.scheduler.revive.interval1000 + The interval length for the scheduler to revive the worker resource offers to run tasks. (in milliseconds) +
    spark.reducer.maxMbInFlight 48
    spark.serializer.objectStreamReset10000 + When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches + objects to prevent writing redundant data, however that stops garbage collection of those + objects. By calling 'reset' you flush that info from the serializer, and allow old + objects to be collected. To turn off this periodic reset set it to a value of <= 0. + By default it will reset the serializer every 10,000 objects. +
    spark.broadcast.factory org.apache.spark.broadcast.
    HttpBroadcastFactory
    spark.ui.filtersNone + Comma separated list of filter class names to apply to the Spark web ui. The filter should be a + standard javax servlet Filter. Parameters to each filter can also be specified by setting a + java system property of spark..params='param1=value1,param2=value2' + (e.g.-Dspark.ui.filters=com.test.filter1 -Dspark.com.test.filter1.params='param1=foo,param2=testing') +
    spark.ui.acls.enablefalse + Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has + access permissions to view the web ui. See spark.ui.view.acls for more details. + Also note this requires the user to be known, if the user comes across as null no checks + are done. Filters can be used to authenticate and set the user. +
    spark.ui.view.aclsEmpty + Comma separated list of users that have view access to the spark web ui. By default only the + user that started the Spark job has view access. +
    spark.shuffle.compress true Whether to overwrite files added through SparkContext.addFile() when the target file exists and its contents do not match those of the source.
    spark.authenticatefalse + Whether spark authenticates its internal connections. See spark.authenticate.secret if not + running on Yarn. +
    spark.authenticate.secretNone + Set the secret key used for Spark to authenticate between components. This needs to be set if + not running on Yarn and authentication is enabled. +
    spark.core.connection.auth.wait.timeout30 + Number of seconds for the connection to wait for authentication to occur before timing + out and giving up. +
    diff --git a/docs/index.md b/docs/index.md index 4eb297df39144..c4f4d79edbc6c 100644 --- a/docs/index.md +++ b/docs/index.md @@ -103,6 +103,7 @@ For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to bui * [Configuration](configuration.html): customize Spark via its configuration system * [Tuning Guide](tuning.html): best practices to optimize performance and memory use +* [Security](security.html): Spark security support * [Hardware Provisioning](hardware-provisioning.html): recommendations for cluster hardware * [Job Scheduling](job-scheduling.html): scheduling resources across and within Spark applications * [Building Spark with Maven](building-with-maven.html): build Spark using the Maven system diff --git a/docs/security.md b/docs/security.md new file mode 100644 index 0000000000000..9e4218fbcfe7d --- /dev/null +++ b/docs/security.md @@ -0,0 +1,18 @@ +--- +layout: global +title: Spark Security +--- + +Spark currently supports authentication via a shared secret. Authentication can be configured to be on via the `spark.authenticate` configuration parameter. This parameter controls whether the Spark communication protocols do authentication using the shared secret. This authentication is a basic handshake to make sure both sides have the same shared secret and are allowed to communicate. If the shared secret is not identical they will not be allowed to communicate. + +The Spark UI can also be secured by using javax servlet filters. A user may want to secure the UI if it has data that other users should not be allowed to see. The javax servlet filter specified by the user can authenticate the user and then once the user is logged in, Spark can compare that user versus the view acls to make sure they are authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' control the behavior of the acls. Note that the person who started the application always has view access to the UI. + +For Spark on Yarn deployments, configuring `spark.authenticate` to true will automatically handle generating and distributing the shared secret. Each application will use a unique shared secret. The Spark UI uses the standard YARN web application proxy mechanism and will authenticate via any installed Hadoop filters. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI. + +For other types of Spark deployments, the spark config `spark.authenticate.secret` should be configured on each of the nodes. This secret will be used by all the Master/Workers and applications. The UI can be secured using a javax servlet filter installed via `spark.ui.filters`. If an authentication filter is enabled, the acls controls can be used by control which users can via the Spark UI. + +IMPORTANT NOTE: The NettyBlockFetcherIterator is not secured so do not use netty for the shuffle is running with authentication on. + +See [Spark Configuration](configuration.html) for more details on the security configs. + +See org.apache.spark.SecurityManager for implementation details about security. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 3d7b390724e77..62d3a52615584 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -23,7 +23,7 @@ import scala.util.Random import akka.actor.{Actor, ActorRef, Props, actorRef2Scala} -import org.apache.spark.SparkConf +import org.apache.spark.{SparkConf, SecurityManager} import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions import org.apache.spark.streaming.receivers.Receiver @@ -112,8 +112,9 @@ object FeederActor { } val Seq(host, port) = args.toSeq - - val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = new SparkConf)._1 + val conf = new SparkConf + val actorSystem = AkkaUtils.createActorSystem("test", host, port.toInt, conf = conf, + securityManager = new SecurityManager(conf))._1 val feeder = actorSystem.actorOf(Props[FeederActor], "FeederActor") println("Feeder started as:" + feeder) diff --git a/pom.xml b/pom.xml index c59fada5cd4a0..3b863856e4634 100644 --- a/pom.xml +++ b/pom.xml @@ -155,6 +155,21 @@ + + org.eclipse.jetty + jetty-util + 7.6.8.v20121106 + + + org.eclipse.jetty + jetty-security + 7.6.8.v20121106 + + + org.eclipse.jetty + jetty-plus + 7.6.8.v20121106 + org.eclipse.jetty jetty-server @@ -295,6 +310,11 @@ mesos ${mesos.version} + + commons-net + commons-net + 2.2 + io.netty netty-all diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index aa1784897566b..138aad7561043 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -226,6 +226,9 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "io.netty" % "netty-all" % "4.0.17.Final", "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-util" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-plus" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-security" % "7.6.8.v20121106", /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"), "org.scalatest" %% "scalatest" % "1.9.1" % "test", @@ -285,6 +288,7 @@ object SparkBuild extends Build { "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", + "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J), diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index e3bcf7f30ac8d..1aa94079fd0ae 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -18,12 +18,15 @@ package org.apache.spark.repl import java.io.{ByteArrayOutputStream, InputStream} -import java.net.{URI, URL, URLClassLoader, URLEncoder} +import java.net.{URI, URL, URLEncoder} import java.util.concurrent.{Executors, ExecutorService} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.spark.SparkEnv +import org.apache.spark.util.Utils + import org.objectweb.asm._ import org.objectweb.asm.Opcodes._ @@ -53,7 +56,13 @@ extends ClassLoader(parent) { if (fileSystem != null) { fileSystem.open(new Path(directory, pathInDirectory)) } else { - new URL(classUri + "/" + urlEncode(pathInDirectory)).openStream() + if (SparkEnv.get.securityManager.isAuthenticationEnabled()) { + val uri = new URI(classUri + "/" + urlEncode(pathInDirectory)) + val newuri = Utils.constructURIForAuthentication(uri, SparkEnv.get.securityManager) + newuri.toURL().openStream() + } else { + new URL(classUri + "/" + urlEncode(pathInDirectory)).openStream() + } } } val bytes = readAndTransformClass(name, inputStream) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index f52ebe4a159f1..9b1da195002c2 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -881,6 +881,8 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, }) def process(settings: Settings): Boolean = savingContextLoader { + if (getMaster() == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + this.settings = settings createInterpreter() @@ -939,16 +941,9 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def createSparkContext(): SparkContext = { val execUri = System.getenv("SPARK_EXECUTOR_URI") - val master = this.master match { - case Some(m) => m - case None => { - val prop = System.getenv("MASTER") - if (prop != null) prop else "local" - } - } val jars = SparkILoop.getAddedJars.map(new java.io.File(_).getAbsolutePath) val conf = new SparkConf() - .setMaster(master) + .setMaster(getMaster()) .setAppName("Spark shell") .setJars(jars) .set("spark.repl.class.uri", intp.classServer.uri) @@ -963,6 +958,17 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, sparkContext } + private def getMaster(): String = { + val master = this.master match { + case Some(m) => m + case None => { + val prop = System.getenv("MASTER") + if (prop != null) prop else "local" + } + } + master + } + /** process command-line arguments and do as they request */ def process(args: Array[String]): Boolean = { val command = new SparkCommandLine(args.toList, msg => echo(msg)) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 1d73d0b6993a8..90a96ad38381e 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -36,7 +36,7 @@ import scala.tools.reflect.StdRuntimeTags._ import scala.util.control.ControlThrowable import util.stackTraceString -import org.apache.spark.{HttpServer, SparkConf, Logging} +import org.apache.spark.{Logging, HttpServer, SecurityManager, SparkConf} import org.apache.spark.util.Utils // /** directory to save .class files to */ @@ -83,15 +83,17 @@ import org.apache.spark.util.Utils * @author Moez A. Abdel-Gawad * @author Lex Spoon */ - class SparkIMain(initialSettings: Settings, val out: JPrintWriter) extends SparkImports with Logging { + class SparkIMain(initialSettings: Settings, val out: JPrintWriter) + extends SparkImports with Logging { imain => - val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") + val conf = new SparkConf() + val SPARK_DEBUG_REPL: Boolean = (System.getenv("SPARK_DEBUG_REPL") == "1") /** Local directory to save .class files too */ val outputDir = { val tmp = System.getProperty("java.io.tmpdir") - val rootDir = new SparkConf().get("spark.repl.classdir", tmp) + val rootDir = conf.get("spark.repl.classdir", tmp) Utils.createTempDir(rootDir) } if (SPARK_DEBUG_REPL) { @@ -99,7 +101,8 @@ import org.apache.spark.util.Utils } val virtualDirectory = new PlainFile(outputDir) // "directory" for classfiles - val classServer = new HttpServer(outputDir) /** Jetty server that will serve our classes to worker nodes */ + val classServer = new HttpServer(outputDir, + new SecurityManager(conf)) /** Jetty server that will serve our classes to worker nodes */ private var currentSettings: Settings = initialSettings var printResults = true // whether to print result lines var totalSilence = false // whether to print anything diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index e045b9f0248f6..bb574f415293a 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -27,7 +27,6 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.records._ @@ -36,7 +35,7 @@ import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -87,27 +86,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, isLastAMRetry = appAttemptId.getAttemptId() >= maxAppAttempts resourceManager = registerWithResourceManager() - // Workaround until hadoop moves to something which has - // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line) - // ignore result. - // This does not, unfortunately, always work reliably ... but alleviates the bug a lot of times - // Hence args.workerCores = numCore disabled above. Any better option? - - // Compute number of threads for akka - //val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() - //if (minimumMemory > 0) { - // val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD - // val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) - - // if (numCore > 0) { - // do not override - hits https://issues.apache.org/jira/browse/HADOOP-8406 - // TODO: Uncomment when hadoop is on a version which has this fixed. - // args.workerCores = numCore - // } - //} - // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) + // setup AmIpFilter for the SparkUI - do this before we start the UI + addAmIpFilter() ApplicationMaster.register(this) + + // Call this to force generation of secret so it gets populated into the + // hadoop UGI. This has to happen before the startUserClass which does a + // doAs in order for the credentials to be passed on to the worker containers. + val securityMgr = new SecurityManager(sparkConf) + // Start the user's JAR userThread = startUserClass() @@ -132,6 +120,20 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, System.exit(0) } + // add the yarn amIpFilter that Yarn requires for properly securing the UI + private def addAmIpFilter() { + val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + System.setProperty("spark.ui.filters", amFilter) + val proxy = YarnConfiguration.getProxyHostAndPort(conf) + val parts : Array[String] = proxy.split(":") + val uriBase = "http://" + proxy + + System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) + + val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase + System.setProperty("spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", + params) + } + /** Get the Yarn approved local directories. */ private def getLocalDirs(): String = { // Hadoop 0.23 and 2.x have different Environment variable names for the diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 138c27910b0b0..b735d01df8097 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -29,7 +29,7 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ import akka.actor.Terminated -import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo @@ -50,8 +50,9 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar private var yarnAllocator: YarnAllocationHandler = _ private var driverClosed:Boolean = false + val securityManager = new SecurityManager(sparkConf) val actorSystem : ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = sparkConf)._1 + conf = sparkConf, securityManager = securityManager)._1 var actor: ActorRef = _ // This actor just working as a monitor to watch on Driver Actor. @@ -110,6 +111,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar // we want to be reasonably responsive without causing too many requests to RM. val schedulerInterval = System.getProperty("spark.yarn.scheduler.heartbeat.interval-ms", "5000").toLong + // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index fe37168e5a7ba..11322b1202f99 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -134,7 +134,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1). This is unsused right now.\n" + + " --worker-cores NUM Number of cores for the workers (Default: 1).\n" + " --master-class CLASS_NAME Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" + " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index d6c12a9f5952d..4c6e1dcd6dac3 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -17,11 +17,13 @@ package org.apache.spark.deploy.yarn -import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.hadoop.io.Text import org.apache.hadoop.mapred.JobConf +import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.conf.Configuration +import org.apache.spark.deploy.SparkHadoopUtil /** * Contains util methods to interact with Hadoop from spark. @@ -44,4 +46,24 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { val jobCreds = conf.getCredentials() jobCreds.mergeAll(UserGroupInformation.getCurrentUser().getCredentials()) } + + override def getCurrentUserCredentials(): Credentials = { + UserGroupInformation.getCurrentUser().getCredentials() + } + + override def addCurrentUserCredentials(creds: Credentials) { + UserGroupInformation.getCurrentUser().addCredentials(creds) + } + + override def addSecretKeyToUserCredentials(key: String, secret: String) { + val creds = new Credentials() + creds.addSecretKey(new Text(key), secret.getBytes()) + addCurrentUserCredentials(creds) + } + + override def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { + val credentials = getCurrentUserCredentials() + if (credentials != null) credentials.getSecretKey(new Text(key)) else null + } + } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index dd117d5810949..b48a2d50db5ef 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -27,7 +27,6 @@ import scala.collection.JavaConversions._ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.ShutdownHookManager import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.protocolrecords._ @@ -37,8 +36,9 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} +import org.apache.hadoop.yarn.webapp.util.WebAppUtils; -import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils @@ -91,12 +91,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, amClient.init(yarnConf) amClient.start() - // Workaround until hadoop moves to something which has - // https://issues.apache.org/jira/browse/HADOOP-8406 - fixed in (2.0.2-alpha but no 0.23 line) - // org.apache.hadoop.io.compress.CompressionCodecFactory.getCodecClasses(conf) + // setup AmIpFilter for the SparkUI - do this before we start the UI + addAmIpFilter() ApplicationMaster.register(this) + // Call this to force generation of secret so it gets populated into the + // hadoop UGI. This has to happen before the startUserClass which does a + // doAs in order for the credentials to be passed on to the worker containers. + val securityMgr = new SecurityManager(sparkConf) + // Start the user's JAR userThread = startUserClass() @@ -121,6 +125,19 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, System.exit(0) } + // add the yarn amIpFilter that Yarn requires for properly securing the UI + private def addAmIpFilter() { + val amFilter = "org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter" + System.setProperty("spark.ui.filters", amFilter) + val proxy = WebAppUtils.getProxyHostAndPort(conf) + val parts : Array[String] = proxy.split(":") + val uriBase = "http://" + proxy + + System.getenv(ApplicationConstants.APPLICATION_WEB_PROXY_BASE_ENV) + + val params = "PROXY_HOST=" + parts(0) + "," + "PROXY_URI_BASE=" + uriBase + System.setProperty("spark.org.apache.hadoop.yarn.server.webproxy.amfilter.AmIpFilter.params", params) + } + /** Get the Yarn approved local directories. */ private def getLocalDirs(): String = { // Hadoop 0.23 and 2.x have different Environment variable names for the @@ -261,7 +278,6 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, val schedulerInterval = sparkConf.getLong("spark.yarn.scheduler.heartbeat.interval-ms", 5000) - // must be <= timeoutInterval / 2. val interval = math.min(timeoutInterval / 2, schedulerInterval) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala index 40600f38e5e73..f1c1fea0b5895 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala @@ -28,7 +28,7 @@ import org.apache.hadoop.yarn.util.{ConverterUtils, Records} import akka.actor._ import akka.remote._ import akka.actor.Terminated -import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo @@ -52,8 +52,9 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar private var amClient: AMRMClient[ContainerRequest] = _ + val securityManager = new SecurityManager(sparkConf) val actorSystem: ActorSystem = AkkaUtils.createActorSystem("sparkYarnAM", Utils.localHostName, 0, - conf = sparkConf)._1 + conf = sparkConf, securityManager = securityManager)._1 var actor: ActorRef = _ // This actor just working as a monitor to watch on Driver Actor. @@ -105,6 +106,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar val interval = math.min(timeoutInterval / 2, schedulerInterval) reporterThread = launchReporterThread(interval) + // Wait for the reporter thread to Finish. reporterThread.join() From 328c73d037c17440c2a91a6c88b4258fbefa0c08 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 6 Mar 2014 17:12:58 -0800 Subject: [PATCH 056/397] SPARK-1197. Change yarn-standalone to yarn-cluster and fix up running on YARN docs This patch changes "yarn-standalone" to "yarn-cluster" (but still supports the former). It also cleans up the Running on YARN docs and adds a section on how to view logs. Author: Sandy Ryza Closes #95 from sryza/sandy-spark-1197 and squashes the following commits: 563ef3a [Sandy Ryza] Review feedback 6ad06d4 [Sandy Ryza] Change yarn-standalone to yarn-cluster and fix up running on YARN docs --- .../scala/org/apache/spark/SparkContext.scala | 14 ++-- .../SparkContextSchedulerCreationSuite.scala | 4 ++ docs/running-on-yarn.md | 65 ++++++++++--------- .../spark/deploy/yarn/ClientArguments.scala | 2 +- 4 files changed, 51 insertions(+), 34 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 24731ad706a49..ce25573834829 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -738,8 +738,10 @@ class SparkContext( key = uri.getScheme match { // A JAR file which exists only on the driver node case null | "file" => - if (SparkHadoopUtil.get.isYarnMode() && master == "yarn-standalone") { - // In order for this to work in yarn standalone mode the user must specify the + // yarn-standalone is deprecated, but still supported + if (SparkHadoopUtil.get.isYarnMode() && + (master == "yarn-standalone" || master == "yarn-cluster")) { + // In order for this to work in yarn-cluster mode the user must specify the // --addjars option to the client to upload the file into the distributed cache // of the AM to make it show up in the current working directory. val fileName = new Path(uri.getPath).getName() @@ -1027,7 +1029,7 @@ class SparkContext( * The SparkContext object contains a number of implicit conversions and parameters for use with * various Spark features. */ -object SparkContext { +object SparkContext extends Logging { private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" @@ -1245,7 +1247,11 @@ object SparkContext { } scheduler - case "yarn-standalone" => + case "yarn-standalone" | "yarn-cluster" => + if (master == "yarn-standalone") { + logWarning( + "\"yarn-standalone\" is deprecated as of Spark 1.0. Use \"yarn-cluster\" instead.") + } val scheduler = try { val clazz = Class.forName("org.apache.spark.scheduler.cluster.YarnClusterScheduler") val cons = clazz.getConstructor(classOf[SparkContext]) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index f28d5c7b133b3..3bb936790d506 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -95,6 +95,10 @@ class SparkContextSchedulerCreationSuite } } + test("yarn-cluster") { + testYarn("yarn-cluster", "org.apache.spark.scheduler.cluster.YarnClusterScheduler") + } + test("yarn-standalone") { testYarn("yarn-standalone", "org.apache.spark.scheduler.cluster.YarnClusterScheduler") } diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index ee1d892a3b630..b17929542c531 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -29,7 +29,7 @@ If you want to test out the YARN deployment mode, you can use the current Spark # Configuration -Most of the configs are the same for Spark on YARN as other deploys. See the Configuration page for more information on those. These are configs that are specific to SPARK on YARN. +Most of the configs are the same for Spark on YARN as for other deployment modes. See the Configuration page for more information on those. These are configs that are specific to Spark on YARN. Environment variables: @@ -41,28 +41,30 @@ System Properties: * `spark.yarn.submit.file.replication`, the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives. * `spark.yarn.preserve.staging.files`, set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them. * `spark.yarn.scheduler.heartbeat.interval-ms`, the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds. -* `spark.yarn.max.worker.failures`, the maximum number of worker failures before failing the application. Default is the number of workers requested times 2 with minimum of 3. +* `spark.yarn.max.worker.failures`, the maximum number of executor failures before failing the application. Default is the number of executors requested times 2 with minimum of 3. # Launching Spark on YARN -Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the hadoop cluster. -This would be used to connect to the cluster, write to the dfs and submit jobs to the resource manager. +Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the Hadoop cluster. +These configs are used to connect to the cluster, write to the dfs, and connect to the YARN ResourceManager. -There are two scheduler mode that can be used to launch spark application on YARN. +There are two scheduler modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. -## Launch spark application by YARN Client with yarn-standalone mode. +Unlike in Spark standalone and Mesos mode, in which the master's address is specified in the "master" parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration. Thus, the master parameter is simply "yarn-client" or "yarn-cluster". -The command to launch the YARN Client is as follows: +## Launching a Spark application with yarn-cluster mode. + +The command to launch the Spark application on the cluster is as follows: SPARK_JAR= ./bin/spark-class org.apache.spark.deploy.yarn.Client \ --jar \ --class \ --args \ - --num-workers \ + --num-workers \ --master-class --master-memory \ - --worker-memory \ - --worker-cores \ + --worker-memory \ + --worker-cores \ --name \ --queue \ --addJars \ @@ -82,35 +84,30 @@ For example: ./bin/spark-class org.apache.spark.deploy.yarn.Client \ --jar examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ --class org.apache.spark.examples.SparkPi \ - --args yarn-standalone \ + --args yarn-cluster \ --num-workers 3 \ --master-memory 4g \ --worker-memory 2g \ --worker-cores 1 - # Examine the output (replace $YARN_APP_ID in the following with the "application identifier" output by the previous command) - # (Note: YARN_APP_LOGS_DIR is usually /tmp/logs or $HADOOP_HOME/logs/userlogs depending on the Hadoop version.) - $ cat $YARN_APP_LOGS_DIR/$YARN_APP_ID/container*_000001/stdout - Pi is roughly 3.13794 - -The above starts a YARN Client programs which start the default Application Master. Then SparkPi will be run as a child thread of Application Master, YARN Client will periodically polls the Application Master for status updates and displays them in the console. The client will exit once your application has finished running. +The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Viewing Logs" section below for how to see driver and executor logs. -With this mode, your application is actually run on the remote machine where the Application Master is run upon. Thus application that involve local interaction will not work well, e.g. spark-shell. +Because the application is run on a remote machine where the Application Master is running, applications that involve local interaction, such as spark-shell, will not work. -## Launch spark application with yarn-client mode. +## Launching a Spark application with yarn-client mode. -With yarn-client mode, the application will be launched locally. Just like running application or spark-shell on Local / Mesos / Standalone mode. The launch method is also the similar with them, just make sure that when you need to specify a master url, use "yarn-client" instead. And you also need to export the env value for SPARK_JAR. +With yarn-client mode, the application will be launched locally, just like running an application or spark-shell on Local / Mesos / Standalone client mode. The launch method is also the same, just make sure to specify the master URL as "yarn-client". You also need to export the env value for SPARK_JAR. Configuration in yarn-client mode: -In order to tune worker core/number/memory etc. You need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. +In order to tune worker cores/number/memory etc., you need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. -* `SPARK_WORKER_INSTANCES`, Number of workers to start (Default: 2) -* `SPARK_WORKER_CORES`, Number of cores for the workers (Default: 1). -* `SPARK_WORKER_MEMORY`, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) +* `SPARK_WORKER_INSTANCES`, Number of executors to start (Default: 2) +* `SPARK_WORKER_CORES`, Number of cores per executor (Default: 1). +* `SPARK_WORKER_MEMORY`, Memory per executor (e.g. 1000M, 2G) (Default: 1G) * `SPARK_MASTER_MEMORY`, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) * `SPARK_YARN_APP_NAME`, The name of your application (Default: Spark) -* `SPARK_YARN_QUEUE`, The hadoop queue to use for allocation requests (Default: 'default') +* `SPARK_YARN_QUEUE`, The YARN queue to use for allocation requests (Default: 'default') * `SPARK_YARN_DIST_FILES`, Comma separated list of files to be distributed with the job. * `SPARK_YARN_DIST_ARCHIVES`, Comma separated list of archives to be distributed with the job. @@ -125,13 +122,23 @@ or MASTER=yarn-client ./bin/spark-shell +## Viewing logs + +In YARN terminology, executors and application masters run inside "containers". YARN has two modes for handling container logs after an application has completed. If log aggregation is turned on (with the yarn.log-aggregation-enable config), container logs are copied to HDFS and deleted on the local machine. These logs can be viewed from anywhere on the cluster with the "yarn logs" command. + + yarn logs -applicationId + +will print out the contents of all log files from all containers from the given application. + +When log aggregation isn't turned on, logs are retained locally on each machine under YARN_APP_LOGS_DIR, which is usually configured to /tmp/logs or $HADOOP_HOME/logs/userlogs depending on the Hadoop version and installation. Viewing logs for a container requires going to the host that contains them and looking in this directory. Subdirectories organize log files by application ID and container ID. + # Building Spark for Hadoop/YARN 2.2.x -See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using the Maven process. +See [Building Spark with Maven](building-with-maven.html) for instructions on how to build Spark using Maven. -# Important Notes +# Important notes - Before Hadoop 2.2, YARN does not support cores in container resource requests. Thus, when running against an earlier version, the numbers of cores given via command line arguments cannot be passed to YARN. Whether core requests are honored in scheduling decisions depends on which scheduler is in use and how it is configured. -- The local directories used for spark will be the local directories configured for YARN (Hadoop Yarn config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored. -- The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt and your application should use the name as appSees.txt to reference it when running on YARN. +- The local directories used by Spark executors will be the local directories configured for YARN (Hadoop YARN config yarn.nodemanager.local-dirs). If the user specifies spark.local.dir, it will be ignored. +- The --files and --archives options support specifying file names with the # similar to Hadoop. For example you can specify: --files localtest.txt#appSees.txt and this will upload the file you have locally named localtest.txt into HDFS but this will be linked to by the name appSees.txt, and your application should use the name as appSees.txt to reference it when running on YARN. - The --addJars option allows the SparkContext.addJar function to work if you are using it with local files. It does not need to be used if you are using it with HDFS, HTTP, HTTPS, or FTP files. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 11322b1202f99..1f894a677d169 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -129,7 +129,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { System.err.println( "Usage: org.apache.spark.deploy.yarn.Client [options] \n" + "Options:\n" + - " --jar JAR_PATH Path to your application's JAR file (required in yarn-standalone mode)\n" + + " --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster mode)\n" + " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + From 9ae919c02f7b7d069215e8dc6cafef0ec79c9d5f Mon Sep 17 00:00:00 2001 From: anitatailor Date: Thu, 6 Mar 2014 17:46:43 -0800 Subject: [PATCH 057/397] Example for cassandra CQL read/write from spark Cassandra read/write using CqlPagingInputFormat/CqlOutputFormat Author: anitatailor Closes #87 from anitatailor/master and squashes the following commits: 3493f81 [anitatailor] Fixed scala style as per review 19480b7 [anitatailor] Example for cassandra CQL read/write from spark --- .../spark/examples/CassandraCQLTest.scala | 137 ++++++++++++++++++ 1 file changed, 137 insertions(+) create mode 100644 examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala new file mode 100644 index 0000000000000..ee283ce6abac2 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -0,0 +1,137 @@ +/* + * 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.examples + +import java.nio.ByteBuffer +import scala.collection.JavaConversions._ +import scala.collection.mutable.ListBuffer +import scala.collection.immutable.Map +import org.apache.cassandra.hadoop.ConfigHelper +import org.apache.cassandra.hadoop.cql3.CqlPagingInputFormat +import org.apache.cassandra.hadoop.cql3.CqlConfigHelper +import org.apache.cassandra.hadoop.cql3.CqlOutputFormat +import org.apache.cassandra.utils.ByteBufferUtil +import org.apache.hadoop.mapreduce.Job +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ + +/* + Need to create following keyspace and column family in cassandra before running this example + Start CQL shell using ./bin/cqlsh and execute following commands + CREATE KEYSPACE retail WITH replication = {'class': 'SimpleStrategy', 'replication_factor': 1}; + use retail; + CREATE TABLE salecount (prod_id text, sale_count int, PRIMARY KEY (prod_id)); + CREATE TABLE ordercf (user_id text, + time timestamp, + prod_id text, + quantity int, + PRIMARY KEY (user_id, time)); + INSERT INTO ordercf (user_id, + time, + prod_id, + quantity) VALUES ('bob', 1385983646000, 'iphone', 1); + INSERT INTO ordercf (user_id, + time, + prod_id, + quantity) VALUES ('tom', 1385983647000, 'samsung', 4); + INSERT INTO ordercf (user_id, + time, + prod_id, + quantity) VALUES ('dora', 1385983648000, 'nokia', 2); + INSERT INTO ordercf (user_id, + time, + prod_id, + quantity) VALUES ('charlie', 1385983649000, 'iphone', 2); +*/ + +/** + * This example demonstrates how to read and write to cassandra column family created using CQL3 + * using Spark. + * Parameters : + * Usage: ./bin/run-example org.apache.spark.examples.CassandraCQLTest local[2] localhost 9160 + * + */ +object CassandraCQLTest { + + def main(args: Array[String]) { + val sc = new SparkContext(args(0), + "CQLTestApp", + System.getenv("SPARK_HOME"), + SparkContext.jarOfClass(this.getClass)) + val cHost: String = args(1) + val cPort: String = args(2) + val KeySpace = "retail" + val InputColumnFamily = "ordercf" + val OutputColumnFamily = "salecount" + + val job = new Job() + job.setInputFormatClass(classOf[CqlPagingInputFormat]) + ConfigHelper.setInputInitialAddress(job.getConfiguration(), cHost) + ConfigHelper.setInputRpcPort(job.getConfiguration(), cPort) + ConfigHelper.setInputColumnFamily(job.getConfiguration(), KeySpace, InputColumnFamily) + ConfigHelper.setInputPartitioner(job.getConfiguration(), "Murmur3Partitioner") + CqlConfigHelper.setInputCQLPageRowSize(job.getConfiguration(), "3") + + /** CqlConfigHelper.setInputWhereClauses(job.getConfiguration(), "user_id='bob'") */ + + /** An UPDATE writes one or more columns to a record in a Cassandra column family */ + val query = "UPDATE " + KeySpace + "." + OutputColumnFamily + " SET sale_count = ? " + CqlConfigHelper.setOutputCql(job.getConfiguration(), query) + + job.setOutputFormatClass(classOf[CqlOutputFormat]) + ConfigHelper.setOutputColumnFamily(job.getConfiguration(), KeySpace, OutputColumnFamily) + ConfigHelper.setOutputInitialAddress(job.getConfiguration(), cHost) + ConfigHelper.setOutputRpcPort(job.getConfiguration(), cPort) + ConfigHelper.setOutputPartitioner(job.getConfiguration(), "Murmur3Partitioner") + + val casRdd = sc.newAPIHadoopRDD(job.getConfiguration(), + classOf[CqlPagingInputFormat], + classOf[java.util.Map[String,ByteBuffer]], + classOf[java.util.Map[String,ByteBuffer]]) + + println("Count: " + casRdd.count) + val productSaleRDD = casRdd.map { + case (key, value) => { + (ByteBufferUtil.string(value.get("prod_id")), ByteBufferUtil.toInt(value.get("quantity"))) + } + } + val aggregatedRDD = productSaleRDD.reduceByKey(_ + _) + aggregatedRDD.collect().foreach { + case (productId, saleCount) => println(productId + ":" + saleCount) + } + + val casoutputCF = aggregatedRDD.map { + case (productId, saleCount) => { + val outColFamKey = Map("prod_id" -> ByteBufferUtil.bytes(productId)) + val outKey: java.util.Map[String, ByteBuffer] = outColFamKey + var outColFamVal = new ListBuffer[ByteBuffer] + outColFamVal += ByteBufferUtil.bytes(saleCount) + val outVal: java.util.List[ByteBuffer] = outColFamVal + (outKey, outVal) + } + } + + casoutputCF.saveAsNewAPIHadoopFile( + KeySpace, + classOf[java.util.Map[String, ByteBuffer]], + classOf[java.util.List[ByteBuffer]], + classOf[CqlOutputFormat], + job.getConfiguration() + ) + } +} From 33baf14b04bcb5cb8dc39ae0773b9e0ef79ef9cf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 6 Mar 2014 17:57:31 -0800 Subject: [PATCH 058/397] Small clean-up to flatmap tests --- .../apache/spark/storage/FlatmapIteratorSuite.scala | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala index b843b4c629e84..bcf138b5ee6d0 100644 --- a/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/FlatmapIteratorSuite.scala @@ -33,34 +33,29 @@ class FlatmapIteratorSuite extends FunSuite with LocalSparkContext { * info from the serializer, and allow old objects to be GC'd */ test("Flatmap Iterator to Disk") { - val sconf = new SparkConf().setMaster("local-cluster[1,1,512]") - .setAppName("iterator_to_disk_test") + val sconf = new SparkConf().setMaster("local").setAppName("iterator_to_disk_test") sc = new SparkContext(sconf) val expand_size = 100 val data = sc.parallelize((1 to 5).toSeq). flatMap( x => Stream.range(0, expand_size)) var persisted = data.persist(StorageLevel.DISK_ONLY) - println(persisted.count()) assert(persisted.count()===500) assert(persisted.filter(_==1).count()===5) } test("Flatmap Iterator to Memory") { - val sconf = new SparkConf().setMaster("local-cluster[1,1,512]") - .setAppName("iterator_to_disk_test") + val sconf = new SparkConf().setMaster("local").setAppName("iterator_to_disk_test") sc = new SparkContext(sconf) val expand_size = 100 val data = sc.parallelize((1 to 5).toSeq). flatMap(x => Stream.range(0, expand_size)) var persisted = data.persist(StorageLevel.MEMORY_ONLY) - println(persisted.count()) assert(persisted.count()===500) assert(persisted.filter(_==1).count()===5) } test("Serializer Reset") { - val sconf = new SparkConf().setMaster("local-cluster[1,1,512]") - .setAppName("serializer_reset_test") + val sconf = new SparkConf().setMaster("local").setAppName("serializer_reset_test") .set("spark.serializer.objectStreamReset", "10") sc = new SparkContext(sconf) val expand_size = 500 From dabeb6f160f7ad7df1c54b1b8b069700dd4b74dd Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 7 Mar 2014 10:22:27 -0800 Subject: [PATCH 059/397] SPARK-1136: Fix FaultToleranceTest for Docker 0.8.1 This patch allows the FaultToleranceTest to work in newer versions of Docker. See https://spark-project.atlassian.net/browse/SPARK-1136 for more details. Besides changing the Docker and FaultToleranceTest internals, this patch also changes the behavior of Master to accept new Workers which share an address with a Worker that we are currently trying to recover. This can only happen when the Worker itself was restarted and got the same IP address/port at the same time as a Master recovery occurs. Finally, this adds a good bit of ASCII art to the test to make failures, successes, and actions more apparent. This is very much needed. Author: Aaron Davidson Closes #5 from aarondav/zookeeper and squashes the following commits: 5d7a72a [Aaron Davidson] SPARK-1136: Fix FaultToleranceTest for Docker 0.8.1 --- .../spark/deploy/FaultToleranceTest.scala | 41 ++++++++++++++++--- .../apache/spark/deploy/master/Master.scala | 11 ++++- .../deploy/master/SparkCuratorUtil.scala | 13 +++++- docker/README.md | 4 +- docker/spark-test/master/default_cmd | 8 +++- docker/spark-test/worker/default_cmd | 8 +++- 6 files changed, 73 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index d48c1892aea9c..f4eb1601be3e4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -30,20 +30,24 @@ import scala.sys.process._ import org.json4s._ import org.json4s.jackson.JsonMethods -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.deploy.master.RecoveryState +import org.apache.spark.{Logging, SparkConf, SparkContext} +import org.apache.spark.deploy.master.{RecoveryState, SparkCuratorUtil} /** * This suite tests the fault tolerance of the Spark standalone scheduler, mainly the Master. * In order to mimic a real distributed cluster more closely, Docker is used. * Execute using - * ./spark-class org.apache.spark.deploy.FaultToleranceTest + * ./bin/spark-class org.apache.spark.deploy.FaultToleranceTest * - * Make sure that that the environment includes the following properties in SPARK_DAEMON_JAVA_OPTS: + * Make sure that that the environment includes the following properties in SPARK_DAEMON_JAVA_OPTS + * *and* SPARK_JAVA_OPTS: * - spark.deploy.recoveryMode=ZOOKEEPER * - spark.deploy.zookeeper.url=172.17.42.1:2181 * Note that 172.17.42.1 is the default docker ip for the host and 2181 is the default ZK port. * + * In case of failure, make sure to kill off prior docker containers before restarting: + * docker kill $(docker ps -q) + * * Unfortunately, due to the Docker dependency this suite cannot be run automatically without a * working installation of Docker. In addition to having Docker, the following are assumed: * - Docker can run without sudo (see http://docs.docker.io/en/latest/use/basics/) @@ -51,10 +55,16 @@ import org.apache.spark.deploy.master.RecoveryState * docker/ directory. Run 'docker/spark-test/build' to generate these. */ private[spark] object FaultToleranceTest extends App with Logging { + + val conf = new SparkConf() + val ZK_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + val masters = ListBuffer[TestMasterInfo]() val workers = ListBuffer[TestWorkerInfo]() var sc: SparkContext = _ + val zk = SparkCuratorUtil.newClient(conf) + var numPassed = 0 var numFailed = 0 @@ -72,6 +82,10 @@ private[spark] object FaultToleranceTest extends App with Logging { sc = null } terminateCluster() + + // Clear ZK directories in between tests (for speed purposes) + SparkCuratorUtil.deleteRecursive(zk, ZK_DIR + "/spark_leader") + SparkCuratorUtil.deleteRecursive(zk, ZK_DIR + "/master_status") } test("sanity-basic") { @@ -168,26 +182,34 @@ private[spark] object FaultToleranceTest extends App with Logging { try { fn numPassed += 1 + logInfo("==============================================") logInfo("Passed: " + name) + logInfo("==============================================") } catch { case e: Exception => numFailed += 1 + logInfo("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") logError("FAILED: " + name, e) + logInfo("!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!!") + sys.exit(1) } afterEach() } def addMasters(num: Int) { + logInfo(s">>>>> ADD MASTERS $num <<<<<") (1 to num).foreach { _ => masters += SparkDocker.startMaster(dockerMountDir) } } def addWorkers(num: Int) { + logInfo(s">>>>> ADD WORKERS $num <<<<<") val masterUrls = getMasterUrls(masters) (1 to num).foreach { _ => workers += SparkDocker.startWorker(dockerMountDir, masterUrls) } } /** Creates a SparkContext, which constructs a Client to interact with our cluster. */ def createClient() = { + logInfo(">>>>> CREATE CLIENT <<<<<") if (sc != null) { sc.stop() } // Counter-hack: Because of a hack in SparkEnv#create() that changes this // property, we need to reset it. @@ -206,6 +228,7 @@ private[spark] object FaultToleranceTest extends App with Logging { } def killLeader(): Unit = { + logInfo(">>>>> KILL LEADER <<<<<") masters.foreach(_.readState()) val leader = getLeader masters -= leader @@ -215,6 +238,7 @@ private[spark] object FaultToleranceTest extends App with Logging { def delay(secs: Duration = 5.seconds) = Thread.sleep(secs.toMillis) def terminateCluster() { + logInfo(">>>>> TERMINATE CLUSTER <<<<<") masters.foreach(_.kill()) workers.foreach(_.kill()) masters.clear() @@ -245,6 +269,7 @@ private[spark] object FaultToleranceTest extends App with Logging { * are all alive in a proper configuration (e.g., only one leader). */ def assertValidClusterState() = { + logInfo(">>>>> ASSERT VALID CLUSTER STATE <<<<<") assertUsable() var numAlive = 0 var numStandby = 0 @@ -326,7 +351,11 @@ private[spark] class TestMasterInfo(val ip: String, val dockerId: DockerId, val val workers = json \ "workers" val liveWorkers = workers.children.filter(w => (w \ "state").extract[String] == "ALIVE") - liveWorkerIPs = liveWorkers.map(w => (w \ "host").extract[String]) + // Extract the worker IP from "webuiaddress" (rather than "host") because the host name + // on containers is a weird hash instead of the actual IP address. + liveWorkerIPs = liveWorkers.map { + w => (w \ "webuiaddress").extract[String].stripPrefix("http://").stripSuffix(":8081") + } numLiveApps = (json \ "activeapps").children.size @@ -403,7 +432,7 @@ private[spark] object Docker extends Logging { def makeRunCmd(imageTag: String, args: String = "", mountDir: String = ""): ProcessBuilder = { val mountCmd = if (mountDir != "") { " -v " + mountDir } else "" - val cmd = "docker run %s %s %s".format(mountCmd, imageTag, args) + val cmd = "docker run -privileged %s %s %s".format(mountCmd, imageTag, args) logDebug("Run command: " + cmd) cmd } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2d6d0c33fac7e..b8dfa44102583 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -531,8 +531,15 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, val workerAddress = worker.actor.path.address if (addressToWorker.contains(workerAddress)) { - logInfo("Attempted to re-register worker at same address: " + workerAddress) - return false + val oldWorker = addressToWorker(workerAddress) + if (oldWorker.state == WorkerState.UNKNOWN) { + // A worker registering from UNKNOWN implies that the worker was restarted during recovery. + // The old worker must thus be dead, so we will remove it and accept the new worker. + removeWorker(oldWorker) + } else { + logInfo("Attempted to re-register worker at same address: " + workerAddress) + return false + } } workers += worker diff --git a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala index 2d35397035a03..4781a80d470e1 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/SparkCuratorUtil.scala @@ -17,11 +17,13 @@ package org.apache.spark.deploy.master -import org.apache.spark.{SparkConf, Logging} +import scala.collection.JavaConversions._ + import org.apache.curator.framework.{CuratorFramework, CuratorFrameworkFactory} import org.apache.curator.retry.ExponentialBackoffRetry import org.apache.zookeeper.KeeperException +import org.apache.spark.{Logging, SparkConf} object SparkCuratorUtil extends Logging { @@ -50,4 +52,13 @@ object SparkCuratorUtil extends Logging { } } } + + def deleteRecursive(zk: CuratorFramework, path: String) { + if (zk.checkExists().forPath(path) != null) { + for (child <- zk.getChildren.forPath(path)) { + zk.delete().forPath(path + "/" + child) + } + zk.delete().forPath(path) + } + } } diff --git a/docker/README.md b/docker/README.md index bf59e77d111f9..40ba9c3065946 100644 --- a/docker/README.md +++ b/docker/README.md @@ -2,4 +2,6 @@ Spark docker files =========== Drawn from Matt Massie's docker files (https://github.com/massie/dockerfiles), -as well as some updates from Andre Schumacher (https://github.com/AndreSchumacher/docker). \ No newline at end of file +as well as some updates from Andre Schumacher (https://github.com/AndreSchumacher/docker). + +Tested with Docker version 0.8.1. diff --git a/docker/spark-test/master/default_cmd b/docker/spark-test/master/default_cmd index a5b1303c2ebdb..5a7da3446f6d2 100755 --- a/docker/spark-test/master/default_cmd +++ b/docker/spark-test/master/default_cmd @@ -19,4 +19,10 @@ IP=$(ip -o -4 addr list eth0 | perl -n -e 'if (m{inet\s([\d\.]+)\/\d+\s}xms) { print $1 }') echo "CONTAINER_IP=$IP" -/opt/spark/spark-class org.apache.spark.deploy.master.Master -i $IP +export SPARK_LOCAL_IP=$IP +export SPARK_PUBLIC_DNS=$IP + +# Avoid the default Docker behavior of mapping our IP address to an unreachable host name +umount /etc/hosts + +/opt/spark/bin/spark-class org.apache.spark.deploy.master.Master -i $IP diff --git a/docker/spark-test/worker/default_cmd b/docker/spark-test/worker/default_cmd index ab6336f70c1c6..31b06cb0eb047 100755 --- a/docker/spark-test/worker/default_cmd +++ b/docker/spark-test/worker/default_cmd @@ -19,4 +19,10 @@ IP=$(ip -o -4 addr list eth0 | perl -n -e 'if (m{inet\s([\d\.]+)\/\d+\s}xms) { print $1 }') echo "CONTAINER_IP=$IP" -/opt/spark/spark-class org.apache.spark.deploy.worker.Worker $1 +export SPARK_LOCAL_IP=$IP +export SPARK_PUBLIC_DNS=$IP + +# Avoid the default Docker behavior of mapping our IP address to an unreachable host name +umount /etc/hosts + +/opt/spark/bin/spark-class org.apache.spark.deploy.worker.Worker $1 From b7cd9e992cbc2e649534a2cdf9b8bde2c1ee26bd Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 7 Mar 2014 10:36:55 -0800 Subject: [PATCH 060/397] SPARK-1195: set map_input_file environment variable in PipedRDD Hadoop uses the config mapreduce.map.input.file to indicate the input filename to the map when the input split is of type FileSplit. Some of the hadoop input and output formats set or use this config. This config can also be used by user code. PipedRDD runs an external process and the configs aren't available to that process. Hadoop Streaming does something very similar and the way they make configs available is exporting them into the environment replacing '.' with '_'. Spark should also export this variable when launching the pipe command so the user code has access to that config. Note that the config mapreduce.map.input.file is the new one, the old one which is deprecated but not yet removed is map.input.file. So we should handle both. Perhaps it would be better to abstract this out somehow so it goes into the HadoopParition code? Author: Thomas Graves Closes #94 from tgravescs/map_input_file and squashes the following commits: cc97a6a [Thomas Graves] Update test to check for existence of command, add a getPipeEnvVars function to HadoopRDD e3401dc [Thomas Graves] Merge remote-tracking branch 'upstream/master' into map_input_file 2ba805e [Thomas Graves] set map_input_file environment variable in PipedRDD --- .../org/apache/spark/rdd/HadoopRDD.scala | 19 ++ .../scala/org/apache/spark/rdd/PipedRDD.scala | 8 + .../org/apache/spark/PipedRDDSuite.scala | 184 +++++++++++++----- 3 files changed, 158 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index a374fc4a871b0..100ddb360732a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -18,8 +18,10 @@ package org.apache.spark.rdd import java.io.EOFException +import scala.collection.immutable.Map import org.apache.hadoop.conf.{Configurable, Configuration} +import org.apache.hadoop.mapred.FileSplit import org.apache.hadoop.mapred.InputFormat import org.apache.hadoop.mapred.InputSplit import org.apache.hadoop.mapred.JobConf @@ -43,6 +45,23 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp override def hashCode(): Int = 41 * (41 + rddId) + idx override val index: Int = idx + + /** + * Get any environment variables that should be added to the users environment when running pipes + * @return a Map with the environment variables and corresponding values, it could be empty + */ + def getPipeEnvVars(): Map[String, String] = { + val envVars: Map[String, String] = if (inputSplit.value.isInstanceOf[FileSplit]) { + val is: FileSplit = inputSplit.value.asInstanceOf[FileSplit] + // map_input_file is deprecated in favor of mapreduce_map_input_file but set both + // since its not removed yet + Map("map_input_file" -> is.getPath().toString(), + "mapreduce_map_input_file" -> is.getPath().toString()) + } else { + Map() + } + envVars + } } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index abd4414e81f5c..4250a9d02f764 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -28,6 +28,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, SparkEnv, TaskContext} + /** * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. @@ -59,6 +60,13 @@ class PipedRDD[T: ClassTag]( val currentEnvVars = pb.environment() envVars.foreach { case (variable, value) => currentEnvVars.put(variable, value) } + // for compatibility with Hadoop which sets these env variables + // so the user code can access the input filename + if (split.isInstanceOf[HadoopPartition]) { + val hadoopSplit = split.asInstanceOf[HadoopPartition] + currentEnvVars.putAll(hadoopSplit.getPipeEnvVars()) + } + val proc = pb.start() val env = SparkEnv.get diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 3a0385a1b0bd9..0bac78d8a6bdf 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -19,74 +19,152 @@ package org.apache.spark import org.scalatest.FunSuite + +import org.apache.spark.rdd.{HadoopRDD, PipedRDD, HadoopPartition} +import org.apache.hadoop.mapred.{JobConf, TextInputFormat, FileSplit} +import org.apache.hadoop.fs.Path + +import scala.collection.Map +import scala.sys.process._ +import scala.util.Try +import org.apache.hadoop.io.{Text, LongWritable} + class PipedRDDSuite extends FunSuite with SharedSparkContext { test("basic pipe") { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + if (testCommandAvailable("cat")) { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe(Seq("cat")) + val piped = nums.pipe(Seq("cat")) - val c = piped.collect() - assert(c.size === 4) - assert(c(0) === "1") - assert(c(1) === "2") - assert(c(2) === "3") - assert(c(3) === "4") + val c = piped.collect() + assert(c.size === 4) + assert(c(0) === "1") + assert(c(1) === "2") + assert(c(2) === "3") + assert(c(3) === "4") + } else { + assert(true) + } } test("advanced pipe") { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val bl = sc.broadcast(List("0")) - - val piped = nums.pipe(Seq("cat"), - Map[String, String](), - (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, - (i:Int, f: String=> Unit) => f(i + "_")) - - val c = piped.collect() - - assert(c.size === 8) - assert(c(0) === "0") - assert(c(1) === "\u0001") - assert(c(2) === "1_") - assert(c(3) === "2_") - assert(c(4) === "0") - assert(c(5) === "\u0001") - assert(c(6) === "3_") - assert(c(7) === "4_") - - val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) - val d = nums1.groupBy(str=>str.split("\t")(0)). - pipe(Seq("cat"), - Map[String, String](), - (f: String => Unit) => {bl.value.map(f(_));f("\u0001")}, - (i:Tuple2[String, Seq[String]], f: String=> Unit) => {for (e <- i._2){ f(e + "_")}}).collect() - assert(d.size === 8) - assert(d(0) === "0") - assert(d(1) === "\u0001") - assert(d(2) === "b\t2_") - assert(d(3) === "b\t4_") - assert(d(4) === "0") - assert(d(5) === "\u0001") - assert(d(6) === "a\t1_") - assert(d(7) === "a\t3_") + if (testCommandAvailable("cat")) { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val bl = sc.broadcast(List("0")) + + val piped = nums.pipe(Seq("cat"), + Map[String, String](), + (f: String => Unit) => { + bl.value.map(f(_)); f("\u0001") + }, + (i: Int, f: String => Unit) => f(i + "_")) + + val c = piped.collect() + + assert(c.size === 8) + assert(c(0) === "0") + assert(c(1) === "\u0001") + assert(c(2) === "1_") + assert(c(3) === "2_") + assert(c(4) === "0") + assert(c(5) === "\u0001") + assert(c(6) === "3_") + assert(c(7) === "4_") + + val nums1 = sc.makeRDD(Array("a\t1", "b\t2", "a\t3", "b\t4"), 2) + val d = nums1.groupBy(str => str.split("\t")(0)). + pipe(Seq("cat"), + Map[String, String](), + (f: String => Unit) => { + bl.value.map(f(_)); f("\u0001") + }, + (i: Tuple2[String, Seq[String]], f: String => Unit) => { + for (e <- i._2) { + f(e + "_") + } + }).collect() + assert(d.size === 8) + assert(d(0) === "0") + assert(d(1) === "\u0001") + assert(d(2) === "b\t2_") + assert(d(3) === "b\t4_") + assert(d(4) === "0") + assert(d(5) === "\u0001") + assert(d(6) === "a\t1_") + assert(d(7) === "a\t3_") + } else { + assert(true) + } } test("pipe with env variable") { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe(Seq("printenv", "MY_TEST_ENV"), Map("MY_TEST_ENV" -> "LALALA")) - val c = piped.collect() - assert(c.size === 2) - assert(c(0) === "LALALA") - assert(c(1) === "LALALA") + if (testCommandAvailable("printenv")) { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val piped = nums.pipe(Seq("printenv", "MY_TEST_ENV"), Map("MY_TEST_ENV" -> "LALALA")) + val c = piped.collect() + assert(c.size === 2) + assert(c(0) === "LALALA") + assert(c(1) === "LALALA") + } else { + assert(true) + } } test("pipe with non-zero exit status") { - val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) - val piped = nums.pipe(Seq("cat nonexistent_file", "2>", "/dev/null")) - intercept[SparkException] { - piped.collect() + if (testCommandAvailable("cat")) { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val piped = nums.pipe(Seq("cat nonexistent_file", "2>", "/dev/null")) + intercept[SparkException] { + piped.collect() + } + } else { + assert(true) } } + test("test pipe exports map_input_file") { + testExportInputFile("map_input_file") + } + + test("test pipe exports mapreduce_map_input_file") { + testExportInputFile("mapreduce_map_input_file") + } + + def testCommandAvailable(command: String): Boolean = { + Try(Process(command) !!).isSuccess + } + + def testExportInputFile(varName: String) { + if (testCommandAvailable("printenv")) { + val nums = new HadoopRDD(sc, new JobConf(), classOf[TextInputFormat], classOf[LongWritable], + classOf[Text], 2) { + override def getPartitions: Array[Partition] = Array(generateFakeHadoopPartition()) + + override val getDependencies = List[Dependency[_]]() + + override def compute(theSplit: Partition, context: TaskContext) = { + new InterruptibleIterator[(LongWritable, Text)](context, Iterator((new LongWritable(1), + new Text("b")))) + } + } + val hadoopPart1 = generateFakeHadoopPartition() + val pipedRdd = new PipedRDD(nums, "printenv " + varName) + val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, + taskMetrics = null) + val rddIter = pipedRdd.compute(hadoopPart1, tContext) + val arr = rddIter.toArray + assert(arr(0) == "/some/path") + } else { + // printenv isn't available so just pass the test + assert(true) + } + } + + def generateFakeHadoopPartition(): HadoopPartition = { + val split = new FileSplit(new Path("/some/path"), 0, 1, + Array[String]("loc1", "loc2", "loc3", "loc4", "loc5")) + new HadoopPartition(sc.newRddId(), 1, split) + } + } From 6e730edcde7ca6cbb5727dff7a42f7284b368528 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 7 Mar 2014 18:48:07 -0800 Subject: [PATCH 061/397] Spark 1165 rdd.intersection in python and java Author: Prashant Sharma Author: Prashant Sharma Closes #80 from ScrapCodes/SPARK-1165/RDD.intersection and squashes the following commits: 9b015e9 [Prashant Sharma] Added a note, shuffle is required for intersection. 1fea813 [Prashant Sharma] correct the lines wrapping d0c71f3 [Prashant Sharma] SPARK-1165 RDD.intersection in java d6effee [Prashant Sharma] SPARK-1165 Implemented RDD.intersection in python. --- .../apache/spark/api/java/JavaDoubleRDD.scala | 8 +++++ .../apache/spark/api/java/JavaPairRDD.scala | 10 ++++++ .../org/apache/spark/api/java/JavaRDD.scala | 9 ++++++ .../java/org/apache/spark/JavaAPISuite.java | 31 +++++++++++++++++++ python/pyspark/rdd.py | 17 ++++++++++ 5 files changed, 75 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index d1787061bc642..f816bb43a5b44 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -140,6 +140,14 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja */ def union(other: JavaDoubleRDD): JavaDoubleRDD = fromRDD(srdd.union(other.srdd)) + /** + * Return the intersection of this RDD and another one. The output will not contain any duplicate + * elements, even if the input RDDs did. + * + * Note that this method performs a shuffle internally. + */ + def intersection(other: JavaDoubleRDD): JavaDoubleRDD = fromRDD(srdd.intersection(other.srdd)) + // Double RDD functions /** Add up the elements in this RDD. */ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 857626fe84af9..0ff428c120353 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -126,6 +126,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def union(other: JavaPairRDD[K, V]): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.union(other.rdd)) + /** + * Return the intersection of this RDD and another one. The output will not contain any duplicate + * elements, even if the input RDDs did. + * + * Note that this method performs a shuffle internally. + */ + def intersection(other: JavaPairRDD[K, V]): JavaPairRDD[K, V] = + new JavaPairRDD[K, V](rdd.intersection(other.rdd)) + + // first() has to be overridden here so that the generated method has the signature // 'public scala.Tuple2 first()'; if the trait's definition is used, // then the method has the signature 'public java.lang.Object first()', diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index e973c46edd1ce..91bf404631f49 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -106,6 +106,15 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) */ def union(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.union(other.rdd)) + + /** + * Return the intersection of this RDD and another one. The output will not contain any duplicate + * elements, even if the input RDDs did. + * + * Note that this method performs a shuffle internally. + */ + def intersection(other: JavaRDD[T]): JavaRDD[T] = wrapRDD(rdd.intersection(other.rdd)) + /** * Return an RDD with the elements from `this` that are not in `other`. * diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index c7d0e2d577726..40e853c39ca99 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -110,6 +110,37 @@ public void sparkContextUnion() { Assert.assertEquals(4, pUnion.count()); } + @SuppressWarnings("unchecked") + @Test + public void intersection() { + List ints1 = Arrays.asList(1, 10, 2, 3, 4, 5); + List ints2 = Arrays.asList(1, 6, 2, 3, 7, 8); + JavaRDD s1 = sc.parallelize(ints1); + JavaRDD s2 = sc.parallelize(ints2); + + JavaRDD intersections = s1.intersection(s2); + Assert.assertEquals(3, intersections.count()); + + ArrayList list = new ArrayList(); + JavaRDD empty = sc.parallelize(list); + JavaRDD emptyIntersection = empty.intersection(s2); + Assert.assertEquals(0, emptyIntersection.count()); + + List doubles = Arrays.asList(1.0, 2.0); + JavaDoubleRDD d1 = sc.parallelizeDoubles(doubles); + JavaDoubleRDD d2 = sc.parallelizeDoubles(doubles); + JavaDoubleRDD dIntersection = d1.intersection(d2); + Assert.assertEquals(2, dIntersection.count()); + + List> pairs = new ArrayList>(); + pairs.add(new Tuple2(1, 2)); + pairs.add(new Tuple2(3, 4)); + JavaPairRDD p1 = sc.parallelizePairs(pairs); + JavaPairRDD p2 = sc.parallelizePairs(pairs); + JavaPairRDD pIntersection = p1.intersection(p2); + Assert.assertEquals(2, pIntersection.count()); + } + @Test public void sortByKey() { List> pairs = new ArrayList>(); diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 097a0a236b145..e72f57d9d1ab0 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -326,6 +326,23 @@ def union(self, other): return RDD(self_copy._jrdd.union(other_copy._jrdd), self.ctx, self.ctx.serializer) + def intersection(self, other): + """ + Return the intersection of this RDD and another one. The output will not + contain any duplicate elements, even if the input RDDs did. + + Note that this method performs a shuffle internally. + + >>> rdd1 = sc.parallelize([1, 10, 2, 3, 4, 5]) + >>> rdd2 = sc.parallelize([1, 6, 2, 3, 7, 8]) + >>> rdd1.intersection(rdd2).collect() + [1, 2, 3] + """ + return self.map(lambda v: (v, None)) \ + .cogroup(other.map(lambda v: (v, None))) \ + .filter(lambda x: (len(x[1][0]) != 0) and (len(x[1][1]) != 0)) \ + .keys() + def _reserialize(self): if self._jrdd_deserializer == self.ctx.serializer: return self From a99fb3747a0bc9498cb1d19ae5b5bb0163e6f52b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 7 Mar 2014 23:10:35 -0800 Subject: [PATCH 062/397] SPARK-1193. Fix indentation in pom.xmls Author: Sandy Ryza Closes #91 from sryza/sandy-spark-1193 and squashes the following commits: a878124 [Sandy Ryza] SPARK-1193. Fix indentation in pom.xmls --- bagel/pom.xml | 8 +- core/pom.xml | 520 +++++++++++++++++++-------------------- examples/pom.xml | 8 +- external/flume/pom.xml | 8 +- external/kafka/pom.xml | 8 +- external/twitter/pom.xml | 8 +- external/zeromq/pom.xml | 8 +- graphx/pom.xml | 8 +- mllib/pom.xml | 8 +- repl/pom.xml | 8 +- streaming/pom.xml | 8 +- tools/pom.xml | 8 +- yarn/alpha/pom.xml | 8 +- yarn/stable/pom.xml | 8 +- 14 files changed, 305 insertions(+), 319 deletions(-) diff --git a/bagel/pom.xml b/bagel/pom.xml index 41aacbd88a7d7..142f75c5d2c64 100644 --- a/bagel/pom.xml +++ b/bagel/pom.xml @@ -37,10 +37,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/core/pom.xml b/core/pom.xml index 4c1c2d4da5f77..3e6e98cd2cf92 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -17,274 +17,260 @@ --> - 4.0.0 - - org.apache.spark - spark-parent - 1.0.0-SNAPSHOT - ../pom.xml - - + 4.0.0 + org.apache.spark - spark-core_2.10 - jar - Spark Project Core - http://spark.apache.org/ + spark-parent + 1.0.0-SNAPSHOT + ../pom.xml + - - - - yarn-alpha - - - org.apache.avro - avro - - - - + org.apache.spark + spark-core_2.10 + jar + Spark Project Core + http://spark.apache.org/ - - - org.apache.hadoop - hadoop-client - - - net.java.dev.jets3t - jets3t - - - commons-logging - commons-logging - - - - - org.apache.curator - curator-recipes - - - org.eclipse.jetty - jetty-plus - - - org.eclipse.jetty - jetty-security - - - org.eclipse.jetty - jetty-util - - - org.eclipse.jetty - jetty-server - - - com.google.guava - guava - - - com.google.code.findbugs - jsr305 - - - org.slf4j - slf4j-api - - - org.slf4j - jul-to-slf4j - - - org.slf4j - jcl-over-slf4j - - - log4j - log4j - - - org.slf4j - slf4j-log4j12 - - - com.ning - compress-lzf - - - org.xerial.snappy - snappy-java - - - org.ow2.asm - asm - - - com.twitter - chill_${scala.binary.version} - 0.3.1 - - - com.twitter - chill-java - 0.3.1 - - - commons-net - commons-net - - - ${akka.group} - akka-remote_${scala.binary.version} - - - ${akka.group} - akka-slf4j_${scala.binary.version} - - - ${akka.group} - akka-testkit_${scala.binary.version} - test - - - org.scala-lang - scala-library - - - org.json4s - json4s-jackson_${scala.binary.version} - 3.2.6 - - - - org.scala-lang - scalap - - - - - it.unimi.dsi - fastutil - - - colt - colt - - - org.apache.mesos - mesos - - - io.netty - netty-all - - - com.clearspring.analytics - stream - - - com.codahale.metrics - metrics-core - - - com.codahale.metrics - metrics-jvm - - - com.codahale.metrics - metrics-json - - - com.codahale.metrics - metrics-ganglia - - - com.codahale.metrics - metrics-graphite - - - org.apache.derby - derby - test - - - commons-io - commons-io - test - - - org.scalatest - scalatest_${scala.binary.version} - test - - - org.mockito - mockito-all - test - - - org.scalacheck - scalacheck_${scala.binary.version} - test - - - org.easymock - easymock - test - - - com.novocode - junit-interface - test - - - - target/scala-${scala.binary.version}/classes - target/scala-${scala.binary.version}/test-classes - - - org.apache.maven.plugins - maven-antrun-plugin - - - test - - run - - - true - - - - - - - - - - - - - - - - - - - - org.scalatest - scalatest-maven-plugin - - - ${basedir}/.. - 1 - ${spark.classpath} - - - - - + + + org.apache.hadoop + hadoop-client + + + net.java.dev.jets3t + jets3t + + + commons-logging + commons-logging + + + + + org.apache.curator + curator-recipes + + + org.eclipse.jetty + jetty-plus + + + org.eclipse.jetty + jetty-security + + + org.eclipse.jetty + jetty-util + + + org.eclipse.jetty + jetty-server + + + com.google.guava + guava + + + com.google.code.findbugs + jsr305 + + + org.slf4j + slf4j-api + + + org.slf4j + jul-to-slf4j + + + org.slf4j + jcl-over-slf4j + + + log4j + log4j + + + org.slf4j + slf4j-log4j12 + + + com.ning + compress-lzf + + + org.xerial.snappy + snappy-java + + + org.ow2.asm + asm + + + com.twitter + chill_${scala.binary.version} + 0.3.1 + + + com.twitter + chill-java + 0.3.1 + + + commons-net + commons-net + + + ${akka.group} + akka-remote_${scala.binary.version} + + + ${akka.group} + akka-slf4j_${scala.binary.version} + + + ${akka.group} + akka-testkit_${scala.binary.version} + test + + + org.scala-lang + scala-library + + + org.json4s + json4s-jackson_${scala.binary.version} + 3.2.6 + + + + org.scala-lang + scalap + + + + + it.unimi.dsi + fastutil + + + colt + colt + + + org.apache.mesos + mesos + + + io.netty + netty-all + + + com.clearspring.analytics + stream + + + com.codahale.metrics + metrics-core + + + com.codahale.metrics + metrics-jvm + + + com.codahale.metrics + metrics-json + + + com.codahale.metrics + metrics-ganglia + + + com.codahale.metrics + metrics-graphite + + + org.apache.derby + derby + test + + + commons-io + commons-io + test + + + org.scalatest + scalatest_${scala.binary.version} + test + + + org.mockito + mockito-all + test + + + org.scalacheck + scalacheck_${scala.binary.version} + test + + + org.easymock + easymock + test + + + com.novocode + junit-interface + test + + + + target/scala-${scala.binary.version}/classes + target/scala-${scala.binary.version}/test-classes + + + org.apache.maven.plugins + maven-antrun-plugin + + + test + + run + + + true + + + + + + + + + + + + + + + + + + + + org.scalatest + scalatest-maven-plugin + + + ${basedir}/.. + 1 + ${spark.classpath} + + + + + diff --git a/examples/pom.xml b/examples/pom.xml index 3aba343f4cf50..9f0e2d0b875b8 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -37,10 +37,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/external/flume/pom.xml b/external/flume/pom.xml index 8783aea3e4a5b..f21963531574b 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -37,10 +37,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/external/kafka/pom.xml b/external/kafka/pom.xml index 79dc38f9844a0..343e1fabd823f 100644 --- a/external/kafka/pom.xml +++ b/external/kafka/pom.xml @@ -37,10 +37,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/external/twitter/pom.xml b/external/twitter/pom.xml index 37bb4fad64f68..398b9f4fbaa7d 100644 --- a/external/twitter/pom.xml +++ b/external/twitter/pom.xml @@ -37,10 +37,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/external/zeromq/pom.xml b/external/zeromq/pom.xml index 65ec0e26da881..77e957f404645 100644 --- a/external/zeromq/pom.xml +++ b/external/zeromq/pom.xml @@ -37,10 +37,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/graphx/pom.xml b/graphx/pom.xml index 5b54dd27efb44..894a7c2641e39 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -37,10 +37,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/mllib/pom.xml b/mllib/pom.xml index 760a2a85d5ffa..9b65cb4b4ce3f 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -37,10 +37,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/repl/pom.xml b/repl/pom.xml index aa01a1760285a..fc49c8b811316 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -37,10 +37,10 @@ yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/streaming/pom.xml b/streaming/pom.xml index 91d6a1375a18c..2343e381e6f7c 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -37,10 +37,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/tools/pom.xml b/tools/pom.xml index b8dd255d40ac4..11433e596f5b0 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -36,10 +36,10 @@ yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/yarn/alpha/pom.xml b/yarn/alpha/pom.xml index bfe12ecec0c09..d0aeaceb0d23c 100644 --- a/yarn/alpha/pom.xml +++ b/yarn/alpha/pom.xml @@ -30,10 +30,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + diff --git a/yarn/stable/pom.xml b/yarn/stable/pom.xml index 9d68603251d1c..e7915d12aef63 100644 --- a/yarn/stable/pom.xml +++ b/yarn/stable/pom.xml @@ -30,10 +30,10 @@ a Hadoop 0.23.X issue --> yarn-alpha - - org.apache.avro - avro - + + org.apache.avro + avro + From 8ad486add941c9686dfb39309adaf5b7ca66345d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Fri, 7 Mar 2014 23:23:59 -0800 Subject: [PATCH 063/397] Allow sbt to use more than 1G of heap. There was a mistake in sbt build file ( introduced by 012bd5fbc97dc40bb61e0e2b9cc97ed0083f37f6 ) in which we set the default to 2048 and the immediately reset it to 1024. Without this, building Spark can run out of permgen space on my machine. Author: Reynold Xin Closes #103 from rxin/sbt and squashes the following commits: 8829c34 [Reynold Xin] Allow sbt to use more than 1G of heap. --- sbt/sbt-launch-lib.bash | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sbt/sbt-launch-lib.bash b/sbt/sbt-launch-lib.bash index 00a6b41013e5f..64e40a88206be 100755 --- a/sbt/sbt-launch-lib.bash +++ b/sbt/sbt-launch-lib.bash @@ -105,7 +105,7 @@ get_mem_opts () { local mem=${1:-2048} local perm=$(( $mem / 4 )) (( $perm > 256 )) || perm=256 - (( $perm < 1024 )) || perm=1024 + (( $perm < 4096 )) || perm=4096 local codecache=$(( $perm / 2 )) echo "-Xms${mem}m -Xmx${mem}m -XX:MaxPermSize=${perm}m -XX:ReservedCodeCacheSize=${codecache}m" From 0b7b7fd45cd9037d23cb090e62be3ff075214fe7 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Fri, 7 Mar 2014 23:26:46 -0800 Subject: [PATCH 064/397] [SPARK-1194] Fix the same-RDD rule for cache replacement SPARK-1194: https://spark-project.atlassian.net/browse/SPARK-1194 In the current implementation, when selecting candidate blocks to be swapped out, once we find a block from the same RDD that the block to be stored belongs to, cache eviction fails and aborts. In this PR, we keep selecting blocks *not* from the RDD that the block to be stored belongs to until either enough free space can be ensured (cache eviction succeeds) or all such blocks are checked (cache eviction fails). Author: Cheng Lian Closes #96 from liancheng/fix-spark-1194 and squashes the following commits: 2524ab9 [Cheng Lian] Added regression test case for SPARK-1194 6e40c22 [Cheng Lian] Remove redundant comments 40cdcb2 [Cheng Lian] Bug fix, and addressed PR comments from @mridulm 62c92ac [Cheng Lian] Fixed SPARK-1194 https://spark-project.atlassian.net/browse/SPARK-1194 --- .../org/apache/spark/storage/MemoryStore.scala | 11 +++++------ .../apache/spark/storage/BlockManagerSuite.scala | 14 ++++++++++++++ 2 files changed, 19 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index b89212eaabf6c..38836d44b04e8 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -236,13 +236,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) while (maxMemory - (currentMemory - selectedMemory) < space && iterator.hasNext) { val pair = iterator.next() val blockId = pair.getKey - if (rddToAdd.isDefined && rddToAdd == getRddId(blockId)) { - logInfo("Will not store " + blockIdToAdd + " as it would require dropping another " + - "block from the same RDD") - return false + if (rddToAdd.isEmpty || rddToAdd != getRddId(blockId)) { + selectedBlocks += blockId + selectedMemory += pair.getValue.size } - selectedBlocks += blockId - selectedMemory += pair.getValue.size } } @@ -264,6 +261,8 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } return true } else { + logInfo(s"Will not store $blockIdToAdd as it would require dropping another block " + + "from the same RDD") return false } } diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 121e47c7b1b41..1036b9f34e9dd 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -662,4 +662,18 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.getSingle("a1") == None, "a1 should not be in store") } } + + test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + store.putSingle(rdd(1, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + // Access rdd_1_0 to ensure it's not least recently used. + assert(store.getSingle(rdd(1, 0)).isDefined, "rdd_1_0 was not in store") + // According to the same-RDD rule, rdd_1_0 should be replaced here. + store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) + // rdd_1_0 should have been replaced, even it's not least recently used. + assert(store.memoryStore.contains(rdd(0, 0)), "rdd_0_0 was not in store") + assert(store.memoryStore.contains(rdd(0, 1)), "rdd_0_1 was not in store") + assert(!store.memoryStore.contains(rdd(1, 0)), "rdd_1_0 was in store") + } } From c2834ec081df392ca501a75b5af06efaa5448509 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sat, 8 Mar 2014 12:40:26 -0800 Subject: [PATCH 065/397] Update junitxml plugin to the latest version to avoid recompilation in every SBT command. Author: Reynold Xin Closes #104 from rxin/junitxml and squashes the following commits: 67ef7bf [Reynold Xin] Update junitxml plugin to the latest version to avoid recompilation in every SBT command. --- project/plugins.sbt | 1 + project/project/SparkPluginBuild.scala | 26 -------------------------- 2 files changed, 1 insertion(+), 26 deletions(-) delete mode 100644 project/project/SparkPluginBuild.scala diff --git a/project/plugins.sbt b/project/plugins.sbt index 914f2e05a402a..32bc044a93221 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -19,3 +19,4 @@ addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.4.0") +addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.0") diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala deleted file mode 100644 index a88a5e14539ec..0000000000000 --- a/project/project/SparkPluginBuild.scala +++ /dev/null @@ -1,26 +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. - */ - -import sbt._ - -object SparkPluginDef extends Build { - lazy val root = Project("plugins", file(".")) dependsOn(junitXmlListener) - /* This is not published in a Maven repository, so we get it from GitHub directly */ - lazy val junitXmlListener = uri( - "https://github.com/chenkelmann/junit_xml_listener.git#3f8029fbfda54dc7a68b1afd2f885935e1090016" - ) -} From e59a3b6c415b95e8137f5a154716b12653a8aed0 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 8 Mar 2014 16:02:42 -0800 Subject: [PATCH 066/397] SPARK-1190: Do not initialize log4j if slf4j log4j backend is not being used Author: Patrick Wendell Closes #107 from pwendell/logging and squashes the following commits: be21c11 [Patrick Wendell] Logging fix --- core/src/main/scala/org/apache/spark/Logging.scala | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index b749e5414dab6..7423082e34f47 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -19,6 +19,7 @@ package org.apache.spark import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} +import org.slf4j.impl.StaticLoggerBinder /** * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows @@ -101,9 +102,11 @@ trait Logging { } private def initializeLogging() { - // If Log4j doesn't seem initialized, load a default properties file + // If Log4j is being used, but is not initialized, load a default properties file + val binder = StaticLoggerBinder.getSingleton + val usingLog4j = binder.getLoggerFactoryClassStr.endsWith("Log4jLoggerFactory") val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements - if (!log4jInitialized) { + if (!log4jInitialized && usingLog4j) { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" val classLoader = this.getClass.getClassLoader Option(classLoader.getResource(defaultLogProps)) match { From 52834d761b059264214dfc6a1f9c70b8bc7ec089 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 9 Mar 2014 11:08:39 -0700 Subject: [PATCH 067/397] SPARK-929: Fully deprecate usage of SPARK_MEM (Continued from old repo, prior discussion at https://github.com/apache/incubator-spark/pull/615) This patch cements our deprecation of the SPARK_MEM environment variable by replacing it with three more specialized variables: SPARK_DAEMON_MEMORY, SPARK_EXECUTOR_MEMORY, and SPARK_DRIVER_MEMORY The creation of the latter two variables means that we can safely set driver/job memory without accidentally setting the executor memory. Neither is public. SPARK_EXECUTOR_MEMORY is only used by the Mesos scheduler (and set within SparkContext). The proper way of configuring executor memory is through the "spark.executor.memory" property. SPARK_DRIVER_MEMORY is the new way of specifying the amount of memory run by jobs launched by spark-class, without possibly affecting executor memory. Other memory considerations: - The repl's memory can be set through the "--drivermem" command-line option, which really just sets SPARK_DRIVER_MEMORY. - run-example doesn't use spark-class, so the only way to modify examples' memory is actually an unusual use of SPARK_JAVA_OPTS (which is normally overriden in all cases by spark-class). This patch also fixes a lurking bug where spark-shell misused spark-class (the first argument is supposed to be the main class name, not java options), as well as a bug in the Windows spark-class2.cmd. I have not yet tested this patch on either Windows or Mesos, however. Author: Aaron Davidson Closes #99 from aarondav/sparkmem and squashes the following commits: 9df4c68 [Aaron Davidson] SPARK-929: Fully deprecate usage of SPARK_MEM --- bin/spark-class | 48 +++++++++++-------- bin/spark-class2.cmd | 47 +++++++++++++----- bin/spark-shell | 28 +++++------ .../scala/org/apache/spark/SparkContext.scala | 20 ++++---- .../scala/org/apache/spark/util/Utils.scala | 2 - docs/tuning.md | 2 +- python/pyspark/java_gateway.py | 2 +- 7 files changed, 90 insertions(+), 59 deletions(-) diff --git a/bin/spark-class b/bin/spark-class index c4225a392d6da..229ae2cebbab3 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -40,34 +40,46 @@ if [ -z "$1" ]; then exit 1 fi -# If this is a standalone cluster daemon, reset SPARK_JAVA_OPTS and SPARK_MEM to reasonable -# values for that; it doesn't need a lot -if [ "$1" = "org.apache.spark.deploy.master.Master" -o "$1" = "org.apache.spark.deploy.worker.Worker" ]; then - SPARK_MEM=${SPARK_DAEMON_MEMORY:-512m} - SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" - # Do not overwrite SPARK_JAVA_OPTS environment variable in this script - OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS" # Empty by default -else - OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" +if [ -n "$SPARK_MEM" ]; then + echo "Warning: SPARK_MEM is deprecated, please use a more specific config option" + echo "(e.g., spark.executor.memory or SPARK_DRIVER_MEMORY)." fi +# Use SPARK_MEM or 512m as the default memory, to be overridden by specific options +DEFAULT_MEM=${SPARK_MEM:-512m} + +SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" -# Add java opts for master, worker, executor. The opts maybe null +# Add java opts and memory settings for master, worker, executors, and repl. case "$1" in + # Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. 'org.apache.spark.deploy.master.Master') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_MASTER_OPTS" + OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS" + OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} ;; 'org.apache.spark.deploy.worker.Worker') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_WORKER_OPTS" + OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS" + OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} ;; + + # Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. 'org.apache.spark.executor.CoarseGrainedExecutorBackend') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" + OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} ;; 'org.apache.spark.executor.MesosExecutorBackend') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_EXECUTOR_OPTS" + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_EXECUTOR_OPTS" + OUR_JAVA_MEM=${SPARK_EXECUTOR_MEMORY:-$DEFAULT_MEM} ;; + + # All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS. 'org.apache.spark.repl.Main') - OUR_JAVA_OPTS="$OUR_JAVA_OPTS $SPARK_REPL_OPTS" + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS $SPARK_REPL_OPTS" + OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} + ;; + *) + OUR_JAVA_OPTS="$SPARK_JAVA_OPTS" + OUR_JAVA_MEM=${SPARK_DRIVER_MEMORY:-$DEFAULT_MEM} ;; esac @@ -83,14 +95,10 @@ else fi fi -# Set SPARK_MEM if it isn't already set since we also use it for this process -SPARK_MEM=${SPARK_MEM:-512m} -export SPARK_MEM - # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$OUR_JAVA_OPTS" JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" -JAVA_OPTS="$JAVA_OPTS -Xms$SPARK_MEM -Xmx$SPARK_MEM" +JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index 80818c78ec24b..f488cfdbeceb6 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -34,22 +34,45 @@ if not "x%1"=="x" goto arg_given goto exit :arg_given -set RUNNING_DAEMON=0 -if "%1"=="spark.deploy.master.Master" set RUNNING_DAEMON=1 -if "%1"=="spark.deploy.worker.Worker" set RUNNING_DAEMON=1 -if "x%SPARK_DAEMON_MEMORY%" == "x" set SPARK_DAEMON_MEMORY=512m +if not "x%SPARK_MEM%"=="x" ( + echo Warning: SPARK_MEM is deprecated, please use a more specific config option + echo e.g., spark.executor.memory or SPARK_DRIVER_MEMORY. +) + +rem Use SPARK_MEM or 512m as the default memory, to be overridden by specific options +set OUR_JAVA_MEM=%SPARK_MEM% +if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m + set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true -if "%RUNNING_DAEMON%"=="1" set SPARK_MEM=%SPARK_DAEMON_MEMORY% -rem Do not overwrite SPARK_JAVA_OPTS environment variable in this script -if "%RUNNING_DAEMON%"=="0" set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% -if "%RUNNING_DAEMON%"=="1" set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -rem Figure out how much memory to use per executor and set it as an environment -rem variable so that our process sees it and can report it to Mesos -if "x%SPARK_MEM%"=="x" set SPARK_MEM=512m +rem Add java opts and memory settings for master, worker, executors, and repl. +rem Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. +if "%1"=="org.apache.spark.deploy.master.Master" ( + set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS% + if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% +) else if "%1"=="org.apache.spark.deploy.worker.Worker" ( + set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS% + if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% + +rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. +) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" ( + set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS% + if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY% +) else if "%1"=="org.apache.spark.executor.MesosExecutorBackend" ( + set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_EXECUTOR_OPTS% + if not "x%SPARK_EXECUTOR_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_EXECUTOR_MEMORY% + +rem All drivers use SPARK_JAVA_OPTS + SPARK_DRIVER_MEMORY. The repl also uses SPARK_REPL_OPTS. +) else if "%1"=="org.apache.spark.repl.Main" ( + set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% %SPARK_REPL_OPTS% + if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% +) else ( + set OUR_JAVA_OPTS=%SPARK_JAVA_OPTS% + if not "x%SPARK_DRIVER_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DRIVER_MEMORY% +) rem Set JAVA_OPTS to be able to load native libraries and to set heap size -set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%SPARK_MEM% -Xmx%SPARK_MEM% +set JAVA_OPTS=%OUR_JAVA_OPTS% -Djava.library.path=%SPARK_LIBRARY_PATH% -Xms%OUR_JAVA_MEM% -Xmx%OUR_JAVA_MEM% rem Attention: when changing the way the JAVA_OPTS are assembled, the change must be reflected in ExecutorRunner.scala! rem Test whether the user has built Spark diff --git a/bin/spark-shell b/bin/spark-shell index 2bff06cf70051..7d3fe3aca7f1d 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -45,13 +45,11 @@ if [ "$1" = "--help" ] || [ "$1" = "-h" ]; then exit fi -SPARK_SHELL_OPTS="" - for o in "$@"; do if [ "$1" = "-c" -o "$1" = "--cores" ]; then shift if [[ "$1" =~ $CORE_PATTERN ]]; then - SPARK_SHELL_OPTS="$SPARK_SHELL_OPTS -Dspark.cores.max=$1" + SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1" shift else echo "ERROR: wrong format for -c/--cores" @@ -61,7 +59,7 @@ for o in "$@"; do if [ "$1" = "-em" -o "$1" = "--execmem" ]; then shift if [[ $1 =~ $MEM_PATTERN ]]; then - SPARK_SHELL_OPTS="$SPARK_SHELL_OPTS -Dspark.executor.memory=$1" + SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.executor.memory=$1" shift else echo "ERROR: wrong format for --execmem/-em" @@ -71,7 +69,7 @@ for o in "$@"; do if [ "$1" = "-dm" -o "$1" = "--drivermem" ]; then shift if [[ $1 =~ $MEM_PATTERN ]]; then - export SPARK_MEM=$1 + export SPARK_DRIVER_MEMORY=$1 shift else echo "ERROR: wrong format for --drivermem/-dm" @@ -125,16 +123,18 @@ if [[ ! $? ]]; then fi if $cygwin; then - # Workaround for issue involving JLine and Cygwin - # (see http://sourceforge.net/p/jline/bugs/40/). - # If you're using the Mintty terminal emulator in Cygwin, may need to set the - # "Backspace sends ^H" setting in "Keys" section of the Mintty options - # (see https://github.com/sbt/sbt/issues/562). - stty -icanon min 1 -echo > /dev/null 2>&1 - $FWDIR/bin/spark-class -Djline.terminal=unix $SPARK_SHELL_OPTS org.apache.spark.repl.Main "$@" - stty icanon echo > /dev/null 2>&1 + # Workaround for issue involving JLine and Cygwin + # (see http://sourceforge.net/p/jline/bugs/40/). + # If you're using the Mintty terminal emulator in Cygwin, may need to set the + # "Backspace sends ^H" setting in "Keys" section of the Mintty options + # (see https://github.com/sbt/sbt/issues/562). + stty -icanon min 1 -echo > /dev/null 2>&1 + export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" + $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + stty icanon echo > /dev/null 2>&1 else - $FWDIR/bin/spark-class $SPARK_SHELL_OPTS org.apache.spark.repl.Main "$@" + export SPARK_REPL_OPTS + $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" fi # record the exit status lest it be overwritten: diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index ce25573834829..cdc0e5a34240e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -162,19 +162,20 @@ class SparkContext( jars.foreach(addJar) } + def warnSparkMem(value: String): String = { + logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + + "deprecated, please use spark.executor.memory instead.") + value + } + private[spark] val executorMemory = conf.getOption("spark.executor.memory") - .orElse(Option(System.getenv("SPARK_MEM"))) + .orElse(Option(System.getenv("SPARK_EXECUTOR_MEMORY"))) + .orElse(Option(System.getenv("SPARK_MEM")).map(warnSparkMem)) .map(Utils.memoryStringToMb) .getOrElse(512) - if (!conf.contains("spark.executor.memory") && sys.env.contains("SPARK_MEM")) { - logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + - "deprecated, instead use spark.executor.memory") - } - // Environment variables to pass to our executors private[spark] val executorEnvs = HashMap[String, String]() - // Note: SPARK_MEM is included for Mesos, but overwritten for standalone mode in ExecutorRunner for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS"); value <- Option(System.getenv(key))) { executorEnvs(key) = value @@ -185,8 +186,9 @@ class SparkContext( value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { executorEnvs(envKey) = value } - // Since memory can be set with a system property too, use that - executorEnvs("SPARK_MEM") = executorMemory + "m" + // The Mesos scheduler backend relies on this environment variable to set executor memory. + // TODO: Set this only in the Mesos scheduler. + executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" executorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 0eb2f78b730f6..53458b6660fab 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -532,8 +532,6 @@ private[spark] object Utils extends Logging { /** * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. - * This is used to figure out how much memory to claim from Mesos based on the SPARK_MEM - * environment variable. */ def memoryStringToMb(str: String): Int = { val lower = str.toLowerCase diff --git a/docs/tuning.md b/docs/tuning.md index 26ff1325bb59c..093df3187a789 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -163,7 +163,7 @@ their work directories), *not* on your driver program. **Cache Size Tuning** One important configuration parameter for GC is the amount of memory that should be used for caching RDDs. -By default, Spark uses 60% of the configured executor memory (`spark.executor.memory` or `SPARK_MEM`) to +By default, Spark uses 60% of the configured executor memory (`spark.executor.memory`) to cache RDDs. This means that 40% of memory is available for any objects created during task execution. In case your tasks slow down and you find that your JVM is garbage-collecting frequently or running out of diff --git a/python/pyspark/java_gateway.py b/python/pyspark/java_gateway.py index c15add5237507..6a16756e0576d 100644 --- a/python/pyspark/java_gateway.py +++ b/python/pyspark/java_gateway.py @@ -29,7 +29,7 @@ def launch_gateway(): # Launch the Py4j gateway using Spark's run command so that we pick up the - # proper classpath and SPARK_MEM settings from spark-env.sh + # proper classpath and settings from spark-env.sh on_windows = platform.system() == "Windows" script = "./bin/spark-class.cmd" if on_windows else "./bin/spark-class" command = [os.path.join(SPARK_HOME, script), "py4j.GatewayServer", From f6f9d02e85d17da2f742ed0062f1648a9293e73c Mon Sep 17 00:00:00 2001 From: Jiacheng Guo Date: Sun, 9 Mar 2014 11:37:44 -0700 Subject: [PATCH 068/397] Add timeout for fetch file Currently, when fetch a file, the connection's connect timeout and read timeout is based on the default jvm setting, in this change, I change it to use spark.worker.timeout. This can be usefull, when the connection status between worker is not perfect. And prevent prematurely remove task set. Author: Jiacheng Guo Closes #98 from guojc/master and squashes the following commits: abfe698 [Jiacheng Guo] add space according request 2a37c34 [Jiacheng Guo] Add timeout for fetch file Currently, when fetch a file, the connection's connect timeout and read timeout is based on the default jvm setting, in this change, I change it to use spark.worker.timeout. This can be usefull, when the connection status between worker is not perfect. And prevent prematurely remove task set. --- core/src/main/scala/org/apache/spark/util/Utils.scala | 4 ++++ docs/configuration.md | 9 +++++++++ 2 files changed, 13 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 53458b6660fab..ac376fc403ada 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -278,6 +278,10 @@ private[spark] object Utils extends Logging { uc = new URL(url).openConnection() } + val timeout = conf.getInt("spark.files.fetchTimeout", 60) * 1000 + uc.setConnectTimeout(timeout) + uc.setReadTimeout(timeout) + uc.connect() val in = uc.getInputStream(); val out = new FileOutputStream(tempFile) Utils.copyStream(in, out, true) diff --git a/docs/configuration.md b/docs/configuration.md index 913c653b0dac4..8f6cb02911de5 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -523,6 +523,15 @@ Apart from these, the following properties are also available, and may be useful Whether to overwrite files added through SparkContext.addFile() when the target file exists and its contents do not match those of the source. + + + spark.files.fetchTimeout + false + + Communication timeout to use when fetching files added through SparkContext.addFile() from + the driver. + + spark.authenticate false From faf4cad1debb76148facc008e0a3308ac96eee7a Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 9 Mar 2014 11:57:06 -0700 Subject: [PATCH 069/397] Fix markup errors introduced in #33 (SPARK-1189) These were causing errors on the configuration page. Author: Patrick Wendell Closes #111 from pwendell/master and squashes the following commits: 8467a86 [Patrick Wendell] Fix markup errors introduced in #33 (SPARK-1189) --- docs/configuration.md | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/docs/configuration.md b/docs/configuration.md index 8f6cb02911de5..a006224d5080c 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -147,13 +147,13 @@ Apart from these, the following properties are also available, and may be useful How many stages the Spark UI remembers before garbage collecting. - + spark.ui.filters None Comma separated list of filter class names to apply to the Spark web ui. The filter should be a standard javax servlet Filter. Parameters to each filter can also be specified by setting a - java system property of spark..params='param1=value1,param2=value2' + java system property of spark.<class name of filter>.params='param1=value1,param2=value2' (e.g.-Dspark.ui.filters=com.test.filter1 -Dspark.com.test.filter1.params='param1=foo,param2=testing') @@ -515,7 +515,7 @@ Apart from these, the following properties are also available, and may be useful the whole cluster by default.
    Note: this setting needs to be configured in the standalone cluster master, not in individual applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. - + spark.files.overwrite From b9be160951b9e7a7e801009e9d6ee6c2b5d2d47e Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 9 Mar 2014 13:17:07 -0700 Subject: [PATCH 070/397] SPARK-782 Clean up for ASM dependency. This makes two changes. 1) Spark uses the shaded version of asm that is (conveniently) published with Kryo. 2) Existing exclude rules around asm are updated to reflect the new groupId of `org.ow2.asm`. This made all of the old rules not work with newer Hadoop versions that pull in new asm versions. Author: Patrick Wendell Closes #100 from pwendell/asm and squashes the following commits: 9235f3f [Patrick Wendell] SPARK-782 Clean up for ASM dependency. --- core/pom.xml | 4 -- .../apache/spark/util/ClosureCleaner.scala | 4 +- .../spark/graphx/util/BytecodeUtils.scala | 4 +- pom.xml | 41 ++++++++++++++++--- project/SparkBuild.scala | 20 ++++----- .../spark/repl/ExecutorClassLoader.scala | 5 ++- 6 files changed, 53 insertions(+), 25 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 3e6e98cd2cf92..4d7d41a9714d7 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -102,10 +102,6 @@ org.xerial.snappy snappy-java
    - - org.ow2.asm - asm - com.twitter chill_${scala.binary.version} diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index 681d0a30cb3f8..a8d20ee332355 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -22,8 +22,8 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.Map import scala.collection.mutable.Set -import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} -import org.objectweb.asm.Opcodes._ +import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} +import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ import org.apache.spark.Logging diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index d1528e2f07cf2..014a7335f85cc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -23,8 +23,8 @@ import scala.collection.mutable.HashSet import org.apache.spark.util.Utils -import org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} -import org.objectweb.asm.Opcodes._ +import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor} +import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ /** diff --git a/pom.xml b/pom.xml index 3b863856e4634..f0c877dcfe7b2 100644 --- a/pom.xml +++ b/pom.xml @@ -221,11 +221,6 @@ snappy-java 1.0.5 - - org.ow2.asm - asm - 4.0 - com.clearspring.analytics stream @@ -245,11 +240,31 @@ com.twitter chill_${scala.binary.version} 0.3.1 + + + org.ow2.asm + asm + + + org.ow2.asm + asm-commons + + com.twitter chill-java 0.3.1 + + + org.ow2.asm + asm + + + org.ow2.asm + asm-commons + + ${akka.group} @@ -435,6 +450,10 @@ asm asm + + org.ow2.asm + asm + org.jboss.netty netty @@ -474,6 +493,10 @@ asm asm + + org.ow2.asm + asm + org.jboss.netty netty @@ -489,6 +512,10 @@ asm asm + + org.ow2.asm + asm + org.jboss.netty netty @@ -505,6 +532,10 @@ asm asm + + org.ow2.asm + asm + org.jboss.netty netty diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 138aad7561043..8fa220c413291 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -257,7 +257,8 @@ object SparkBuild extends Build { val slf4jVersion = "1.7.5" val excludeNetty = ExclusionRule(organization = "org.jboss.netty") - val excludeAsm = ExclusionRule(organization = "asm") + val excludeAsm = ExclusionRule(organization = "org.ow2.asm") + val excludeOldAsm = ExclusionRule(organization = "asm") val excludeCommonsLogging = ExclusionRule(organization = "commons-logging") val excludeSLF4J = ExclusionRule(organization = "org.slf4j") val excludeScalap = ExclusionRule(organization = "org.scala-lang", artifact = "scalap") @@ -280,7 +281,6 @@ object SparkBuild extends Build { "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 "com.ning" % "compress-lzf" % "1.0.0", "org.xerial.snappy" % "snappy-java" % "1.0.5", - "org.ow2.asm" % "asm" % "4.0", "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", @@ -291,15 +291,15 @@ object SparkBuild extends Build { "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J), + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", "com.codahale.metrics" % "metrics-ganglia" % "3.0.0", "com.codahale.metrics" % "metrics-graphite" % "3.0.0", - "com.twitter" %% "chill" % "0.3.1", - "com.twitter" % "chill-java" % "0.3.1", + "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), + "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro @@ -320,7 +320,7 @@ object SparkBuild extends Build { name := "spark-examples", libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", - "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging), + "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), "org.apache.cassandra" % "cassandra-all" % "1.2.6" exclude("com.google.guava", "guava") exclude("com.googlecode.concurrentlinkedhashmap", "concurrentlinkedhashmap-lru") @@ -397,10 +397,10 @@ object SparkBuild extends Build { def yarnEnabledSettings = Seq( libraryDependencies ++= Seq( // Exclude rule required for all ? - "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm), - "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeNetty, excludeAsm), - "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeNetty, excludeAsm), - "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm) + "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-api" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-common" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm), + "org.apache.hadoop" % "hadoop-yarn-client" % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeOldAsm) ) ) diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index 1aa94079fd0ae..ee972887feda6 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -27,8 +27,9 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkEnv import org.apache.spark.util.Utils -import org.objectweb.asm._ -import org.objectweb.asm.Opcodes._ + +import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ +import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ /** From 5d98cfc1c8fb17fbbeacc7192ac21c0b038cbd16 Mon Sep 17 00:00:00 2001 From: Chen Chao Date: Sun, 9 Mar 2014 22:42:12 -0700 Subject: [PATCH 071/397] maintain arbitrary state data for each key RT Author: Chen Chao Closes #114 from CrazyJvm/patch-1 and squashes the following commits: dcb0df5 [Chen Chao] maintain arbitrary state data for each key --- docs/streaming-programming-guide.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 2a56cf07d0cfc..f9904d45013f6 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -539,7 +539,7 @@ common ones are as follows. updateStateByKey(func) Return a new "state" DStream where the state for each key is updated by applying the given function on the previous state of the key and the new values for the key. This can be - used to maintain arbitrary state data for each ket. + used to maintain arbitrary state data for each key. From e1e09e0ef6b18e034727403d81747d899b042219 Mon Sep 17 00:00:00 2001 From: Prabin Banka Date: Mon, 10 Mar 2014 13:27:00 -0700 Subject: [PATCH 072/397] SPARK-977 Added Python RDD.zip function was raised earlier as a part of apache/incubator-spark#486 Author: Prabin Banka Closes #76 from prabinb/python-api-zip and squashes the following commits: b1a31a0 [Prabin Banka] Added Python RDD.zip function --- python/pyspark/rdd.py | 20 +++++++++++++++++++- python/pyspark/serializers.py | 29 ++++++++++++++++++++++++++++- 2 files changed, 47 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index e72f57d9d1ab0..5ab27ff4029d8 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -30,7 +30,7 @@ import warnings from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ - BatchedSerializer, CloudPickleSerializer, pack_long + BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long from pyspark.join import python_join, python_left_outer_join, \ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter @@ -1081,6 +1081,24 @@ def coalesce(self, numPartitions, shuffle=False): jrdd = self._jrdd.coalesce(numPartitions) return RDD(jrdd, self.ctx, self._jrdd_deserializer) + def zip(self, other): + """ + Zips this RDD with another one, returning key-value pairs with the first element in each RDD + second element in each RDD, etc. Assumes that the two RDDs have the same number of + partitions and the same number of elements in each partition (e.g. one was made through + a map on the other). + + >>> x = sc.parallelize(range(0,5)) + >>> y = sc.parallelize(range(1000, 1005)) + >>> x.zip(y).collect() + [(0, 1000), (1, 1001), (2, 1002), (3, 1003), (4, 1004)] + """ + pairRDD = self._jrdd.zip(other._jrdd) + deserializer = PairDeserializer(self._jrdd_deserializer, + other._jrdd_deserializer) + return RDD(pairRDD, self.ctx, deserializer) + + # TODO: `lookup` is disabled because we can't make direct comparisons based # on the key; we need to compare the hash of the key to the hash of the # keys in the pairs. This could be an expensive operation, since those diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 8c6ad79059c23..12c63f186a2b7 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -204,7 +204,7 @@ def __init__(self, key_ser, val_ser): self.key_ser = key_ser self.val_ser = val_ser - def load_stream(self, stream): + def prepare_keys_values(self, stream): key_stream = self.key_ser._load_stream_without_unbatching(stream) val_stream = self.val_ser._load_stream_without_unbatching(stream) key_is_batched = isinstance(self.key_ser, BatchedSerializer) @@ -212,6 +212,10 @@ def load_stream(self, stream): for (keys, vals) in izip(key_stream, val_stream): keys = keys if key_is_batched else [keys] vals = vals if val_is_batched else [vals] + yield (keys, vals) + + def load_stream(self, stream): + for (keys, vals) in self.prepare_keys_values(stream): for pair in product(keys, vals): yield pair @@ -224,6 +228,29 @@ def __str__(self): (str(self.key_ser), str(self.val_ser)) +class PairDeserializer(CartesianDeserializer): + """ + Deserializes the JavaRDD zip() of two PythonRDDs. + """ + + def __init__(self, key_ser, val_ser): + self.key_ser = key_ser + self.val_ser = val_ser + + def load_stream(self, stream): + for (keys, vals) in self.prepare_keys_values(stream): + for pair in izip(keys, vals): + yield pair + + def __eq__(self, other): + return isinstance(other, PairDeserializer) and \ + self.key_ser == other.key_ser and self.val_ser == other.val_ser + + def __str__(self): + return "PairDeserializer<%s, %s>" % \ + (str(self.key_ser), str(self.val_ser)) + + class NoOpSerializer(FramedSerializer): def loads(self, obj): return obj From f5518989b67a0941ca79368e73811895a5fa8669 Mon Sep 17 00:00:00 2001 From: jyotiska Date: Mon, 10 Mar 2014 13:34:49 -0700 Subject: [PATCH 073/397] [SPARK-972] Added detailed callsite info for ValueError in context.py (resubmitted) Author: jyotiska Closes #34 from jyotiska/pyspark_code and squashes the following commits: c9439be [jyotiska] replaced dict with namedtuple a6bf4cd [jyotiska] added callsite info for context.py --- python/pyspark/context.py | 16 +++++++++++++++- python/pyspark/rdd.py | 21 ++++++++++++++------- 2 files changed, 29 insertions(+), 8 deletions(-) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index c9f42d3aacb58..bf2454fd7e38e 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -20,6 +20,7 @@ import sys from threading import Lock from tempfile import NamedTemporaryFile +from collections import namedtuple from pyspark import accumulators from pyspark.accumulators import Accumulator @@ -29,6 +30,7 @@ from pyspark.java_gateway import launch_gateway from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer from pyspark.storagelevel import StorageLevel +from pyspark import rdd from pyspark.rdd import RDD from py4j.java_collections import ListConverter @@ -83,6 +85,11 @@ def __init__(self, master=None, appName=None, sparkHome=None, pyFiles=None, ... ValueError:... """ + if rdd._extract_concise_traceback() is not None: + self._callsite = rdd._extract_concise_traceback() + else: + tempNamedTuple = namedtuple("Callsite", "function file linenum") + self._callsite = tempNamedTuple(function=None, file=None, linenum=None) SparkContext._ensure_initialized(self, gateway=gateway) self.environment = environment or {} @@ -169,7 +176,14 @@ def _ensure_initialized(cls, instance=None, gateway=None): if instance: if SparkContext._active_spark_context and SparkContext._active_spark_context != instance: - raise ValueError("Cannot run multiple SparkContexts at once") + currentMaster = SparkContext._active_spark_context.master + currentAppName = SparkContext._active_spark_context.appName + callsite = SparkContext._active_spark_context._callsite + + # Raise error if there is already a running Spark context + raise ValueError("Cannot run multiple SparkContexts at once; existing SparkContext(app=%s, master=%s)" \ + " created by %s at %s:%s " \ + % (currentAppName, currentMaster, callsite.function, callsite.file, callsite.linenum)) else: SparkContext._active_spark_context = instance diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 5ab27ff4029d8..e1043ad564611 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -18,6 +18,7 @@ from base64 import standard_b64encode as b64enc import copy from collections import defaultdict +from collections import namedtuple from itertools import chain, ifilter, imap import operator import os @@ -42,12 +43,14 @@ __all__ = ["RDD"] def _extract_concise_traceback(): + """ + This function returns the traceback info for a callsite, returns a dict + with function name, file name and line number + """ tb = traceback.extract_stack() + callsite = namedtuple("Callsite", "function file linenum") if len(tb) == 0: - return "I'm lost!" - # HACK: This function is in a file called 'rdd.py' in the top level of - # everything PySpark. Just trim off the directory name and assume - # everything in that tree is PySpark guts. + return None file, line, module, what = tb[len(tb) - 1] sparkpath = os.path.dirname(file) first_spark_frame = len(tb) - 1 @@ -58,16 +61,20 @@ def _extract_concise_traceback(): break if first_spark_frame == 0: file, line, fun, what = tb[0] - return "%s at %s:%d" % (fun, file, line) + return callsite(function=fun, file=file, linenum=line) sfile, sline, sfun, swhat = tb[first_spark_frame] ufile, uline, ufun, uwhat = tb[first_spark_frame-1] - return "%s at %s:%d" % (sfun, ufile, uline) + return callsite(function=sfun, file=ufile, linenum=uline) _spark_stack_depth = 0 class _JavaStackTrace(object): def __init__(self, sc): - self._traceback = _extract_concise_traceback() + tb = _extract_concise_traceback() + if tb is not None: + self._traceback = "%s at %s:%s" % (tb.function, tb.file, tb.linenum) + else: + self._traceback = "Error! Could not extract traceback info" self._context = sc def __enter__(self): From a59419c27e45f06be5143c58d48affb0a5158bdf Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 10 Mar 2014 13:37:11 -0700 Subject: [PATCH 074/397] SPARK-1168, Added foldByKey to pyspark. Author: Prashant Sharma Closes #115 from ScrapCodes/SPARK-1168/pyspark-foldByKey and squashes the following commits: db6f67e [Prashant Sharma] SPARK-1168, Added foldByKey to pyspark. --- python/pyspark/rdd.py | 14 ++++++++++++++ 1 file changed, 14 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index e1043ad564611..39916d21c76c5 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -946,7 +946,21 @@ def _mergeCombiners(iterator): combiners[k] = mergeCombiners(combiners[k], v) return combiners.iteritems() return shuffled.mapPartitions(_mergeCombiners) + + def foldByKey(self, zeroValue, func, numPartitions=None): + """ + Merge the values for each key using an associative function "func" and a neutral "zeroValue" + which may be added to the result an arbitrary number of times, and must not change + the result (e.g., 0 for addition, or 1 for multiplication.). + >>> rdd = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) + >>> from operator import add + >>> rdd.foldByKey(0, add).collect() + [('a', 2), ('b', 1)] + """ + return self.combineByKey(lambda v: func(zeroValue, v), func, func, numPartitions) + + # TODO: support variant with custom partitioner def groupByKey(self, numPartitions=None): """ From 2a5161708f4d2f743c7bd69ed3d98bb7bff46460 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 10 Mar 2014 16:28:41 -0700 Subject: [PATCH 075/397] SPARK-1205: Clean up callSite/origin/generator. This patch removes the `generator` field and simplifies + documents the tracking of callsites. There are two places where we care about call sites, when a job is run and when an RDD is created. This patch retains both of those features but does a slight refactoring and renaming to make things less confusing. There was another feature of an rdd called the `generator` which was by default the user class that in which the RDD was created. This is used exclusively in the JobLogger. It been subsumed by the ability to name a job group. The job logger can later be refectored to read the job group directly (will require some work) but for now this just preserves the default logged value of the user class. I'm not sure any users ever used the ability to override this. Author: Patrick Wendell Closes #106 from pwendell/callsite and squashes the following commits: fc1d009 [Patrick Wendell] Compile fix e17fb76 [Patrick Wendell] Review feedback: callSite -> creationSite 62e77ef [Patrick Wendell] Review feedback 576e60b [Patrick Wendell] SPARK-1205: Clean up callSite/origin/generator. --- .../scala/org/apache/spark/SparkContext.scala | 11 +++++------ .../org/apache/spark/api/java/JavaRDD.scala | 2 -- .../apache/spark/api/java/JavaRDDLike.scala | 5 ----- .../main/scala/org/apache/spark/rdd/RDD.scala | 18 ++++-------------- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../org/apache/spark/scheduler/JobLogger.scala | 10 +++------- .../org/apache/spark/scheduler/Stage.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 4 ++-- 8 files changed, 16 insertions(+), 38 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index cdc0e5a34240e..745e3fa4e85f6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -832,13 +832,12 @@ class SparkContext( setLocalProperty("externalCallSite", null) } + /** + * Capture the current user callsite and return a formatted version for printing. If the user + * has overridden the call site, this will return the user's version. + */ private[spark] def getCallSite(): String = { - val callSite = getLocalProperty("externalCallSite") - if (callSite == null) { - Utils.formatSparkCallSite - } else { - callSite - } + Option(getLocalProperty("externalCallSite")).getOrElse(Utils.formatCallSiteInfo()) } /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 91bf404631f49..01d9357a2556d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -135,8 +135,6 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) def subtract(other: JavaRDD[T], p: Partitioner): JavaRDD[T] = wrapRDD(rdd.subtract(other, p)) - def generator: String = rdd.generator - override def toString = rdd.toString /** Assign a name to this RDD */ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index af0114bee3f49..a89419bbd10e7 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -19,7 +19,6 @@ package org.apache.spark.api.java import java.util.{Comparator, List => JList} -import scala.Tuple2 import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -500,8 +499,4 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def name(): String = rdd.name - /** Reset generator */ - def setGenerator(_generator: String) = { - rdd.setGenerator(_generator) - } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3fe56963e0008..4afa7523dd802 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -126,14 +126,6 @@ abstract class RDD[T: ClassTag]( this } - /** User-defined generator of this RDD*/ - @transient var generator = Utils.getCallSiteInfo.firstUserClass - - /** Reset generator*/ - def setGenerator(_generator: String) = { - generator = _generator - } - /** * Set this RDD's storage level to persist its values across operations after the first time * it is computed. This can only be used to assign a new storage level if the RDD does not @@ -1031,8 +1023,9 @@ abstract class RDD[T: ClassTag]( private var storageLevel: StorageLevel = StorageLevel.NONE - /** Record user function generating this RDD. */ - @transient private[spark] val origin = sc.getCallSite() + /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ + @transient private[spark] val creationSiteInfo = Utils.getCallSiteInfo + private[spark] def getCreationSite = Utils.formatCallSiteInfo(creationSiteInfo) private[spark] def elementClassTag: ClassTag[T] = classTag[T] @@ -1095,10 +1088,7 @@ abstract class RDD[T: ClassTag]( } override def toString: String = "%s%s[%d] at %s".format( - Option(name).map(_ + " ").getOrElse(""), - getClass.getSimpleName, - id, - origin) + Option(name).map(_ + " ").getOrElse(""), getClass.getSimpleName, id, getCreationSite) def toJavaRDD() : JavaRDD[T] = { new JavaRDD(this)(elementClassTag) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index dc5b25d845dc2..d83d0341c61ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -279,7 +279,7 @@ class DAGScheduler( } else { // Kind of ugly: need to register RDDs with the cache and map output tracker here // since we can't do it in the RDD constructor because # of partitions is unknown - logInfo("Registering RDD " + rdd.id + " (" + rdd.origin + ")") + logInfo("Registering RDD " + rdd.id + " (" + rdd.getCreationSite + ")") mapOutputTracker.registerShuffle(shuffleDep.shuffleId, rdd.partitions.size) } stage diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 80f9ec7d03007..01cbcc390c6cd 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -213,14 +213,10 @@ class JobLogger(val user: String, val logDirName: String) * @param indent Indent number before info */ protected def recordRddInStageGraph(jobID: Int, rdd: RDD[_], indent: Int) { + val cacheStr = if (rdd.getStorageLevel != StorageLevel.NONE) "CACHED" else "NONE" val rddInfo = - if (rdd.getStorageLevel != StorageLevel.NONE) { - "RDD_ID=" + rdd.id + " " + getRddName(rdd) + " CACHED" + " " + - rdd.origin + " " + rdd.generator - } else { - "RDD_ID=" + rdd.id + " " + getRddName(rdd) + " NONE" + " " + - rdd.origin + " " + rdd.generator - } + s"RDD_ID=$rdd.id ${getRddName(rdd)} $cacheStr " + + s"${rdd.getCreationSite} ${rdd.creationSiteInfo.firstUserClass}" jobLogInfo(jobID, indentString(indent) + rddInfo, false) rdd.dependencies.foreach { case shufDep: ShuffleDependency[_, _] => diff --git a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala index a78b0186b9eab..5c1fc30e4a557 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Stage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Stage.scala @@ -100,7 +100,7 @@ private[spark] class Stage( id } - val name = callSite.getOrElse(rdd.origin) + val name = callSite.getOrElse(rdd.getCreationSite) override def toString = "Stage " + id diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index ac376fc403ada..38a275d438959 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -719,8 +719,8 @@ private[spark] object Utils extends Logging { new CallSiteInfo(lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) } - def formatSparkCallSite = { - val callSiteInfo = getCallSiteInfo + /** Returns a printable version of the call site info suitable for logs. */ + def formatCallSiteInfo(callSiteInfo: CallSiteInfo = Utils.getCallSiteInfo) = { "%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile, callSiteInfo.firstUserLine) } From 2a2c9645e4ea08cd1408151a33d2d52f6752404a Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 10 Mar 2014 17:42:33 -0700 Subject: [PATCH 076/397] SPARK-1211. In ApplicationMaster, set spark.master system property to "y... ...arn-cluster" Author: Sandy Ryza Closes #118 from sryza/sandy-spark-1211 and squashes the following commits: d4001c7 [Sandy Ryza] SPARK-1211. In ApplicationMaster, set spark.master system property to "yarn-cluster" --- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 +++ .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 +++ 2 files changed, 6 insertions(+) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index bb574f415293a..87785cdc60c52 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -79,6 +79,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // other spark processes running on the same box System.setProperty("spark.ui.port", "0") + // when running the AM, the Spark master is always "yarn-cluster" + System.setProperty("spark.master", "yarn-cluster") + // Use priority 30 as its higher then HDFS. Its same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index b48a2d50db5ef..57d15774290dd 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -82,6 +82,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // other spark processes running on the same box System.setProperty("spark.ui.port", "0") + // when running the AM, the Spark master is always "yarn-cluster" + System.setProperty("spark.master", "yarn-cluster") + // Use priority 30 as it's higher then HDFS. It's same priority as MapReduce is using. ShutdownHookManager.get().addShutdownHook(new AppMasterShutdownHook(this), 30) From 16788a654246067fd966033b5dc9bc0d4c759b70 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 11 Mar 2014 11:16:59 -0700 Subject: [PATCH 077/397] SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues... This patch removes Ganglia integration from the default build. It allows users willing to link against LGPL code to use Ganglia by adding build flags or linking against a new Spark artifact called spark-ganglia-lgpl. This brings Spark in line with the Apache policy on LGPL code enumerated here: https://www.apache.org/legal/3party.html#options-optional Author: Patrick Wendell Closes #108 from pwendell/ganglia and squashes the following commits: 326712a [Patrick Wendell] Responding to review feedback 5f28ee4 [Patrick Wendell] SPARK-1167: Remove metrics-ganglia from default build due to LGPL issues. --- assembly/pom.xml | 10 +++++ core/pom.xml | 4 -- dev/audit-release/README.md | 11 +++++ .../src/main/scala/SparkApp.scala | 15 ++++++- dev/audit-release/sbt_app_ganglia/build.sbt | 31 +++++++++++++ .../src/main/scala/SparkApp.scala | 39 ++++++++++++++++ dev/create-release/create-release.sh | 4 +- docs/monitoring.md | 13 +++++- extras/spark-ganglia-lgpl/pom.xml | 45 +++++++++++++++++++ .../spark/metrics/sink/GangliaSink.scala | 0 pom.xml | 9 +++- project/SparkBuild.scala | 25 ++++++++--- 12 files changed, 190 insertions(+), 16 deletions(-) create mode 100644 dev/audit-release/README.md create mode 100644 dev/audit-release/sbt_app_ganglia/build.sbt create mode 100644 dev/audit-release/sbt_app_ganglia/src/main/scala/SparkApp.scala create mode 100644 extras/spark-ganglia-lgpl/pom.xml rename {core => extras/spark-ganglia-lgpl}/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala (100%) diff --git a/assembly/pom.xml b/assembly/pom.xml index 82a5985504b4e..22bbbc57d81d4 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -158,6 +158,16 @@
    + + spark-ganglia-lgpl + + + org.apache.spark + spark-ganglia-lgpl_${scala.binary.version} + ${project.version} + + + bigtop-dist + + 4.0.0 + + org.apache.spark + spark-parent + 1.0.0-SNAPSHOT + ../../pom.xml + + + + org.apache.spark + spark-ganglia-lgpl_2.10 + jar + Spark Ganglia Integration + + + + org.apache.spark + spark-core_${scala.binary.version} + ${project.version} + + + + com.codahale.metrics + metrics-ganglia + + + diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala similarity index 100% rename from core/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala rename to extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala diff --git a/pom.xml b/pom.xml index f0c877dcfe7b2..986626f029d75 100644 --- a/pom.xml +++ b/pom.xml @@ -756,12 +756,19 @@ 0.23.7 - yarn + + + + spark-ganglia-lgpl + + extras/spark-ganglia-lgpl + + java8-tests diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 8fa220c413291..b0c3bf29dfd4f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -65,7 +65,7 @@ object SparkBuild extends Build { 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(core, graphx, bagel, mllib, repl, streaming) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*) lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects") @@ -91,19 +91,26 @@ object SparkBuild extends Build { lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() - // Conditionally include the java 8 sub-project + // Include Ganglia integration if the user has enabled Ganglia + // This is isolated from the normal build due to LGPL-licensed code in the library + lazy val isGangliaEnabled = Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined + lazy val gangliaProj = Project("spark-ganglia-lgpl", file("extras/spark-ganglia-lgpl"), settings = gangliaSettings).dependsOn(core) + val maybeGanglia: Seq[ClasspathDependency] = if (isGangliaEnabled) Seq(gangliaProj) else Seq() + val maybeGangliaRef: Seq[ProjectReference] = if (isGangliaEnabled) Seq(gangliaProj) else Seq() + + // Include the Java 8 project if the JVM version is 8+ lazy val javaVersion = System.getProperty("java.specification.version") lazy val isJava8Enabled = javaVersion.toDouble >= "1.8".toDouble val maybeJava8Tests = if (isJava8Enabled) Seq[ProjectReference](java8Tests) else Seq[ProjectReference]() lazy val java8Tests = Project("java8-tests", file("extras/java8-tests"), settings = java8TestsSettings). dependsOn(core) dependsOn(streaming % "compile->compile;test->test") - // Conditionally include the yarn sub-project + // Include the YARN project if the user has enabled YARN lazy val yarnAlpha = Project("yarn-alpha", file("yarn/alpha"), settings = yarnAlphaSettings) dependsOn(core) lazy val yarn = Project("yarn", file("yarn/stable"), settings = yarnSettings) dependsOn(core) - lazy val maybeYarn = if (isYarnEnabled) Seq[ClasspathDependency](if (isNewHadoop) yarn else yarnAlpha) else Seq[ClasspathDependency]() - lazy val maybeYarnRef = if (isYarnEnabled) Seq[ProjectReference](if (isNewHadoop) yarn else yarnAlpha) else Seq[ProjectReference]() + lazy val maybeYarn: Seq[ClasspathDependency] = if (isYarnEnabled) Seq(if (isNewHadoop) yarn else yarnAlpha) else Seq() + lazy val maybeYarnRef: Seq[ProjectReference] = if (isYarnEnabled) Seq(if (isNewHadoop) yarn else yarnAlpha) else Seq() lazy val externalTwitter = Project("external-twitter", file("external/twitter"), settings = twitterSettings) .dependsOn(streaming % "compile->compile;test->test") @@ -127,7 +134,7 @@ object SparkBuild extends Build { .dependsOn(core, mllib, graphx, bagel, streaming, externalTwitter) dependsOn(allExternal: _*) // Everything except assembly, tools, java8Tests 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) ++ maybeYarnRef ++ maybeGangliaRef lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests @@ -296,7 +303,6 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-core" % "3.0.0", "com.codahale.metrics" % "metrics-jvm" % "3.0.0", "com.codahale.metrics" % "metrics-json" % "3.0.0", - "com.codahale.metrics" % "metrics-ganglia" % "3.0.0", "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), @@ -384,6 +390,11 @@ object SparkBuild extends Build { name := "spark-yarn" ) + def gangliaSettings = sharedSettings ++ Seq( + name := "spark-ganglia-lgpl", + libraryDependencies += "com.codahale.metrics" % "metrics-ganglia" % "3.0.0" + ) + def java8TestsSettings = sharedSettings ++ Seq( name := "java8-tests", javacOptions := Seq("-target", "1.8", "-source", "1.8"), From 2409af9dcf238e1ad87080a389e05a696c41dc72 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 11 Mar 2014 22:39:17 -0700 Subject: [PATCH 078/397] SPARK-1064 This reopens PR 649 from incubator-spark against the new repo Author: Sandy Ryza Closes #102 from sryza/sandy-spark-1064 and squashes the following commits: 270e490 [Sandy Ryza] Handle different application classpath variables in different versions 88b04e0 [Sandy Ryza] SPARK-1064. Make it possible to run on YARN without bundling Hadoop jars in Spark assembly --- docs/building-with-maven.md | 6 +++ pom.xml | 46 +++++++++++++++++++ .../apache/spark/deploy/yarn/ClientBase.scala | 43 ++++++++++++++++- 3 files changed, 94 insertions(+), 1 deletion(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index d3bc34e68b240..730a6e7932564 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -88,3 +88,9 @@ Running only java 8 tests and nothing else. Java 8 tests are run when -Pjava8-tests profile is enabled, they will run in spite of -DskipTests. For these tests to run your system must have a JDK 8 installation. If you have JDK 8 installed but it is not the system default, you can set JAVA_HOME to point to JDK 8 before running the tests. + +## Packaging without Hadoop dependencies for deployment on YARN ## + +The assembly jar produced by "mvn package" will, by default, include all of Spark's dependencies, including Hadoop and some of its ecosystem projects. On YARN deployments, this causes multiple versions of these to appear on executor classpaths: the version packaged in the Spark assembly and the version on each node, included with yarn.application.classpath. The "hadoop-provided" profile builds the assembly without including Hadoop-ecosystem projects, like ZooKeeper and Hadoop itself. + + diff --git a/pom.xml b/pom.xml index 986626f029d75..d2c36dc260fa4 100644 --- a/pom.xml +++ b/pom.xml @@ -807,5 +807,51 @@ + + + + hadoop-provided + + false + + + + org.apache.hadoop + hadoop-client + provided + + + org.apache.hadoop + hadoop-yarn-api + provided + + + org.apache.hadoop + hadoop-yarn-common + provided + + + org.apache.hadoop + hadoop-yarn-client + provided + + + org.apache.avro + avro + provided + + + org.apache.avro + avro-ipc + provided + + + org.apache.zookeeper + zookeeper + provided + + + + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 4b6c7db836b02..74de4293d9096 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -29,8 +29,10 @@ import org.apache.hadoop.fs._ import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.io.DataOutputBuffer import org.apache.hadoop.mapred.Master +import org.apache.hadoop.mapreduce.MRJobConfig import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation +import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords._ @@ -379,9 +381,48 @@ object ClientBase { // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { - for (c <- conf.getStrings(YarnConfiguration.YARN_APPLICATION_CLASSPATH)) { + val classpathEntries = Option(conf.getStrings( + YarnConfiguration.YARN_APPLICATION_CLASSPATH)).getOrElse( + getDefaultYarnApplicationClasspath()) + for (c <- classpathEntries) { Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim) } + + val mrClasspathEntries = Option(conf.getStrings( + "mapreduce.application.classpath")).getOrElse( + getDefaultMRApplicationClasspath()) + if (mrClasspathEntries != null) { + for (c <- mrClasspathEntries) { + Apps.addToEnvironment(env, Environment.CLASSPATH.name, c.trim) + } + } + } + + def getDefaultYarnApplicationClasspath(): Array[String] = { + try { + val field = classOf[MRJobConfig].getField("DEFAULT_YARN_APPLICATION_CLASSPATH") + field.get(null).asInstanceOf[Array[String]] + } catch { + case err: NoSuchFieldError => null + } + } + + /** + * In Hadoop 0.23, the MR application classpath comes with the YARN application + * classpath. In Hadoop 2.0, it's an array of Strings, and in 2.2+ it's a String. + * So we need to use reflection to retrieve it. + */ + def getDefaultMRApplicationClasspath(): Array[String] = { + try { + val field = classOf[MRJobConfig].getField("DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH") + if (field.getType == classOf[String]) { + StringUtils.getStrings(field.get(null).asInstanceOf[String]) + } else { + field.get(null).asInstanceOf[Array[String]] + } + } catch { + case err: NoSuchFieldError => null + } } def populateClasspath(conf: Configuration, sparkConf: SparkConf, addLog4j: Boolean, env: HashMap[String, String]) { From af7f2f10902c7b42e08797f7467dd06e4803594c Mon Sep 17 00:00:00 2001 From: prabinb Date: Tue, 11 Mar 2014 23:57:05 -0700 Subject: [PATCH 079/397] Spark-1163, Added missing Python RDD functions Author: prabinb Closes #92 from prabinb/python-api-rdd and squashes the following commits: 51129ca [prabinb] Added missing Python RDD functions Added __repr__ function to StorageLevel class. Added doctest for RDD.getStorageLevel(). --- python/pyspark/rdd.py | 42 ++++++++++++++++++++++++++++++++++ python/pyspark/storagelevel.py | 4 ++++ 2 files changed, 46 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 39916d21c76c5..0f28dbd6fcc09 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -36,6 +36,7 @@ python_right_outer_join, python_cogroup from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler +from pyspark.storagelevel import StorageLevel from py4j.java_collections import ListConverter, MapConverter @@ -1119,6 +1120,47 @@ def zip(self, other): other._jrdd_deserializer) return RDD(pairRDD, self.ctx, deserializer) + def name(self): + """ + Return the name of this RDD. + """ + name_ = self._jrdd.name() + if not name_: + return None + return name_.encode('utf-8') + + def setName(self, name): + """ + Assign a name to this RDD. + >>> rdd1 = sc.parallelize([1,2]) + >>> rdd1.setName('RDD1') + >>> rdd1.name() + 'RDD1' + """ + self._jrdd.setName(name) + + def toDebugString(self): + """ + A description of this RDD and its recursive dependencies for debugging. + """ + debug_string = self._jrdd.toDebugString() + if not debug_string: + return None + return debug_string.encode('utf-8') + + def getStorageLevel(self): + """ + Get the RDD's current storage level. + >>> rdd1 = sc.parallelize([1,2]) + >>> rdd1.getStorageLevel() + StorageLevel(False, False, False, 1) + """ + java_storage_level = self._jrdd.getStorageLevel() + storage_level = StorageLevel(java_storage_level.useDisk(), + java_storage_level.useMemory(), + java_storage_level.deserialized(), + java_storage_level.replication()) + return storage_level # TODO: `lookup` is disabled because we can't make direct comparisons based # on the key; we need to compare the hash of the key to the hash of the diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index b31f4762e69bc..c3e3a44e8e7ab 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -31,6 +31,10 @@ def __init__(self, useDisk, useMemory, deserialized, replication = 1): self.deserialized = deserialized self.replication = replication + def __repr__(self): + return "StorageLevel(%s, %s, %s, %s)" % ( + self.useDisk, self.useMemory, self.deserialized, self.replication) + StorageLevel.DISK_ONLY = StorageLevel(True, False, False) StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, 2) StorageLevel.MEMORY_ONLY = StorageLevel(False, True, True) From c8c59b326e587b7515ba4f43399c67d949df244f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 12 Mar 2014 10:32:01 -0700 Subject: [PATCH 080/397] [SPARK-1232] Fix the hadoop 0.23 yarn build Author: Thomas Graves Closes #127 from tgravescs/SPARK-1232 and squashes the following commits: c05cfd4 [Thomas Graves] Fix the hadoop 0.23 yarn build --- core/pom.xml | 12 ++++++++++++ 1 file changed, 12 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 2248f9d0446c0..a6f478b09bda0 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -30,6 +30,18 @@ jar Spark Project Core http://spark.apache.org/ + + + + yarn-alpha + + + org.apache.avro + avro + + + + From b5162f4426268fdd3acc592e636b8a59c478db9f Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 12 Mar 2014 11:25:41 -0700 Subject: [PATCH 081/397] [SPARK-1233] Fix running hadoop 0.23 due to java.lang.NoSuchFieldException: DEFAULT_M... ...APREDUCE_APPLICATION_CLASSPATH Author: Thomas Graves Closes #129 from tgravescs/SPARK-1233 and squashes the following commits: 85ff5a6 [Thomas Graves] Fix running hadoop 0.23 due to java.lang.NoSuchFieldException: DEFAULT_MAPREDUCE_APPLICATION_CLASSPATH --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 74de4293d9096..74c5e0f18e7bd 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -404,6 +404,7 @@ object ClientBase { field.get(null).asInstanceOf[Array[String]] } catch { case err: NoSuchFieldError => null + case err: NoSuchFieldException => null } } @@ -422,6 +423,7 @@ object ClientBase { } } catch { case err: NoSuchFieldError => null + case err: NoSuchFieldException => null } } From 5d1ec64e7934ad7f922cdab516fa5de690644780 Mon Sep 17 00:00:00 2001 From: liguoqiang Date: Wed, 12 Mar 2014 12:59:51 -0700 Subject: [PATCH 082/397] Fix #SPARK-1149 Bad partitioners can cause Spark to hang Author: liguoqiang Closes #44 from witgo/SPARK-1149 and squashes the following commits: 3dcdcaf [liguoqiang] Merge branch 'master' into SPARK-1149 8425395 [liguoqiang] Merge remote-tracking branch 'upstream/master' into SPARK-1149 3dad595 [liguoqiang] review comment e3e56aa [liguoqiang] Merge branch 'master' into SPARK-1149 b0d5c07 [liguoqiang] review comment d0a6005 [liguoqiang] review comment 3395ee7 [liguoqiang] Merge remote-tracking branch 'upstream/master' into SPARK-1149 ac006a3 [liguoqiang] code Formatting 3feb3a8 [liguoqiang] Merge branch 'master' into SPARK-1149 adc443e [liguoqiang] partitions check bugfix 928e1e3 [liguoqiang] Added a unit test for PairRDDFunctions.lookup with bad partitioner db6ecc5 [liguoqiang] Merge branch 'master' into SPARK-1149 1e3331e [liguoqiang] Merge branch 'master' into SPARK-1149 3348619 [liguoqiang] Optimize performance for partitions check 61e5a87 [liguoqiang] Merge branch 'master' into SPARK-1149 e68210a [liguoqiang] add partition index check to submitJob 3a65903 [liguoqiang] make the code more readable 6bb725e [liguoqiang] fix #SPARK-1149 Bad partitioners can cause Spark to hang --- .../scala/org/apache/spark/SparkContext.scala | 6 ++++++ .../apache/spark/rdd/PairRDDFunctionsSuite.scala | 16 ++++++++++++++++ 2 files changed, 22 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 745e3fa4e85f6..852ed8fe1fb91 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -852,6 +852,9 @@ class SparkContext( partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { + partitions.foreach{ p => + require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") + } val callSite = getCallSite val cleanedFunc = clean(func) logInfo("Starting job: " + callSite) @@ -955,6 +958,9 @@ class SparkContext( resultHandler: (Int, U) => Unit, resultFunc: => R): SimpleFutureAction[R] = { + partitions.foreach{ p => + require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") + } val cleanF = clean(processPartition) val callSite = getCallSite val waiter = dagScheduler.submitJob( diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index 85e8eb5dc3a1e..f9e994b13dfbc 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -373,6 +373,22 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { assert(shuffled.lookup(5) === Seq(6,7)) assert(shuffled.lookup(-1) === Seq()) } + + test("lookup with bad partitioner") { + val pairs = sc.parallelize(Array((1,2), (3,4), (5,6), (5,7))) + + val p = new Partitioner { + def numPartitions: Int = 2 + + def getPartition(key: Any): Int = key.hashCode() % 2 + } + val shuffled = pairs.partitionBy(p) + + assert(shuffled.partitioner === Some(p)) + assert(shuffled.lookup(1) === Seq(2)) + intercept[IllegalArgumentException] {shuffled.lookup(-1)} + } + } /* From b8afe3052086547879ebf28d6e36207e0d370710 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 12 Mar 2014 15:57:44 -0700 Subject: [PATCH 083/397] SPARK-1162 Added top in python. Author: Prashant Sharma Closes #93 from ScrapCodes/SPARK-1162/pyspark-top-takeOrdered and squashes the following commits: ece1fa4 [Prashant Sharma] Added top in python. --- python/pyspark/rdd.py | 25 +++++++++++++++++++++++++ 1 file changed, 25 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 0f28dbd6fcc09..6d549b40e5698 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -29,6 +29,7 @@ from tempfile import NamedTemporaryFile from threading import Thread import warnings +from heapq import heappush, heappop, heappushpop from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long @@ -660,6 +661,30 @@ def mergeMaps(m1, m2): m1[k] += v return m1 return self.mapPartitions(countPartition).reduce(mergeMaps) + + def top(self, num): + """ + Get the top N elements from a RDD. + + Note: It returns the list sorted in ascending order. + >>> sc.parallelize([10, 4, 2, 12, 3]).top(1) + [12] + >>> sc.parallelize([2, 3, 4, 5, 6]).cache().top(2) + [5, 6] + """ + def topIterator(iterator): + q = [] + for k in iterator: + if len(q) < num: + heappush(q, k) + else: + heappushpop(q, k) + yield q + + def merge(a, b): + return next(topIterator(a + b)) + + return sorted(self.mapPartitions(topIterator).reduce(merge)) def take(self, num): """ From 9032f7c0d5f1ae7985a20d54ca04c297201aae85 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 12 Mar 2014 17:43:12 -0700 Subject: [PATCH 084/397] SPARK-1160: Deprecate toArray in RDD https://spark-project.atlassian.net/browse/SPARK-1160 reported by @mateiz: "It's redundant with collect() and the name doesn't make sense in Java, where we return a List (we can't return an array due to the way Java generics work). It's also missing in Python." In this patch, I deprecated the method and changed the source files using it by replacing toArray with collect() directly Author: CodingCat Closes #105 from CodingCat/SPARK-1060 and squashes the following commits: 286f163 [CodingCat] deprecate in JavaRDDLike ee17b4e [CodingCat] add message and since 2ff7319 [CodingCat] deprecate toArray in RDD --- .../main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 1 + .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 +- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 1 + core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala | 4 ++-- .../src/main/scala/org/apache/spark/examples/SparkALS.scala | 4 ++-- .../scala/org/apache/spark/examples/mllib/SparkSVD.scala | 2 +- .../src/main/scala/org/apache/spark/mllib/linalg/SVD.scala | 4 ++-- .../test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala | 6 +++--- 8 files changed, 13 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index a89419bbd10e7..3df68d4ce508d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -283,6 +283,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return an array that contains all of the elements in this RDD. */ + @deprecated("use collect", "1.0.0") def toArray(): JList[T] = collect() /** diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 2384c8f2b6fd4..b20ed99f89f34 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -423,7 +423,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Return the key-value pairs in this RDD to the master as a Map. */ def collectAsMap(): Map[K, V] = { - val data = self.toArray() + val data = self.collect() val map = new mutable.HashMap[K, V] map.sizeHint(data.length) data.foreach { case (k, v) => map.put(k, v) } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 4afa7523dd802..b50c9963b9d2c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -658,6 +658,7 @@ abstract class RDD[T: ClassTag]( /** * Return an array that contains all of the elements in this RDD. */ + @deprecated("use collect", "1.0.0") def toArray(): Array[T] = collect() /** diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index b50307cfa49b7..4ceea557f569c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -26,13 +26,13 @@ import cern.jet.random.engine.DRand import org.apache.spark.{Partition, TaskContext} -@deprecated("Replaced by PartitionwiseSampledRDDPartition", "1.0") +@deprecated("Replaced by PartitionwiseSampledRDDPartition", "1.0.0") private[spark] class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition with Serializable { override val index: Int = prev.index } -@deprecated("Replaced by PartitionwiseSampledRDD", "1.0") +@deprecated("Replaced by PartitionwiseSampledRDD", "1.0.0") class SampledRDD[T: ClassTag]( prev: RDD[T], withReplacement: Boolean, diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 17bafc2218a31..ce4b3c8451e00 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -128,11 +128,11 @@ object SparkALS { println("Iteration " + iter + ":") ms = sc.parallelize(0 until M, slices) .map(i => update(i, msb.value(i), usb.value, Rc.value)) - .toArray + .collect() msb = sc.broadcast(ms) // Re-broadcast ms because it was updated us = sc.parallelize(0 until U, slices) .map(i => update(i, usb.value(i), msb.value, algebra.transpose(Rc.value))) - .toArray + .collect() usb = sc.broadcast(us) // Re-broadcast us because it was updated println("RMSE = " + rmse(R, ms, us)) println() diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala index 19676fcc1a2b0..ce2b133368e85 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala @@ -54,6 +54,6 @@ object SparkSVD { val s = decomposed.S.data val v = decomposed.V.data - println("singular values = " + s.toArray.mkString) + println("singular values = " + s.collect().mkString) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index 8803c4c1a07be..e4a26eeb07c60 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -109,7 +109,7 @@ object SVD { // Construct jblas A^T A locally val ata = DoubleMatrix.zeros(n, n) - for (entry <- emits.toArray) { + for (entry <- emits.collect()) { ata.put(entry._1._1, entry._1._2, entry._2) } @@ -178,7 +178,7 @@ object SVD { val s = decomposed.S.data val v = decomposed.V.data - println("Computed " + s.toArray.length + " singular values and vectors") + println("Computed " + s.collect().length + " singular values and vectors") u.saveAsTextFile(output_u) s.saveAsTextFile(output_s) v.saveAsTextFile(output_v) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala index 32f3f141cd652..a92386865a189 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala @@ -50,7 +50,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val m = matrix.m val n = matrix.n val ret = DoubleMatrix.zeros(m, n) - matrix.data.toArray.map(x => ret.put(x.i, x.j, x.mval)) + matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) ret } @@ -106,7 +106,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val u = decomposed.U val s = decomposed.S val v = decomposed.V - val retrank = s.data.toArray.length + val retrank = s.data.collect().length assert(retrank == 1, "rank returned not one") @@ -139,7 +139,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val u = decomposed.U val s = decomposed.S val v = decomposed.V - val retrank = s.data.toArray.length + val retrank = s.data.collect().length val densea = getDenseMatrix(a) val svd = Singular.sparseSVD(densea) From 31a704004f9b4ad34f92ae5c95ae6e90d0ab62c7 Mon Sep 17 00:00:00 2001 From: jianghan Date: Wed, 12 Mar 2014 19:46:12 -0700 Subject: [PATCH 085/397] Fix example bug: compile error Author: jianghan Closes #132 from pooorman/master and squashes the following commits: 54afbe0 [jianghan] Fix example bug: compile error --- .../main/java/org/apache/spark/examples/JavaLogQuery.java | 2 +- .../main/java/org/apache/spark/examples/JavaPageRank.java | 6 +++--- .../main/java/org/apache/spark/examples/JavaWordCount.java | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index a518fe2f27eb0..617e4a6d045e0 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -124,7 +124,7 @@ public Stats call(Stats stats, Stats stats2) { List, Stats>> output = counts.collect(); for (Tuple2 t : output) { - System.out.println(t._1 + "\t" + t._2); + System.out.println(t._1() + "\t" + t._2()); } System.exit(0); } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index e53925b50c2ce..eb70fb547564c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -90,8 +90,8 @@ public Double call(List rs) { @Override public Iterable> call(Tuple2, Double> s) { List> results = new ArrayList>(); - for (String n : s._1) { - results.add(new Tuple2(n, s._2 / s._1.size())); + for (String n : s._1()) { + results.add(new Tuple2(n, s._2() / s._1().size())); } return results; } @@ -109,7 +109,7 @@ public Double call(Double sum) { // Collects all URL ranks and dump them to console. List> output = ranks.collect(); for (Tuple2 tuple : output) { - System.out.println(tuple._1 + " has rank: " + tuple._2 + "."); + System.out.println(tuple._1() + " has rank: " + tuple._2() + "."); } System.exit(0); diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index fa1b977ab19f1..3ae1d8f7ca938 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -65,7 +65,7 @@ public Integer call(Integer i1, Integer i2) { List> output = counts.collect(); for (Tuple2 tuple : output) { - System.out.println(tuple._1 + ": " + tuple._2); + System.out.println(tuple._1() + ": " + tuple._2()); } System.exit(0); } From 6bd2eaa4a5bcf811c5b85be27c5e50058b5d0c12 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Wed, 12 Mar 2014 19:49:18 -0700 Subject: [PATCH 086/397] hot fix for PR105 - change to Java annotation Author: CodingCat Closes #133 from CodingCat/SPARK-1160-2 and squashes the following commits: 6607155 [CodingCat] hot fix for PR105 - change to Java annotation --- .../src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 3df68d4ce508d..2ba4fb8c18b1d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -282,8 +282,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Return an array that contains all of the elements in this RDD. + * @deprecated As of Spark 1.0.0, toArray() is deprecated, use {@link #collect()} instead */ - @deprecated("use collect", "1.0.0") + @Deprecated def toArray(): JList[T] = collect() /** From 4ea23db0efff2f39ac5b8f0bd1d9a6ffa3eceb0d Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 12 Mar 2014 23:16:59 -0700 Subject: [PATCH 087/397] SPARK-1019: pyspark RDD take() throws an NPE Author: Patrick Wendell Closes #112 from pwendell/pyspark-take and squashes the following commits: daae80e [Patrick Wendell] SPARK-1019: pyspark RDD take() throws an NPE --- core/src/main/scala/org/apache/spark/TaskContext.scala | 3 ++- .../scala/org/apache/spark/api/python/PythonRDD.scala | 8 ++++++++ 2 files changed, 10 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index cae983ed4c652..be53ca2968cfb 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -46,6 +46,7 @@ class TaskContext( } def executeOnCompleteCallbacks() { - onCompleteCallbacks.foreach{_()} + // Process complete callbacks in the reverse order of registration + onCompleteCallbacks.reverse.foreach{_()} } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index e4d0285710e84..b67286a4e3b75 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -100,6 +100,14 @@ private[spark] class PythonRDD[T: ClassTag]( } }.start() + /* + * Partial fix for SPARK-1019: Attempts to stop reading the input stream since + * other completion callbacks might invalidate the input. Because interruption + * is not synchronous this still leaves a potential race where the interruption is + * processed only after the stream becomes invalid. + */ + context.addOnCompleteCallback(() => context.interrupted = true) + // Return an iterator that read lines from the process's stdout val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) val stdoutIterator = new Iterator[Array[Byte]] { From e4e8d8f395aea48f0cae00d7c381a863c48a2837 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Thu, 13 Mar 2014 00:43:19 -0700 Subject: [PATCH 088/397] [SPARK-1237, 1238] Improve the computation of YtY for implicit ALS Computing YtY can be implemented using BLAS's DSPR operations instead of generating y_i y_i^T and then combining them. The latter generates many k-by-k matrices. On the movielens data, this change improves the performance by 10-20%. The algorithm remains the same, verified by computing RMSE on the movielens data. To compare the results, I also added an option to set a random seed in ALS. JIRA: 1. https://spark-project.atlassian.net/browse/SPARK-1237 2. https://spark-project.atlassian.net/browse/SPARK-1238 Author: Xiangrui Meng Closes #131 from mengxr/als and squashes the following commits: ed00432 [Xiangrui Meng] minor changes d984623 [Xiangrui Meng] minor changes 2fc1641 [Xiangrui Meng] remove commented code 4c7cde2 [Xiangrui Meng] allow specifying a random seed in ALS 200bef0 [Xiangrui Meng] optimize computeYtY and updateBlock --- .../spark/mllib/recommendation/ALS.scala | 174 ++++++++++++------ .../spark/mllib/recommendation/ALSSuite.scala | 15 +- 2 files changed, 134 insertions(+), 55 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 8958040e36640..777d0db2d6653 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -89,10 +89,15 @@ case class Rating(val user: Int, val product: Int, val rating: Double) * indicated user * preferences rather than explicit ratings given to items. */ -class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var lambda: Double, - var implicitPrefs: Boolean, var alpha: Double) - extends Serializable with Logging -{ +class ALS private ( + var numBlocks: Int, + var rank: Int, + var iterations: Int, + var lambda: Double, + var implicitPrefs: Boolean, + var alpha: Double, + var seed: Long = System.nanoTime() + ) extends Serializable with Logging { def this() = this(-1, 10, 10, 0.01, false, 1.0) /** @@ -132,6 +137,12 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l this } + /** Sets a random seed to have deterministic results. */ + def setSeed(seed: Long): ALS = { + this.seed = seed + this + } + /** * Run ALS with the configured parameters on an input RDD of (user, product, rating) triples. * Returns a MatrixFactorizationModel with feature vectors for each user and product. @@ -155,7 +166,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l // Initialize user and product factors randomly, but use a deterministic seed for each // partition so that fault recovery works - val seedGen = new Random() + val seedGen = new Random(seed) val seed1 = seedGen.nextInt() val seed2 = seedGen.nextInt() // Hash an integer to propagate random bits at all positions, similar to java.util.HashTable @@ -210,21 +221,46 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l */ def computeYtY(factors: RDD[(Int, Array[Array[Double]])]) = { if (implicitPrefs) { - Option( - factors.flatMapValues { case factorArray => - factorArray.view.map { vector => - val x = new DoubleMatrix(vector) - x.mmul(x.transpose()) - } - }.reduceByKeyLocally((a, b) => a.addi(b)) - .values - .reduce((a, b) => a.addi(b)) - ) + val n = rank * (rank + 1) / 2 + val LYtY = factors.values.aggregate(new DoubleMatrix(n))( seqOp = (L, Y) => { + Y.foreach(y => dspr(1.0, new DoubleMatrix(y), L)) + L + }, combOp = (L1, L2) => { + L1.addi(L2) + }) + val YtY = new DoubleMatrix(rank, rank) + fillFullMatrix(LYtY, YtY) + Option(YtY) } else { None } } + /** + * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's DSPR. + * + * @param L the lower triangular part of the matrix packed in an array (row major) + */ + private def dspr(alpha: Double, x: DoubleMatrix, L: DoubleMatrix) = { + val n = x.length + var i = 0 + var j = 0 + var idx = 0 + var axi = 0.0 + val xd = x.data + val Ld = L.data + while (i < n) { + axi = alpha * xd(i) + j = 0 + while (j <= i) { + Ld(idx) += axi * xd(j) + j += 1 + idx += 1 + } + i += 1 + } + } + /** * Flatten out blocked user or product factors into an RDD of (id, factor vector) pairs */ @@ -376,7 +412,8 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l for (productBlock <- 0 until numBlocks) { for (p <- 0 until blockFactors(productBlock).length) { val x = new DoubleMatrix(blockFactors(productBlock)(p)) - fillXtX(x, tempXtX) + tempXtX.fill(0.0) + dspr(1.0, x, tempXtX) val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) for (i <- 0 until us.length) { implicitPrefs match { @@ -387,7 +424,7 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l // Extension to the original paper to handle rs(i) < 0. confidence is a function // of |rs(i)| instead so that it is never negative: val confidence = 1 + alpha * abs(rs(i)) - userXtX(us(i)).addi(tempXtX.mul(confidence - 1)) + SimpleBlas.axpy(confidence - 1.0, tempXtX, userXtX(us(i))) // For rs(i) < 0, the corresponding entry in P is 0 now, not 1 -- negative rs(i) // means we try to reconstruct 0. We add terms only where P = 1, so, term below // is now only added for rs(i) > 0: @@ -400,38 +437,19 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l } // Solve the least-squares problem for each user and return the new feature vectors - userXtX.zipWithIndex.map{ case (triangularXtX, index) => + Array.range(0, numUsers).map { index => // Compute the full XtX matrix from the lower-triangular part we got above - fillFullMatrix(triangularXtX, fullXtX) + fillFullMatrix(userXtX(index), fullXtX) // Add regularization (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda) // Solve the resulting matrix, which is symmetric and positive-definite implicitPrefs match { case false => Solve.solvePositive(fullXtX, userXy(index)).data - case true => Solve.solvePositive(fullXtX.add(YtY.value.get), userXy(index)).data + case true => Solve.solvePositive(fullXtX.addi(YtY.value.get), userXy(index)).data } } } - /** - * Set xtxDest to the lower-triangular part of x transpose * x. For efficiency in summing - * these matrices, we store xtxDest as only rank * (rank+1) / 2 values, namely the values - * at (0,0), (1,0), (1,1), (2,0), (2,1), (2,2), etc in that order. - */ - private def fillXtX(x: DoubleMatrix, xtxDest: DoubleMatrix) { - var i = 0 - var pos = 0 - while (i < x.length) { - var j = 0 - while (j <= i) { - xtxDest.data(pos) = x.data(i) * x.data(j) - pos += 1 - j += 1 - } - i += 1 - } - } - /** * Given a triangular matrix in the order of fillXtX above, compute the full symmetric square * matrix that it represents, storing it into destMatrix. @@ -455,9 +473,10 @@ class ALS private (var numBlocks: Int, var rank: Int, var iterations: Int, var l /** - * Top-level methods for calling Alternating Least Squares (ALS) matrix factorizaton. + * Top-level methods for calling Alternating Least Squares (ALS) matrix factorization. */ object ALS { + /** * Train a matrix factorization model given an RDD of ratings given by users to some products, * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the @@ -470,15 +489,39 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) * @param blocks level of parallelism to split computation into + * @param seed random seed */ def train( ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, - blocks: Int) - : MatrixFactorizationModel = - { + blocks: Int, + seed: Long + ): MatrixFactorizationModel = { + new ALS(blocks, rank, iterations, lambda, false, 1.0, seed).run(ratings) + } + + /** + * Train a matrix factorization model given an RDD of ratings given by users to some products, + * in the form of (userID, productID, rating) pairs. We approximate the ratings matrix as the + * product of two lower-rank matrices of a given rank (number of features). To solve for these + * features, we run a given number of iterations of ALS. This is done using a level of + * parallelism given by `blocks`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + * @param lambda regularization factor (recommended: 0.01) + * @param blocks level of parallelism to split computation into + */ + def train( + ratings: RDD[Rating], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int + ): MatrixFactorizationModel = { new ALS(blocks, rank, iterations, lambda, false, 1.0).run(ratings) } @@ -495,8 +538,7 @@ object ALS { * @param lambda regularization factor (recommended: 0.01) */ def train(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double) - : MatrixFactorizationModel = - { + : MatrixFactorizationModel = { train(ratings, rank, iterations, lambda, -1) } @@ -512,8 +554,7 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) */ def train(ratings: RDD[Rating], rank: Int, iterations: Int) - : MatrixFactorizationModel = - { + : MatrixFactorizationModel = { train(ratings, rank, iterations, 0.01, -1) } @@ -530,6 +571,7 @@ object ALS { * @param lambda regularization factor (recommended: 0.01) * @param blocks level of parallelism to split computation into * @param alpha confidence parameter (only applies when immplicitPrefs = true) + * @param seed random seed */ def trainImplicit( ratings: RDD[Rating], @@ -537,9 +579,34 @@ object ALS { iterations: Int, lambda: Double, blocks: Int, - alpha: Double) - : MatrixFactorizationModel = - { + alpha: Double, + seed: Long + ): MatrixFactorizationModel = { + new ALS(blocks, rank, iterations, lambda, true, alpha, seed).run(ratings) + } + + /** + * Train a matrix factorization model given an RDD of 'implicit preferences' given by users + * to some products, in the form of (userID, productID, preference) pairs. We approximate the + * ratings matrix as the product of two lower-rank matrices of a given rank (number of features). + * To solve for these features, we run a given number of iterations of ALS. This is done using + * a level of parallelism given by `blocks`. + * + * @param ratings RDD of (userID, productID, rating) pairs + * @param rank number of features to use + * @param iterations number of iterations of ALS (recommended: 10-20) + * @param lambda regularization factor (recommended: 0.01) + * @param blocks level of parallelism to split computation into + * @param alpha confidence parameter (only applies when immplicitPrefs = true) + */ + def trainImplicit( + ratings: RDD[Rating], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int, + alpha: Double + ): MatrixFactorizationModel = { new ALS(blocks, rank, iterations, lambda, true, alpha).run(ratings) } @@ -555,8 +622,8 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) * @param lambda regularization factor (recommended: 0.01) */ - def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, - alpha: Double): MatrixFactorizationModel = { + def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int, lambda: Double, alpha: Double) + : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, lambda, -1, alpha) } @@ -573,8 +640,7 @@ object ALS { * @param iterations number of iterations of ALS (recommended: 10-20) */ def trainImplicit(ratings: RDD[Rating], rank: Int, iterations: Int) - : MatrixFactorizationModel = - { + : MatrixFactorizationModel = { trainImplicit(ratings, rank, iterations, 0.01, -1, 1.0) } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 45e7d2db00c42..5aab9aba8f9c0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -23,9 +23,10 @@ import scala.util.Random import org.scalatest.FunSuite -import org.jblas._ +import org.jblas.DoubleMatrix import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.SparkContext._ object ALSSuite { @@ -115,6 +116,18 @@ class ALSSuite extends FunSuite with LocalSparkContext { testALS(100, 200, 2, 15, 0.7, 0.4, true, false, true) } + test("pseudorandomness") { + val ratings = sc.parallelize(ALSSuite.generateRatings(10, 20, 5, 0.5, false, false)._1, 2) + val model11 = ALS.train(ratings, 5, 1, 1.0, 2, 1) + val model12 = ALS.train(ratings, 5, 1, 1.0, 2, 1) + val u11 = model11.userFeatures.values.flatMap(_.toList).collect().toList + val u12 = model12.userFeatures.values.flatMap(_.toList).collect().toList + val model2 = ALS.train(ratings, 5, 1, 1.0, 2, 2) + val u2 = model2.userFeatures.values.flatMap(_.toList).collect().toList + assert(u11 == u12) + assert(u11 != u2) + } + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * From 698373211ef3cdf841c82d48168cd5dbe00a57b4 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Thu, 13 Mar 2014 12:11:33 -0700 Subject: [PATCH 089/397] SPARK-1183. Don't use "worker" to mean executor Author: Sandy Ryza Closes #120 from sryza/sandy-spark-1183 and squashes the following commits: 5066a4a [Sandy Ryza] Remove "worker" in a couple comments 0bd1e46 [Sandy Ryza] Remove --am-class from usage bfc8fe0 [Sandy Ryza] Remove am-class from doc and fix yarn-alpha 607539f [Sandy Ryza] Address review comments 74d087a [Sandy Ryza] SPARK-1183. Don't use "worker" to mean executor --- docs/cluster-overview.md | 2 +- docs/graphx-programming-guide.md | 2 +- docs/job-scheduling.md | 4 +- docs/mllib-classification-regression.md | 4 +- docs/python-programming-guide.md | 6 +- docs/running-on-yarn.md | 29 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 38 ++--- ...rLauncher.scala => ExecutorLauncher.scala} | 28 ++-- ...rRunnable.scala => ExecutorRunnable.scala} | 14 +- .../deploy/yarn/YarnAllocationHandler.scala | 124 ++++++++-------- .../yarn/ApplicationMasterArguments.scala | 27 ++-- .../spark/deploy/yarn/ClientArguments.scala | 46 ++++-- .../apache/spark/deploy/yarn/ClientBase.scala | 18 +-- .../yarn/ClientDistributedCacheManager.scala | 4 +- ...eUtil.scala => ExecutorRunnableUtil.scala} | 14 +- .../cluster/YarnClientClusterScheduler.scala | 4 +- .../cluster/YarnClientSchedulerBackend.scala | 26 ++-- .../spark/deploy/yarn/ApplicationMaster.scala | 38 ++--- ...rLauncher.scala => ExecutorLauncher.scala} | 26 ++-- ...rRunnable.scala => ExecutorRunnable.scala} | 14 +- .../deploy/yarn/YarnAllocationHandler.scala | 138 +++++++++--------- 21 files changed, 312 insertions(+), 294 deletions(-) rename yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/{WorkerLauncher.scala => ExecutorLauncher.scala} (91%) rename yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/{WorkerRunnable.scala => ExecutorRunnable.scala} (93%) rename yarn/common/src/main/scala/org/apache/spark/deploy/yarn/{WorkerRunnableUtil.scala => ExecutorRunnableUtil.scala} (95%) rename yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/{WorkerLauncher.scala => ExecutorLauncher.scala} (92%) rename yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/{WorkerRunnable.scala => ExecutorRunnable.scala} (90%) diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index e16703292cc22..a555a7b5023e3 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -13,7 +13,7 @@ object in your main program (called the _driver program_). Specifically, to run on a cluster, the SparkContext can connect to several types of _cluster managers_ (either Spark's own standalone cluster manager or Mesos/YARN), which allocate resources across applications. Once connected, Spark acquires *executors* on nodes in the cluster, which are -worker processes that run computations and store data for your application. +processes that run computations and store data for your application. Next, it sends your application code (defined by JAR or Python files passed to SparkContext) to the executors. Finally, SparkContext sends *tasks* for the executors to run. diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 3dfed7bea9ea8..1238e3e0a4e7d 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -135,7 +135,7 @@ Like RDDs, property graphs are immutable, distributed, and fault-tolerant. Chan structure of the graph are accomplished by producing a new graph with the desired changes. Note that substantial parts of the original graph (i.e., unaffected structure, attributes, and indicies) are reused in the new graph reducing the cost of this inherently functional data-structure. The -graph is partitioned across the workers using a range of vertex-partitioning heuristics. As with +graph is partitioned across the executors using a range of vertex-partitioning heuristics. As with RDDs, each partition of the graph can be recreated on a different machine in the event of a failure. Logically the property graph corresponds to a pair of typed collections (RDDs) encoding the diff --git a/docs/job-scheduling.md b/docs/job-scheduling.md index df2faa5e41b18..94604f301dd46 100644 --- a/docs/job-scheduling.md +++ b/docs/job-scheduling.md @@ -39,8 +39,8 @@ Resource allocation can be configured as follows, based on the cluster type: * **Mesos:** To use static partitioning on Mesos, set the `spark.mesos.coarse` configuration property to `true`, and optionally set `spark.cores.max` to limit each application's resource share as in the standalone mode. You should also set `spark.executor.memory` to control the executor memory. -* **YARN:** The `--num-workers` option to the Spark YARN client controls how many workers it will allocate - on the cluster, while `--worker-memory` and `--worker-cores` control the resources per worker. +* **YARN:** The `--num-executors` option to the Spark YARN client controls how many executors it will allocate + on the cluster, while `--executor-memory` and `--executor-cores` control the resources per executor. A second option available on Mesos is _dynamic sharing_ of CPU cores. In this mode, each Spark application still has a fixed and independent memory allocation (set by `spark.executor.memory`), but when the diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index 18a3e8e075086..d5bd8042ca2ec 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -77,8 +77,8 @@ between the two goals of small loss and small model complexity. **Distributed Datasets.** For all currently implemented optimization methods for classification, the data must be -distributed between the worker machines *by examples*. Every machine holds a consecutive block of -the `$n$` example/label pairs `$(\x_i,y_i)$`. +distributed between processes on the worker machines *by examples*. Machines hold consecutive +blocks of the `$n$` example/label pairs `$(\x_i,y_i)$`. In other words, the input distributed dataset ([RDD](scala-programming-guide.html#resilient-distributed-datasets-rdds)) must be the set of vectors `$\x_i\in\R^d$`. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 57ed54c9cf4c0..cbe7d820b455e 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -43,9 +43,9 @@ def is_error(line): errors = logData.filter(is_error) {% endhighlight %} -PySpark will automatically ship these functions to workers, along with any objects that they reference. -Instances of classes will be serialized and shipped to workers by PySpark, but classes themselves cannot be automatically distributed to workers. -The [Standalone Use](#standalone-use) section describes how to ship code dependencies to workers. +PySpark will automatically ship these functions to executors, along with any objects that they reference. +Instances of classes will be serialized and shipped to executors by PySpark, but classes themselves cannot be automatically distributed to executors. +The [Standalone Use](#standalone-use) section describes how to ship code dependencies to executors. In addition, PySpark fully supports interactive use---simply run `./bin/pyspark` to launch an interactive shell. diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index b17929542c531..2e9dec4856ee9 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -41,7 +41,7 @@ System Properties: * `spark.yarn.submit.file.replication`, the HDFS replication level for the files uploaded into HDFS for the application. These include things like the spark jar, the app jar, and any distributed cache files/archives. * `spark.yarn.preserve.staging.files`, set to true to preserve the staged files(spark jar, app jar, distributed cache files) at the end of the job rather then delete them. * `spark.yarn.scheduler.heartbeat.interval-ms`, the interval in ms in which the Spark application master heartbeats into the YARN ResourceManager. Default is 5 seconds. -* `spark.yarn.max.worker.failures`, the maximum number of executor failures before failing the application. Default is the number of executors requested times 2 with minimum of 3. +* `spark.yarn.max.executor.failures`, the maximum number of executor failures before failing the application. Default is the number of executors requested times 2 with minimum of 3. # Launching Spark on YARN @@ -60,11 +60,10 @@ The command to launch the Spark application on the cluster is as follows: --jar \ --class \ --args \ - --num-workers \ - --master-class - --master-memory \ - --worker-memory \ - --worker-cores \ + --num-executors \ + --driver-memory \ + --executor-memory \ + --executor-cores \ --name \ --queue \ --addJars \ @@ -85,10 +84,10 @@ For example: --jar examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ --class org.apache.spark.examples.SparkPi \ --args yarn-cluster \ - --num-workers 3 \ - --master-memory 4g \ - --worker-memory 2g \ - --worker-cores 1 + --num-executors 3 \ + --driver-memory 4g \ + --executor-memory 2g \ + --executor-cores 1 The above starts a YARN client program which starts the default Application Master. Then SparkPi will be run as a child thread of Application Master. The client will periodically poll the Application Master for status updates and display them in the console. The client will exit once your application has finished running. Refer to the "Viewing Logs" section below for how to see driver and executor logs. @@ -100,12 +99,12 @@ With yarn-client mode, the application will be launched locally, just like runni Configuration in yarn-client mode: -In order to tune worker cores/number/memory etc., you need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. +In order to tune executor cores/number/memory etc., you need to export environment variables or add them to the spark configuration file (./conf/spark_env.sh). The following are the list of options. -* `SPARK_WORKER_INSTANCES`, Number of executors to start (Default: 2) -* `SPARK_WORKER_CORES`, Number of cores per executor (Default: 1). -* `SPARK_WORKER_MEMORY`, Memory per executor (e.g. 1000M, 2G) (Default: 1G) -* `SPARK_MASTER_MEMORY`, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) +* `SPARK_EXECUTOR_INSTANCES`, Number of executors to start (Default: 2) +* `SPARK_EXECUTOR_CORES`, Number of cores per executor (Default: 1). +* `SPARK_EXECUTOR_MEMORY`, Memory per executor (e.g. 1000M, 2G) (Default: 1G) +* `SPARK_DRIVER_MEMORY`, Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb) * `SPARK_YARN_APP_NAME`, The name of your application (Default: Spark) * `SPARK_YARN_QUEUE`, The YARN queue to use for allocation requests (Default: 'default') * `SPARK_YARN_DIST_FILES`, Comma separated list of files to be distributed with the job. diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 87785cdc60c52..910484ed5432a 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -61,9 +61,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, YarnConfiguration.DEFAULT_RM_AM_MAX_RETRIES) private var isLastAMRetry: Boolean = true - // Default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", - math.max(args.numWorkers * 2, 3)) + // Default to numExecutors * 2, with minimum of 3 + private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", + sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) private var registered = false @@ -96,7 +96,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // Call this to force generation of secret so it gets populated into the // hadoop UGI. This has to happen before the startUserClass which does a - // doAs in order for the credentials to be passed on to the worker containers. + // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) // Start the user's JAR @@ -115,7 +115,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } // Allocate all containers - allocateWorkers() + allocateExecutors() // Wait for the user class to Finish userThread.join() @@ -215,7 +215,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, t } - // this need to happen before allocateWorkers + // this need to happen before allocateExecutors private def waitForSparkContextInitialized() { logInfo("Waiting for spark context initialization") try { @@ -260,21 +260,21 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } } - private def allocateWorkers() { + private def allocateExecutors() { try { - logInfo("Allocating " + args.numWorkers + " workers.") + logInfo("Allocating " + args.numExecutors + " executors.") // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure // Exists the loop if the user thread exits. - while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") + "max number of executor failures reached") } yarnAllocator.allocateContainers( - math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) ApplicationMaster.incrementAllocatorLoop(1) Thread.sleep(100) } @@ -283,7 +283,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } - logInfo("All workers have launched.") + logInfo("All executors have launched.") // Launch a progress reporter thread, else the app will get killed after expiration // (def: 10mins) timeout. @@ -309,15 +309,15 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, val t = new Thread { override def run() { while (userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") + "max number of executor failures reached") } - val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - if (missingWorkerCount > 0) { + val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning + if (missingExecutorCount > 0) { logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingWorkerCount)) - yarnAllocator.allocateContainers(missingWorkerCount) + format(missingExecutorCount)) + yarnAllocator.allocateContainers(missingExecutorCount) } else sendProgress() Thread.sleep(sleepTime) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala similarity index 91% rename from yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index b735d01df8097..7b0e020263835 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -34,7 +34,7 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo -class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) +class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = this(args, new Configuration(), sparkConf) @@ -89,7 +89,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar val minimumMemory = appMasterResponse.getMinimumResourceCapability().getMemory() if (minimumMemory > 0) { - val mem = args.workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD + val mem = args.executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD val numCore = (mem / minimumMemory) + (if (0 != (mem % minimumMemory)) 1 else 0) if (numCore > 0) { @@ -102,7 +102,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar waitForSparkMaster() // Allocate all containers - allocateWorkers() + allocateExecutors() // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. @@ -199,7 +199,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar } - private def allocateWorkers() { + private def allocateExecutors() { // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = @@ -208,16 +208,16 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar yarnAllocator = YarnAllocationHandler.newAllocator(yarnConf, resourceManager, appAttemptId, args, preferredNodeLocationData, sparkConf) - logInfo("Allocating " + args.numWorkers + " workers.") + logInfo("Allocating " + args.numExecutors + " executors.") // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - while ((yarnAllocator.getNumWorkersRunning < args.numWorkers) && (!driverClosed)) { - yarnAllocator.allocateContainers(math.max(args.numWorkers - yarnAllocator.getNumWorkersRunning, 0)) + while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { + yarnAllocator.allocateContainers(math.max(args.numExecutors - yarnAllocator.getNumExecutorsRunning, 0)) Thread.sleep(100) } - logInfo("All workers have launched.") + logInfo("All executors have launched.") } @@ -228,10 +228,10 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar val t = new Thread { override def run() { while (!driverClosed) { - val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - if (missingWorkerCount > 0) { - logInfo("Allocating " + missingWorkerCount + " containers to make up for (potentially ?) lost containers") - yarnAllocator.allocateContainers(missingWorkerCount) + val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning + if (missingExecutorCount > 0) { + logInfo("Allocating " + missingExecutorCount + " containers to make up for (potentially ?) lost containers") + yarnAllocator.allocateContainers(missingExecutorCount) } else sendProgress() Thread.sleep(sleepTime) @@ -264,9 +264,9 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar } -object WorkerLauncher { +object ExecutorLauncher { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new WorkerLauncher(args).run() + new ExecutorLauncher(args).run() } } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala similarity index 93% rename from yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 8c686e393f4f8..981e8b05f602d 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -38,16 +38,16 @@ import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records, ProtoUtils} import org.apache.spark.{SparkConf, Logging} -class WorkerRunnable( +class ExecutorRunnable( container: Container, conf: Configuration, spConf: SparkConf, masterAddress: String, slaveId: String, hostname: String, - workerMemory: Int, - workerCores: Int) - extends Runnable with WorkerRunnableUtil with Logging { + executorMemory: Int, + executorCores: Int) + extends Runnable with ExecutorRunnableUtil with Logging { var rpc: YarnRPC = YarnRPC.create(conf) var cm: ContainerManager = _ @@ -55,7 +55,7 @@ class WorkerRunnable( val yarnConf: YarnConfiguration = new YarnConfiguration(conf) def run = { - logInfo("Starting Worker Container") + logInfo("Starting Executor Container") cm = connectToCM startContainer } @@ -81,8 +81,8 @@ class WorkerRunnable( credentials.writeTokenStorageToStream(dob) ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, workerMemory, workerCores) - logInfo("Setting up worker with commands: " + commands) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) + logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) // Send the start request to the ContainerManager diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index e91257be8ed00..2056667af50cb 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -58,9 +58,9 @@ private[yarn] class YarnAllocationHandler( val conf: Configuration, val resourceManager: AMRMProtocol, val appAttemptId: ApplicationAttemptId, - val maxWorkers: Int, - val workerMemory: Int, - val workerCores: Int, + val maxExecutors: Int, + val executorMemory: Int, + val executorCores: Int, val preferredHostToCount: Map[String, Int], val preferredRackToCount: Map[String, Int], val sparkConf: SparkConf) @@ -84,39 +84,39 @@ private[yarn] class YarnAllocationHandler( // Containers to be released in next request to RM private val pendingReleaseContainers = new ConcurrentHashMap[ContainerId, Boolean] - private val numWorkersRunning = new AtomicInteger() - // Used to generate a unique id per worker - private val workerIdCounter = new AtomicInteger() + private val numExecutorsRunning = new AtomicInteger() + // Used to generate a unique id per executor + private val executorIdCounter = new AtomicInteger() private val lastResponseId = new AtomicInteger() - private val numWorkersFailed = new AtomicInteger() + private val numExecutorsFailed = new AtomicInteger() - def getNumWorkersRunning: Int = numWorkersRunning.intValue + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue - def getNumWorkersFailed: Int = numWorkersFailed.intValue + def getNumExecutorsFailed: Int = numExecutorsFailed.intValue def isResourceConstraintSatisfied(container: Container): Boolean = { - container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + container.getResource.getMemory >= (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD) } - def allocateContainers(workersToRequest: Int) { + def allocateContainers(executorsToRequest: Int) { // We need to send the request only once from what I understand ... but for now, not modifying // this much. // Keep polling the Resource Manager for containers - val amResp = allocateWorkerResources(workersToRequest).getAMResponse + val amResp = allocateExecutorResources(executorsToRequest).getAMResponse val _allocatedContainers = amResp.getAllocatedContainers() if (_allocatedContainers.size > 0) { logDebug(""" Allocated containers: %d - Current worker count: %d + Current executor count: %d Containers released: %s Containers to be released: %s Cluster resources: %s """.format( _allocatedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers, amResp.getAvailableResources)) @@ -221,59 +221,59 @@ private[yarn] class YarnAllocationHandler( // Run each of the allocated containers for (container <- allocatedContainers) { - val numWorkersRunningNow = numWorkersRunning.incrementAndGet() - val workerHostname = container.getNodeId.getHost + val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() + val executorHostname = container.getNodeId.getHost val containerId = container.getId assert( - container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) + container.getResource.getMemory >= (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) - if (numWorkersRunningNow > maxWorkers) { + if (numExecutorsRunningNow > maxExecutors) { logInfo("""Ignoring container %s at host %s, since we already have the required number of - containers for it.""".format(containerId, workerHostname)) + containers for it.""".format(containerId, executorHostname)) releasedContainerList.add(containerId) // reset counter back to old value. - numWorkersRunning.decrementAndGet() + numExecutorsRunning.decrementAndGet() } else { // Deallocate + allocate can result in reusing id's wrongly - so use a different counter - // (workerIdCounter) - val workerId = workerIdCounter.incrementAndGet().toString + // (executorIdCounter) + val executorId = executorIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - logInfo("launching container on " + containerId + " host " + workerHostname) + logInfo("launching container on " + containerId + " host " + executorHostname) // Just to be safe, simply remove it from pendingReleaseContainers. // Should not be there, but .. pendingReleaseContainers.remove(containerId) - val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) + val rack = YarnAllocationHandler.lookupRack(conf, executorHostname) allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]()) containerSet += containerId - allocatedContainerToHostMap.put(containerId, workerHostname) + allocatedContainerToHostMap.put(containerId, executorHostname) if (rack != null) { allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) } } new Thread( - new WorkerRunnable(container, conf, sparkConf, driverUrl, workerId, - workerHostname, workerMemory, workerCores) + new ExecutorRunnable(container, conf, sparkConf, driverUrl, executorId, + executorHostname, executorMemory, executorCores) ).start() } } logDebug(""" Finished processing %d containers. - Current number of workers running: %d, + Current number of executors running: %d, releasedContainerList: %s, pendingReleaseContainers: %s """.format( allocatedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers)) } @@ -292,7 +292,7 @@ private[yarn] class YarnAllocationHandler( } else { // Simply decrement count - next iteration of ReporterThread will take care of allocating. - numWorkersRunning.decrementAndGet() + numExecutorsRunning.decrementAndGet() logInfo("Completed container %s (state: %s, exit status: %s)".format( containerId, completedContainer.getState, @@ -302,7 +302,7 @@ private[yarn] class YarnAllocationHandler( // now I think its ok as none of the containers are expected to exit if (completedContainer.getExitStatus() != 0) { logInfo("Container marked as failed: " + containerId) - numWorkersFailed.incrementAndGet() + numExecutorsFailed.incrementAndGet() } } @@ -332,12 +332,12 @@ private[yarn] class YarnAllocationHandler( } logDebug(""" Finished processing %d completed containers. - Current number of workers running: %d, + Current number of executors running: %d, releasedContainerList: %s, pendingReleaseContainers: %s """.format( completedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers)) } @@ -387,18 +387,18 @@ private[yarn] class YarnAllocationHandler( retval } - private def allocateWorkerResources(numWorkers: Int): AllocateResponse = { + private def allocateExecutorResources(numExecutors: Int): AllocateResponse = { var resourceRequests: List[ResourceRequest] = null // default. - if (numWorkers <= 0 || preferredHostToCount.isEmpty) { - logDebug("numWorkers: " + numWorkers + ", host preferences: " + preferredHostToCount.isEmpty) + if (numExecutors <= 0 || preferredHostToCount.isEmpty) { + logDebug("numExecutors: " + numExecutors + ", host preferences: " + preferredHostToCount.isEmpty) resourceRequests = List( - createResourceRequest(AllocationType.ANY, null, numWorkers, YarnAllocationHandler.PRIORITY)) + createResourceRequest(AllocationType.ANY, null, numExecutors, YarnAllocationHandler.PRIORITY)) } else { - // request for all hosts in preferred nodes and for numWorkers - + // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. val hostContainerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](preferredHostToCount.size) @@ -419,7 +419,7 @@ private[yarn] class YarnAllocationHandler( val anyContainerRequests: ResourceRequest = createResourceRequest( AllocationType.ANY, resource = null, - numWorkers, + numExecutors, YarnAllocationHandler.PRIORITY) val containerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest]( @@ -441,9 +441,9 @@ private[yarn] class YarnAllocationHandler( val releasedContainerList = createReleasedContainerList() req.addAllReleases(releasedContainerList) - if (numWorkers > 0) { - logInfo("Allocating %d worker containers with %d of memory each.".format(numWorkers, - workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) + if (numExecutors > 0) { + logInfo("Allocating %d executor containers with %d of memory each.".format(numExecutors, + executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) } else { logDebug("Empty allocation req .. release : " + releasedContainerList) @@ -464,7 +464,7 @@ private[yarn] class YarnAllocationHandler( private def createResourceRequest( requestType: AllocationType.AllocationType, resource:String, - numWorkers: Int, + numExecutors: Int, priority: Int): ResourceRequest = { // If hostname specified, we need atleast two requests - node local and rack local. @@ -473,7 +473,7 @@ private[yarn] class YarnAllocationHandler( case AllocationType.HOST => { assert(YarnAllocationHandler.ANY_HOST != resource) val hostname = resource - val nodeLocal = createResourceRequestImpl(hostname, numWorkers, priority) + val nodeLocal = createResourceRequestImpl(hostname, numExecutors, priority) // Add to host->rack mapping YarnAllocationHandler.populateRackInfo(conf, hostname) @@ -482,10 +482,10 @@ private[yarn] class YarnAllocationHandler( } case AllocationType.RACK => { val rack = resource - createResourceRequestImpl(rack, numWorkers, priority) + createResourceRequestImpl(rack, numExecutors, priority) } case AllocationType.ANY => createResourceRequestImpl( - YarnAllocationHandler.ANY_HOST, numWorkers, priority) + YarnAllocationHandler.ANY_HOST, numExecutors, priority) case _ => throw new IllegalArgumentException( "Unexpected/unsupported request type: " + requestType) } @@ -493,13 +493,13 @@ private[yarn] class YarnAllocationHandler( private def createResourceRequestImpl( hostname:String, - numWorkers: Int, + numExecutors: Int, priority: Int): ResourceRequest = { val rsrcRequest = Records.newRecord(classOf[ResourceRequest]) val memCapability = Records.newRecord(classOf[Resource]) // There probably is some overhead here, let's reserve a bit more memory. - memCapability.setMemory(workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + memCapability.setMemory(executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD) rsrcRequest.setCapability(memCapability) val pri = Records.newRecord(classOf[Priority]) @@ -508,7 +508,7 @@ private[yarn] class YarnAllocationHandler( rsrcRequest.setHostName(hostname) - rsrcRequest.setNumContainers(java.lang.Math.max(numWorkers, 0)) + rsrcRequest.setNumContainers(java.lang.Math.max(numExecutors, 0)) rsrcRequest } @@ -560,9 +560,9 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numWorkers, - args.workerMemory, - args.workerCores, + args.numExecutors, + args.executorMemory, + args.executorCores, Map[String, Int](), Map[String, Int](), sparkConf) @@ -582,9 +582,9 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numWorkers, - args.workerMemory, - args.workerCores, + args.numExecutors, + args.executorMemory, + args.executorCores, hostToCount, rackToCount, sparkConf) @@ -594,9 +594,9 @@ object YarnAllocationHandler { conf: Configuration, resourceManager: AMRMProtocol, appAttemptId: ApplicationAttemptId, - maxWorkers: Int, - workerMemory: Int, - workerCores: Int, + maxExecutors: Int, + executorMemory: Int, + executorCores: Int, map: collection.Map[String, collection.Set[SplitInfo]], sparkConf: SparkConf): YarnAllocationHandler = { @@ -606,9 +606,9 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - maxWorkers, - workerMemory, - workerCores, + maxExecutors, + executorMemory, + executorCores, hostToCount, rackToCount, sparkConf) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala index f76a5ddd39e90..25cc9016b10a6 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMasterArguments.scala @@ -24,9 +24,9 @@ class ApplicationMasterArguments(val args: Array[String]) { var userJar: String = null var userClass: String = null var userArgs: Seq[String] = Seq[String]() - var workerMemory = 1024 - var workerCores = 1 - var numWorkers = 2 + var executorMemory = 1024 + var executorCores = 1 + var numExecutors = 2 parseArgs(args.toList) @@ -36,7 +36,8 @@ class ApplicationMasterArguments(val args: Array[String]) { var args = inputArgs while (! args.isEmpty) { - + // --num-workers, --worker-memory, and --worker-cores are deprecated since 1.0, + // the properties with executor in their names are preferred. args match { case ("--jar") :: value :: tail => userJar = value @@ -50,16 +51,16 @@ class ApplicationMasterArguments(val args: Array[String]) { userArgsBuffer += value args = tail - case ("--num-workers") :: IntParam(value) :: tail => - numWorkers = value + case ("--num-workers" | "--num-executors") :: IntParam(value) :: tail => + numExecutors = value args = tail - case ("--worker-memory") :: IntParam(value) :: tail => - workerMemory = value + case ("--worker-memory" | "--executor-memory") :: IntParam(value) :: tail => + executorMemory = value args = tail - case ("--worker-cores") :: IntParam(value) :: tail => - workerCores = value + case ("--worker-cores" | "--executor-cores") :: IntParam(value) :: tail => + executorCores = value args = tail case Nil => @@ -86,9 +87,9 @@ class ApplicationMasterArguments(val args: Array[String]) { " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + - " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1)\n" + - " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n") + " --num-executors NUM Number of executors to start (Default: 2)\n" + + " --executor-cores NUM Number of cores for the executors (Default: 1)\n" + + " --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G)\n") System.exit(exitCode) } } diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index 1f894a677d169..a001060cdb746 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -33,9 +33,9 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { var userJar: String = null var userClass: String = null var userArgs: Seq[String] = Seq[String]() - var workerMemory = 1024 // MB - var workerCores = 1 - var numWorkers = 2 + var executorMemory = 1024 // MB + var executorCores = 1 + var numExecutors = 2 var amQueue = sparkConf.get("QUEUE", "default") var amMemory: Int = 512 // MB var amClass: String = "org.apache.spark.deploy.yarn.ApplicationMaster" @@ -67,24 +67,39 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { userArgsBuffer += value args = tail - case ("--master-class") :: value :: tail => + case ("--master-class" | "--am-class") :: value :: tail => + if (args(0) == "--master-class") { + println("--master-class is deprecated. Use --am-class instead.") + } amClass = value args = tail - case ("--master-memory") :: MemoryParam(value) :: tail => + case ("--master-memory" | "--driver-memory") :: MemoryParam(value) :: tail => + if (args(0) == "--master-memory") { + println("--master-memory is deprecated. Use --driver-memory instead.") + } amMemory = value args = tail - case ("--num-workers") :: IntParam(value) :: tail => - numWorkers = value + case ("--num-workers" | "--num-executors") :: IntParam(value) :: tail => + if (args(0) == "--num-workers") { + println("--num-workers is deprecated. Use --num-executors instead.") + } + numExecutors = value args = tail - case ("--worker-memory") :: MemoryParam(value) :: tail => - workerMemory = value + case ("--worker-memory" | "--executor-memory") :: MemoryParam(value) :: tail => + if (args(0) == "--worker-memory") { + println("--worker-memory is deprecated. Use --executor-memory instead.") + } + executorMemory = value args = tail - case ("--worker-cores") :: IntParam(value) :: tail => - workerCores = value + case ("--worker-cores" | "--executor-memory") :: IntParam(value) :: tail => + if (args(0) == "--worker-cores") { + println("--worker-cores is deprecated. Use --executor-cores instead.") + } + executorCores = value args = tail case ("--queue") :: value :: tail => @@ -133,11 +148,10 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { " --class CLASS_NAME Name of your application's main class (required)\n" + " --args ARGS Arguments to be passed to your application's main class.\n" + " Mutliple invocations are possible, each will be passed in order.\n" + - " --num-workers NUM Number of workers to start (Default: 2)\n" + - " --worker-cores NUM Number of cores for the workers (Default: 1).\n" + - " --master-class CLASS_NAME Class Name for Master (Default: spark.deploy.yarn.ApplicationMaster)\n" + - " --master-memory MEM Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb)\n" + - " --worker-memory MEM Memory per Worker (e.g. 1000M, 2G) (Default: 1G)\n" + + " --num-executors NUM Number of executors to start (Default: 2)\n" + + " --executor-cores NUM Number of cores for the executors (Default: 1).\n" + + " --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb)\n" + + " --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G)\n" + " --name NAME The name of your application (Default: Spark)\n" + " --queue QUEUE The hadoop queue to use for allocation requests (Default: 'default')\n" + " --addJars jars Comma separated list of local jars that want SparkContext.addJar to work with.\n" + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 74c5e0f18e7bd..57e5761cba896 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -73,10 +73,10 @@ trait ClientBase extends Logging { ((args.userJar == null && args.amClass == classOf[ApplicationMaster].getName) -> "Error: You must specify a user jar when running in standalone mode!"), (args.userClass == null) -> "Error: You must specify a user class!", - (args.numWorkers <= 0) -> "Error: You must specify at least 1 worker!", + (args.numExecutors <= 0) -> "Error: You must specify at least 1 executor!", (args.amMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: AM memory size must be" + "greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD), - (args.workerMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Worker memory size" + + (args.executorMemory <= YarnAllocationHandler.MEMORY_OVERHEAD) -> ("Error: Executor memory size" + "must be greater than: " + YarnAllocationHandler.MEMORY_OVERHEAD.toString) ).foreach { case(cond, errStr) => if (cond) { @@ -95,9 +95,9 @@ trait ClientBase extends Logging { logInfo("Max mem capabililty of a single resource in this cluster " + maxMem) // If we have requested more then the clusters max for a single resource then exit. - if (args.workerMemory > maxMem) { - logError("Required worker memory (%d MB), is above the max threshold (%d MB) of this cluster.". - format(args.workerMemory, maxMem)) + if (args.executorMemory > maxMem) { + logError("Required executor memory (%d MB), is above the max threshold (%d MB) of this cluster.". + format(args.executorMemory, maxMem)) System.exit(1) } val amMem = args.amMemory + YarnAllocationHandler.MEMORY_OVERHEAD @@ -276,7 +276,7 @@ trait ClientBase extends Logging { env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() - // Set the environment variables to be passed on to the Workers. + // Set the environment variables to be passed on to the executors. distCacheMgr.setDistFilesEnv(env) distCacheMgr.setDistArchivesEnv(env) @@ -360,9 +360,9 @@ trait ClientBase extends Logging { " --class " + args.userClass + " --jar " + args.userJar + userArgsToString(args) + - " --worker-memory " + args.workerMemory + - " --worker-cores " + args.workerCores + - " --num-workers " + args.numWorkers + + " --executor-memory " + args.executorMemory + + " --executor-cores " + args.executorCores + + " --num-executors " + args.numExecutors + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 535abbfb7f638..68cda0f1c9f8b 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -46,10 +46,10 @@ class ClientDistributedCacheManager() extends Logging { /** * Add a resource to the list of distributed cache resources. This list can - * be sent to the ApplicationMaster and possibly the workers so that it can + * be sent to the ApplicationMaster and possibly the executors so that it can * be downloaded into the Hadoop distributed cache for use by this application. * Adds the LocalResource to the localResources HashMap passed in and saves - * the stats of the resources to they can be sent to the workers and verified. + * the stats of the resources to they can be sent to the executors and verified. * * @param fs FileSystem * @param conf Configuration diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala similarity index 95% rename from yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala rename to yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index bfa8f84bf7f85..da0a6f74efcd5 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -39,7 +39,7 @@ import org.apache.spark.{SparkConf, Logging} import org.apache.hadoop.yarn.conf.YarnConfiguration -trait WorkerRunnableUtil extends Logging { +trait ExecutorRunnableUtil extends Logging { val yarnConf: YarnConfiguration val sparkConf: SparkConf @@ -49,13 +49,13 @@ trait WorkerRunnableUtil extends Logging { masterAddress: String, slaveId: String, hostname: String, - workerMemory: Int, - workerCores: Int) = { + executorMemory: Int, + executorCores: Int) = { // Extra options for the JVM var JAVA_OPTS = "" // Set the JVM memory - val workerMemoryString = workerMemory + "m" - JAVA_OPTS += "-Xms" + workerMemoryString + " -Xmx" + workerMemoryString + " " + val executorMemoryString = executorMemory + "m" + JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " if (env.isDefinedAt("SPARK_JAVA_OPTS")) { JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " } @@ -97,7 +97,7 @@ trait WorkerRunnableUtil extends Logging { val commands = List[String](javaCommand + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. - // Not killing the task leaves various aspects of the worker and (to some extent) the jvm in + // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in // an inconsistent state. // TODO: If the OOM is not recoverable by rescheduling it on different node, then do // 'something' to fail job ... akin to blacklisting trackers in mapred ? @@ -107,7 +107,7 @@ trait WorkerRunnableUtil extends Logging { masterAddress + " " + slaveId + " " + hostname + " " + - workerCores + + executorCores + " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala index 522e0a9ad7eeb..6b91e6b9eb899 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientClusterScheduler.scala @@ -25,7 +25,7 @@ import org.apache.spark.util.Utils /** * - * This scheduler launch worker through Yarn - by call into Client to launch WorkerLauncher as AM. + * This scheduler launches executors through Yarn - by calling into Client to launch ExecutorLauncher as AM. */ private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configuration) extends TaskSchedulerImpl(sc) { @@ -40,7 +40,7 @@ private[spark] class YarnClientClusterScheduler(sc: SparkContext, conf: Configur override def postStartHook() { - // The yarn application is running, but the worker might not yet ready + // The yarn application is running, but the executor might not yet ready // Wait for a few seconds for the slaves to bootstrap and register with master - best case attempt Thread.sleep(2000L) logInfo("YarnClientClusterScheduler.postStartHook done") diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index e7130d24072ca..d1f13e3c369ed 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -53,20 +53,24 @@ private[spark] class YarnClientSchedulerBackend( "--class", "notused", "--jar", null, "--args", hostport, - "--master-class", "org.apache.spark.deploy.yarn.WorkerLauncher" + "--am-class", "org.apache.spark.deploy.yarn.ExecutorLauncher" ) // process any optional arguments, use the defaults already defined in ClientArguments // if things aren't specified - Map("--master-memory" -> "SPARK_MASTER_MEMORY", - "--num-workers" -> "SPARK_WORKER_INSTANCES", - "--worker-memory" -> "SPARK_WORKER_MEMORY", - "--worker-cores" -> "SPARK_WORKER_CORES", - "--queue" -> "SPARK_YARN_QUEUE", - "--name" -> "SPARK_YARN_APP_NAME", - "--files" -> "SPARK_YARN_DIST_FILES", - "--archives" -> "SPARK_YARN_DIST_ARCHIVES") - .foreach { case (optName, optParam) => addArg(optName, optParam, argsArrayBuf) } + Map("SPARK_MASTER_MEMORY" -> "--driver-memory", + "SPARK_DRIVER_MEMORY" -> "--driver-memory", + "SPARK_WORKER_INSTANCES" -> "--num-executors", + "SPARK_WORKER_MEMORY" -> "--executor-memory", + "SPARK_WORKER_CORES" -> "--executor-cores", + "SPARK_EXECUTOR_INSTANCES" -> "--num-executors", + "SPARK_EXECUTOR_MEMORY" -> "--executor-memory", + "SPARK_EXECUTOR_CORES" -> "--executor-cores", + "SPARK_YARN_QUEUE" -> "--queue", + "SPARK_YARN_APP_NAME" -> "--name", + "SPARK_YARN_DIST_FILES" -> "--files", + "SPARK_YARN_DIST_ARCHIVES" -> "--archives") + .foreach { case (optParam, optName) => addArg(optName, optParam, argsArrayBuf) } logDebug("ClientArguments called with: " + argsArrayBuf) val args = new ClientArguments(argsArrayBuf.toArray, conf) @@ -77,7 +81,7 @@ private[spark] class YarnClientSchedulerBackend( def waitForApp() { - // TODO : need a better way to find out whether the workers are ready or not + // TODO : need a better way to find out whether the executors are ready or not // maybe by resource usage report? while(true) { val report = client.getApplicationReport(appId) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 57d15774290dd..30735cbfdf26e 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -64,9 +64,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, private var isLastAMRetry: Boolean = true private var amClient: AMRMClient[ContainerRequest] = _ - // Default to numWorkers * 2, with minimum of 3 - private val maxNumWorkerFailures = sparkConf.getInt("spark.yarn.max.worker.failures", - math.max(args.numWorkers * 2, 3)) + // Default to numExecutors * 2, with minimum of 3 + private val maxNumExecutorFailures = sparkConf.getInt("spark.yarn.max.executor.failures", + sparkConf.getInt("spark.yarn.max.worker.failures", math.max(args.numExecutors * 2, 3))) private var registered = false @@ -101,7 +101,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // Call this to force generation of secret so it gets populated into the // hadoop UGI. This has to happen before the startUserClass which does a - // doAs in order for the credentials to be passed on to the worker containers. + // doAs in order for the credentials to be passed on to the executor containers. val securityMgr = new SecurityManager(sparkConf) // Start the user's JAR @@ -120,7 +120,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } // Allocate all containers - allocateWorkers() + allocateExecutors() // Wait for the user class to Finish userThread.join() @@ -202,7 +202,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, t } - // This need to happen before allocateWorkers() + // This need to happen before allocateExecutors() private def waitForSparkContextInitialized() { logInfo("Waiting for Spark context initialization") try { @@ -247,18 +247,18 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, } } - private def allocateWorkers() { + private def allocateExecutors() { try { - logInfo("Allocating " + args.numWorkers + " workers.") + logInfo("Allocating " + args.numExecutors + " executors.") // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - yarnAllocator.addResourceRequests(args.numWorkers) + yarnAllocator.addResourceRequests(args.numExecutors) // Exits the loop if the user thread exits. - while (yarnAllocator.getNumWorkersRunning < args.numWorkers && userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + while (yarnAllocator.getNumExecutorsRunning < args.numExecutors && userThread.isAlive) { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") + "max number of executor failures reached") } yarnAllocator.allocateResources() ApplicationMaster.incrementAllocatorLoop(1) @@ -269,7 +269,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // so that the loop in ApplicationMaster#sparkContextInitialized() breaks. ApplicationMaster.incrementAllocatorLoop(ApplicationMaster.ALLOCATOR_LOOP_WAIT_COUNT) } - logInfo("All workers have launched.") + logInfo("All executors have launched.") // Launch a progress reporter thread, else the app will get killed after expiration // (def: 10mins) timeout. @@ -294,16 +294,16 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, val t = new Thread { override def run() { while (userThread.isAlive) { - if (yarnAllocator.getNumWorkersFailed >= maxNumWorkerFailures) { + if (yarnAllocator.getNumExecutorsFailed >= maxNumExecutorFailures) { finishApplicationMaster(FinalApplicationStatus.FAILED, - "max number of worker failures reached") + "max number of executor failures reached") } - val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - + val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning - yarnAllocator.getNumPendingAllocate - if (missingWorkerCount > 0) { + if (missingExecutorCount > 0) { logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingWorkerCount)) - yarnAllocator.addResourceRequests(missingWorkerCount) + format(missingExecutorCount)) + yarnAllocator.addResourceRequests(missingExecutorCount) } sendProgress() Thread.sleep(sleepTime) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala similarity index 92% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index f1c1fea0b5895..b697f103914fd 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -35,7 +35,7 @@ import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest -class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) +class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { def this(args: ApplicationMasterArguments, sparkConf: SparkConf) = @@ -93,7 +93,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar waitForSparkMaster() // Allocate all containers - allocateWorkers() + allocateExecutors() // Launch a progress reporter thread, else app will get killed after expiration (def: 10mins) timeout // ensure that progress is sent before YarnConfiguration.RM_AM_EXPIRY_INTERVAL_MS elapse. @@ -175,7 +175,7 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar } - private def allocateWorkers() { + private def allocateExecutors() { // Fixme: should get preferredNodeLocationData from SparkContext, just fake a empty one for now. val preferredNodeLocationData: scala.collection.Map[String, scala.collection.Set[SplitInfo]] = @@ -189,18 +189,18 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar preferredNodeLocationData, sparkConf) - logInfo("Allocating " + args.numWorkers + " workers.") + logInfo("Allocating " + args.numExecutors + " executors.") // Wait until all containers have finished // TODO: This is a bit ugly. Can we make it nicer? // TODO: Handle container failure - yarnAllocator.addResourceRequests(args.numWorkers) - while ((yarnAllocator.getNumWorkersRunning < args.numWorkers) && (!driverClosed)) { + yarnAllocator.addResourceRequests(args.numExecutors) + while ((yarnAllocator.getNumExecutorsRunning < args.numExecutors) && (!driverClosed)) { yarnAllocator.allocateResources() Thread.sleep(100) } - logInfo("All workers have launched.") + logInfo("All executors have launched.") } @@ -211,12 +211,12 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar val t = new Thread { override def run() { while (!driverClosed) { - val missingWorkerCount = args.numWorkers - yarnAllocator.getNumWorkersRunning - + val missingExecutorCount = args.numExecutors - yarnAllocator.getNumExecutorsRunning - yarnAllocator.getNumPendingAllocate - if (missingWorkerCount > 0) { + if (missingExecutorCount > 0) { logInfo("Allocating %d containers to make up for (potentially) lost containers". - format(missingWorkerCount)) - yarnAllocator.addResourceRequests(missingWorkerCount) + format(missingExecutorCount)) + yarnAllocator.addResourceRequests(missingExecutorCount) } sendProgress() Thread.sleep(sleepTime) @@ -244,9 +244,9 @@ class WorkerLauncher(args: ApplicationMasterArguments, conf: Configuration, spar } -object WorkerLauncher { +object ExecutorLauncher { def main(argStrings: Array[String]) { val args = new ApplicationMasterArguments(argStrings) - new WorkerLauncher(args).run() + new ExecutorLauncher(args).run() } } diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala similarity index 90% rename from yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala rename to yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index ab4a79be70485..53c403f7d0913 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/WorkerRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -38,16 +38,16 @@ import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} import org.apache.spark.{SparkConf, Logging} -class WorkerRunnable( +class ExecutorRunnable( container: Container, conf: Configuration, spConf: SparkConf, masterAddress: String, slaveId: String, hostname: String, - workerMemory: Int, - workerCores: Int) - extends Runnable with WorkerRunnableUtil with Logging { + executorMemory: Int, + executorCores: Int) + extends Runnable with ExecutorRunnableUtil with Logging { var rpc: YarnRPC = YarnRPC.create(conf) var nmClient: NMClient = _ @@ -55,7 +55,7 @@ class WorkerRunnable( val yarnConf: YarnConfiguration = new YarnConfiguration(conf) def run = { - logInfo("Starting Worker Container") + logInfo("Starting Executor Container") nmClient = NMClient.createNMClient() nmClient.init(yarnConf) nmClient.start() @@ -78,9 +78,9 @@ class WorkerRunnable( credentials.writeTokenStorageToStream(dob) ctx.setTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, workerMemory, workerCores) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) - logInfo("Setting up worker with commands: " + commands) + logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) // Send the start request to the ContainerManager diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 1ac61124cb028..e31c4060e8452 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -60,9 +60,9 @@ private[yarn] class YarnAllocationHandler( val conf: Configuration, val amClient: AMRMClient[ContainerRequest], val appAttemptId: ApplicationAttemptId, - val maxWorkers: Int, - val workerMemory: Int, - val workerCores: Int, + val maxExecutors: Int, + val executorMemory: Int, + val executorCores: Int, val preferredHostToCount: Map[String, Int], val preferredRackToCount: Map[String, Int], val sparkConf: SparkConf) @@ -89,20 +89,20 @@ private[yarn] class YarnAllocationHandler( // Number of container requests that have been sent to, but not yet allocated by the // ApplicationMaster. private val numPendingAllocate = new AtomicInteger() - private val numWorkersRunning = new AtomicInteger() - // Used to generate a unique id per worker - private val workerIdCounter = new AtomicInteger() + private val numExecutorsRunning = new AtomicInteger() + // Used to generate a unique id per executor + private val executorIdCounter = new AtomicInteger() private val lastResponseId = new AtomicInteger() - private val numWorkersFailed = new AtomicInteger() + private val numExecutorsFailed = new AtomicInteger() def getNumPendingAllocate: Int = numPendingAllocate.intValue - def getNumWorkersRunning: Int = numWorkersRunning.intValue + def getNumExecutorsRunning: Int = numExecutorsRunning.intValue - def getNumWorkersFailed: Int = numWorkersFailed.intValue + def getNumExecutorsFailed: Int = numExecutorsFailed.intValue def isResourceConstraintSatisfied(container: Container): Boolean = { - container.getResource.getMemory >= (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + container.getResource.getMemory >= (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD) } def releaseContainer(container: Container) { @@ -127,13 +127,13 @@ private[yarn] class YarnAllocationHandler( logDebug(""" Allocated containers: %d - Current worker count: %d + Current executor count: %d Containers released: %s Containers to-be-released: %s Cluster resources: %s """.format( allocatedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers, allocateResponse.getAvailableResources)) @@ -240,64 +240,64 @@ private[yarn] class YarnAllocationHandler( // Run each of the allocated containers. for (container <- allocatedContainersToProcess) { - val numWorkersRunningNow = numWorkersRunning.incrementAndGet() - val workerHostname = container.getNodeId.getHost + val numExecutorsRunningNow = numExecutorsRunning.incrementAndGet() + val executorHostname = container.getNodeId.getHost val containerId = container.getId - val workerMemoryOverhead = (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD) - assert(container.getResource.getMemory >= workerMemoryOverhead) + val executorMemoryOverhead = (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD) + assert(container.getResource.getMemory >= executorMemoryOverhead) - if (numWorkersRunningNow > maxWorkers) { + if (numExecutorsRunningNow > maxExecutors) { logInfo("""Ignoring container %s at host %s, since we already have the required number of - containers for it.""".format(containerId, workerHostname)) + containers for it.""".format(containerId, executorHostname)) releaseContainer(container) - numWorkersRunning.decrementAndGet() + numExecutorsRunning.decrementAndGet() } else { - val workerId = workerIdCounter.incrementAndGet().toString + val executorId = executorIdCounter.incrementAndGet().toString val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( sparkConf.get("spark.driver.host"), sparkConf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) - logInfo("Launching container %s for on host %s".format(containerId, workerHostname)) + logInfo("Launching container %s for on host %s".format(containerId, executorHostname)) // To be safe, remove the container from `pendingReleaseContainers`. pendingReleaseContainers.remove(containerId) - val rack = YarnAllocationHandler.lookupRack(conf, workerHostname) + val rack = YarnAllocationHandler.lookupRack(conf, executorHostname) allocatedHostToContainersMap.synchronized { - val containerSet = allocatedHostToContainersMap.getOrElseUpdate(workerHostname, + val containerSet = allocatedHostToContainersMap.getOrElseUpdate(executorHostname, new HashSet[ContainerId]()) containerSet += containerId - allocatedContainerToHostMap.put(containerId, workerHostname) + allocatedContainerToHostMap.put(containerId, executorHostname) if (rack != null) { allocatedRackCount.put(rack, allocatedRackCount.getOrElse(rack, 0) + 1) } } - logInfo("Launching WorkerRunnable. driverUrl: %s, workerHostname: %s".format(driverUrl, workerHostname)) - val workerRunnable = new WorkerRunnable( + logInfo("Launching ExecutorRunnable. driverUrl: %s, executorHostname: %s".format(driverUrl, executorHostname)) + val executorRunnable = new ExecutorRunnable( container, conf, sparkConf, driverUrl, - workerId, - workerHostname, - workerMemory, - workerCores) - new Thread(workerRunnable).start() + executorId, + executorHostname, + executorMemory, + executorCores) + new Thread(executorRunnable).start() } } logDebug(""" Finished allocating %s containers (from %s originally). - Current number of workers running: %d, + Current number of executors running: %d, releasedContainerList: %s, pendingReleaseContainers: %s """.format( allocatedContainersToProcess, allocatedContainers, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers)) } @@ -314,9 +314,9 @@ private[yarn] class YarnAllocationHandler( // `pendingReleaseContainers`. pendingReleaseContainers.remove(containerId) } else { - // Decrement the number of workers running. The next iteration of the ApplicationMaster's + // Decrement the number of executors running. The next iteration of the ApplicationMaster's // reporting thread will take care of allocating. - numWorkersRunning.decrementAndGet() + numExecutorsRunning.decrementAndGet() logInfo("Completed container %s (state: %s, exit status: %s)".format( containerId, completedContainer.getState, @@ -326,7 +326,7 @@ private[yarn] class YarnAllocationHandler( // now I think its ok as none of the containers are expected to exit if (completedContainer.getExitStatus() != 0) { logInfo("Container marked as failed: " + containerId) - numWorkersFailed.incrementAndGet() + numExecutorsFailed.incrementAndGet() } } @@ -364,12 +364,12 @@ private[yarn] class YarnAllocationHandler( } logDebug(""" Finished processing %d completed containers. - Current number of workers running: %d, + Current number of executors running: %d, releasedContainerList: %s, pendingReleaseContainers: %s """.format( completedContainers.size, - numWorkersRunning.get(), + numExecutorsRunning.get(), releasedContainerList, pendingReleaseContainers)) } @@ -421,18 +421,18 @@ private[yarn] class YarnAllocationHandler( retval } - def addResourceRequests(numWorkers: Int) { + def addResourceRequests(numExecutors: Int) { val containerRequests: List[ContainerRequest] = - if (numWorkers <= 0 || preferredHostToCount.isEmpty) { - logDebug("numWorkers: " + numWorkers + ", host preferences: " + + if (numExecutors <= 0 || preferredHostToCount.isEmpty) { + logDebug("numExecutors: " + numExecutors + ", host preferences: " + preferredHostToCount.isEmpty) createResourceRequests( AllocationType.ANY, resource = null, - numWorkers, + numExecutors, YarnAllocationHandler.PRIORITY).toList } else { - // Request for all hosts in preferred nodes and for numWorkers - + // Request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. val hostContainerRequests = new ArrayBuffer[ContainerRequest](preferredHostToCount.size) for ((candidateHost, candidateCount) <- preferredHostToCount) { @@ -452,7 +452,7 @@ private[yarn] class YarnAllocationHandler( val anyContainerRequests = createResourceRequests( AllocationType.ANY, resource = null, - numWorkers, + numExecutors, YarnAllocationHandler.PRIORITY) val containerRequestBuffer = new ArrayBuffer[ContainerRequest]( @@ -468,11 +468,11 @@ private[yarn] class YarnAllocationHandler( amClient.addContainerRequest(request) } - if (numWorkers > 0) { - numPendingAllocate.addAndGet(numWorkers) - logInfo("Will Allocate %d worker containers, each with %d memory".format( - numWorkers, - (workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD))) + if (numExecutors > 0) { + numPendingAllocate.addAndGet(numExecutors) + logInfo("Will Allocate %d executor containers, each with %d memory".format( + numExecutors, + (executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD))) } else { logDebug("Empty allocation request ...") } @@ -494,7 +494,7 @@ private[yarn] class YarnAllocationHandler( private def createResourceRequests( requestType: AllocationType.AllocationType, resource: String, - numWorkers: Int, + numExecutors: Int, priority: Int ): ArrayBuffer[ContainerRequest] = { @@ -507,7 +507,7 @@ private[yarn] class YarnAllocationHandler( val nodeLocal = constructContainerRequests( Array(hostname), racks = null, - numWorkers, + numExecutors, priority) // Add `hostname` to the global (singleton) host->rack mapping in YarnAllocationHandler. @@ -516,10 +516,10 @@ private[yarn] class YarnAllocationHandler( } case AllocationType.RACK => { val rack = resource - constructContainerRequests(hosts = null, Array(rack), numWorkers, priority) + constructContainerRequests(hosts = null, Array(rack), numExecutors, priority) } case AllocationType.ANY => constructContainerRequests( - hosts = null, racks = null, numWorkers, priority) + hosts = null, racks = null, numExecutors, priority) case _ => throw new IllegalArgumentException( "Unexpected/unsupported request type: " + requestType) } @@ -528,18 +528,18 @@ private[yarn] class YarnAllocationHandler( private def constructContainerRequests( hosts: Array[String], racks: Array[String], - numWorkers: Int, + numExecutors: Int, priority: Int ): ArrayBuffer[ContainerRequest] = { - val memoryRequest = workerMemory + YarnAllocationHandler.MEMORY_OVERHEAD - val resource = Resource.newInstance(memoryRequest, workerCores) + val memoryRequest = executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD + val resource = Resource.newInstance(memoryRequest, executorCores) val prioritySetting = Records.newRecord(classOf[Priority]) prioritySetting.setPriority(priority) val requests = new ArrayBuffer[ContainerRequest]() - for (i <- 0 until numWorkers) { + for (i <- 0 until numExecutors) { requests += new ContainerRequest(resource, hosts, racks, prioritySetting) } requests @@ -574,9 +574,9 @@ object YarnAllocationHandler { conf, amClient, appAttemptId, - args.numWorkers, - args.workerMemory, - args.workerCores, + args.numExecutors, + args.executorMemory, + args.executorCores, Map[String, Int](), Map[String, Int](), sparkConf) @@ -596,9 +596,9 @@ object YarnAllocationHandler { conf, amClient, appAttemptId, - args.numWorkers, - args.workerMemory, - args.workerCores, + args.numExecutors, + args.executorMemory, + args.executorCores, hostToSplitCount, rackToSplitCount, sparkConf) @@ -608,9 +608,9 @@ object YarnAllocationHandler { conf: Configuration, amClient: AMRMClient[ContainerRequest], appAttemptId: ApplicationAttemptId, - maxWorkers: Int, - workerMemory: Int, - workerCores: Int, + maxExecutors: Int, + executorMemory: Int, + executorCores: Int, map: collection.Map[String, collection.Set[SplitInfo]], sparkConf: SparkConf ): YarnAllocationHandler = { @@ -619,9 +619,9 @@ object YarnAllocationHandler { conf, amClient, appAttemptId, - maxWorkers, - workerMemory, - workerCores, + maxExecutors, + executorMemory, + executorCores, hostToCount, rackToCount, sparkConf) From ca4bf8c572c2f70b484830f1db414b5073744ab6 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 13 Mar 2014 12:16:04 -0700 Subject: [PATCH 090/397] SPARK-1236 - Upgrade Jetty to 9.1.3.v20140225. Author: Reynold Xin Closes #113 from rxin/jetty9 and squashes the following commits: 867a2ce [Reynold Xin] Updated Jetty version to 9.1.3.v20140225 in Maven build file. d7c97ca [Reynold Xin] Return the correctly bound port. d14706f [Reynold Xin] Upgrade Jetty to 9.1.3.v20140225. --- .../scala/org/apache/spark/HttpServer.scala | 21 ++++--- .../org/apache/spark/ui/JettyUtils.scala | 58 ++++++++++--------- pom.xml | 8 +-- project/SparkBuild.scala | 8 +-- 4 files changed, 49 insertions(+), 46 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index cb5df25fa48df..d14693cb78a2d 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -21,10 +21,9 @@ import java.io.File import org.eclipse.jetty.util.security.{Constraint, Password} import org.eclipse.jetty.security.authentication.DigestAuthenticator -import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService, SecurityHandler} +import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService} -import org.eclipse.jetty.server.Server -import org.eclipse.jetty.server.bio.SocketConnector +import org.eclipse.jetty.server.{Server, ServerConnector} import org.eclipse.jetty.server.handler.{DefaultHandler, HandlerList, ResourceHandler} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -43,7 +42,7 @@ private[spark] class ServerStateException(message: String) extends Exception(mes */ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityManager) extends Logging { - private var server: Server = null + private var server: Server = _ private var port: Int = -1 def start() { @@ -51,16 +50,16 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan throw new ServerStateException("Server is already started") } else { logInfo("Starting HTTP Server") - server = new Server() - val connector = new SocketConnector - connector.setMaxIdleTime(60*1000) + val threadPool = new QueuedThreadPool + threadPool.setDaemon(true) + + server = new Server(threadPool) + val connector = new ServerConnector(server) + connector.setIdleTimeout(60 * 1000) connector.setSoLingerTime(-1) connector.setPort(0) server.addConnector(connector) - val threadPool = new QueuedThreadPool - threadPool.setDaemon(true) - server.setThreadPool(threadPool) val resHandler = new ResourceHandler resHandler.setResourceBase(resourceBase.getAbsolutePath) @@ -79,7 +78,7 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan } server.start() - port = server.getConnectors()(0).getLocalPort() + port = connector.getLocalPort } } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 7c35cd165ad7c..cc7324939668b 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -19,6 +19,7 @@ package org.apache.spark.ui import java.net.InetSocketAddress import java.net.URL +import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletResponse, HttpServletRequest} import scala.annotation.tailrec @@ -28,7 +29,7 @@ import scala.xml.Node import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} -import org.eclipse.jetty.server.{DispatcherType, Server} +import org.eclipse.jetty.server.{NetworkConnector, Server} import org.eclipse.jetty.server.handler.HandlerList import org.eclipse.jetty.servlet.{DefaultServlet, FilterHolder, ServletContextHandler, ServletHolder} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -60,8 +61,7 @@ private[spark] object JettyUtils extends Logging { def createServlet[T <% AnyRef](servletParams: ServletParams[T], securityMgr: SecurityManager): HttpServlet = { new HttpServlet { - override def doGet(request: HttpServletRequest, - response: HttpServletResponse) { + override def doGet(request: HttpServletRequest, response: HttpServletResponse) { if (securityMgr.checkUIViewPermissions(request.getRemoteUser())) { response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) response.setStatus(HttpServletResponse.SC_OK) @@ -72,7 +72,7 @@ private[spark] object JettyUtils extends Logging { response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.sendError(HttpServletResponse.SC_UNAUTHORIZED, - "User is not authorized to access this page."); + "User is not authorized to access this page.") } } } @@ -120,26 +120,25 @@ private[spark] object JettyUtils extends Logging { private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) - filters.foreach { - case filter : String => - if (!filter.isEmpty) { - logInfo("Adding filter: " + filter) - val holder : FilterHolder = new FilterHolder() - holder.setClassName(filter) - // get any parameters for each filter - val paramName = "spark." + filter + ".params" - val params = conf.get(paramName, "").split(',').map(_.trim()).toSet - params.foreach { - case param : String => - if (!param.isEmpty) { - val parts = param.split("=") - if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) - } - } - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, - DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) - handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } + filters.foreach { filter => + if (!filter.isEmpty) { + logInfo("Adding filter: " + filter) + val holder : FilterHolder = new FilterHolder() + holder.setClassName(filter) + // get any parameters for each filter + val paramName = "spark." + filter + ".params" + val params = conf.get(paramName, "").split(',').map(_.trim()).toSet + params.foreach { + case param : String => + if (!param.isEmpty) { + val parts = param.split("=") + if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) + } } + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, + DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) + handlers.foreach { handler => handler.addFilter(holder, "/*", enumDispatcher) } + } } } @@ -150,7 +149,10 @@ private[spark] object JettyUtils extends Logging { * If the desired port number is contented, continues incrementing ports until a free port is * found. Returns the chosen port and the jetty Server object. */ - def startJettyServer(hostName: String, port: Int, handlers: Seq[ServletContextHandler], + def startJettyServer( + hostName: String, + port: Int, + handlers: Seq[ServletContextHandler], conf: SparkConf): (Server, Int) = { addFilters(handlers, conf) @@ -160,16 +162,18 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { val server = new Server(new InetSocketAddress(hostName, currentPort)) - val pool = new QueuedThreadPool + // Unfortunately Jetty 9 doesn't allow us to set both the thread pool and the port number in + // constructor. But fortunately the pool allocated by Jetty is always a QueuedThreadPool. + val pool = server.getThreadPool.asInstanceOf[QueuedThreadPool] pool.setDaemon(true) - server.setThreadPool(pool) + server.setHandler(handlerList) Try { server.start() } match { case s: Success[_] => - (server, server.getConnectors.head.getLocalPort) + (server, server.getConnectors.head.asInstanceOf[NetworkConnector].getLocalPort) case f: Failure[_] => server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) diff --git a/pom.xml b/pom.xml index d2c36dc260fa4..441e931cc8e77 100644 --- a/pom.xml +++ b/pom.xml @@ -158,22 +158,22 @@ org.eclipse.jetty jetty-util - 7.6.8.v20121106 + 9.1.3.v20140225 org.eclipse.jetty jetty-security - 7.6.8.v20121106 + 9.1.3.v20140225 org.eclipse.jetty jetty-plus - 7.6.8.v20121106 + 9.1.3.v20140225 org.eclipse.jetty jetty-server - 7.6.8.v20121106 + 9.1.3.v20140225 com.google.guava diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b0c3bf29dfd4f..bfc16e814b845 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -232,10 +232,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", - "org.eclipse.jetty" % "jetty-util" % "7.6.8.v20121106", - "org.eclipse.jetty" % "jetty-plus" % "7.6.8.v20121106", - "org.eclipse.jetty" % "jetty-security" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-server" % "9.1.3.v20140225", + "org.eclipse.jetty" % "jetty-util" % "9.1.3.v20140225", + "org.eclipse.jetty" % "jetty-plus" % "9.1.3.v20140225", + "org.eclipse.jetty" % "jetty-security" % "9.1.3.v20140225", /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"), "org.scalatest" %% "scalatest" % "1.9.1" % "test", From 181b130a0c2a0752009fdf5602e5d6d87f5b1212 Mon Sep 17 00:00:00 2001 From: Tianshuo Deng Date: Thu, 13 Mar 2014 20:27:36 -0700 Subject: [PATCH 091/397] [bugfix] wrong client arg, should use executor-cores client arg is wrong, it should be executor-cores. it causes executor fail to start when executor-cores is specified Author: Tianshuo Deng Closes #138 from tsdeng/bugfix_wrong_client_args and squashes the following commits: 304826d [Tianshuo Deng] wrong client arg, should use executor-cores --- .../scala/org/apache/spark/deploy/yarn/ClientArguments.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index a001060cdb746..c565f2dde24fc 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -95,7 +95,7 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { executorMemory = value args = tail - case ("--worker-cores" | "--executor-memory") :: IntParam(value) :: tail => + case ("--worker-cores" | "--executor-cores") :: IntParam(value) :: tail => if (args(0) == "--worker-cores") { println("--worker-cores is deprecated. Use --executor-cores instead.") } From e19044cb1048c3755d1ea2cb43879d2225d49b54 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 14 Mar 2014 11:40:26 -0700 Subject: [PATCH 092/397] Fix serialization of MutablePair. Also provide an interface for easy updating. Author: Michael Armbrust Closes #141 from marmbrus/mutablePair and squashes the following commits: f5c4783 [Michael Armbrust] Change function name to update 8bfd973 [Michael Armbrust] Fix serialization of MutablePair. Also provide an interface for easy updating. --- .../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..2c1a6f8fd0a44 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 update(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 97e4459e1e4cca8696535e10a91733c15f960107 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 15 Mar 2014 16:43:27 -0700 Subject: [PATCH 093/397] SPARK-1254. Consolidate, order, and harmonize repository declarations in Maven/SBT builds This suggestion addresses a few minor suboptimalities with how repositories are handled. 1) Use HTTPS consistently to access repos, instead of HTTP 2) Consolidate repository declarations in the parent POM file, in the case of the Maven build, so that their ordering can be controlled to put the fully optional Cloudera repo at the end, after required repos. (This was prompted by the untimely failure of the Cloudera repo this week, which made the Spark build fail. #2 would have prevented that.) 3) Update SBT build to match Maven build in this regard 4) Update SBT build to not refer to Sonatype snapshot repos. This wasn't in Maven, and a build generally would not refer to external snapshots, but I'm not 100% sure on this one. Author: Sean Owen Closes #145 from srowen/SPARK-1254 and squashes the following commits: 42f9bfc [Sean Owen] Use HTTPS for repos; consolidate repos in parent in order to put optional Cloudera repo last; harmonize SBT build repos with Maven; remove snapshot repos from SBT build which weren't in Maven --- examples/pom.xml | 15 --------------- external/mqtt/pom.xml | 14 -------------- pom.xml | 32 ++++++++++++++++++++++++++++++-- project/SparkBuild.scala | 23 ++++++++++++----------- streaming/pom.xml | 14 -------------- 5 files changed, 42 insertions(+), 56 deletions(-) diff --git a/examples/pom.xml b/examples/pom.xml index 9f0e2d0b875b8..382a38d9400b9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -45,21 +45,6 @@ - - - apache-repo - Apache Repository - https://repository.apache.org/content/repositories/releases - - true - - - false - - - - - org.apache.spark diff --git a/external/mqtt/pom.xml b/external/mqtt/pom.xml index 06c751df7f9c8..3710a63541d78 100644 --- a/external/mqtt/pom.xml +++ b/external/mqtt/pom.xml @@ -45,20 +45,6 @@ - - - mqtt-repo - MQTT Repository - https://repo.eclipse.org/content/repositories/paho-releases - - true - - - false - - - - org.apache.spark diff --git a/pom.xml b/pom.xml index 441e931cc8e77..377b6463136dd 100644 --- a/pom.xml +++ b/pom.xml @@ -127,7 +127,18 @@ maven-repo Maven Repository - http://repo.maven.apache.org/maven2 + https://repo.maven.apache.org/maven2 + + true + + + false + + + + apache-repo + Apache Repository + https://repository.apache.org/content/repositories/releases true @@ -138,7 +149,18 @@ jboss-repo JBoss Repository - http://repository.jboss.org/nexus/content/repositories/releases + https://repository.jboss.org/nexus/content/repositories/releases + + true + + + false + + + + mqtt-repo + MQTT Repository + https://repo.eclipse.org/content/repositories/paho-releases true @@ -150,6 +172,12 @@ cloudera-repo Cloudera Repository https://repository.cloudera.com/artifactory/cloudera-repos + + true + + + false + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index bfc16e814b845..db24f124f3cf8 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -174,12 +174,18 @@ object SparkBuild extends Build { // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), - // also check the local Maven repository ~/.m2 - resolvers ++= Seq(Resolver.file("Local Maven Repo", file(Path.userHome + "/.m2/repository"))), - - // For Sonatype publishing - resolvers ++= Seq("sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", - "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/"), + resolvers ++= Seq( + "Maven Repository" at "https://repo.maven.apache.org/maven2", + "Apache Repository" at "https://repository.apache.org/content/repositories/releases", + "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", + "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", + "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", + // For Sonatype publishing + //"sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", + //"sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", + // also check the local Maven repository ~/.m2 + Resolver.mavenLocal + ), publishMavenStyle := true, @@ -272,10 +278,6 @@ object SparkBuild extends Build { def coreSettings = sharedSettings ++ Seq( name := "spark-core", - resolvers ++= Seq( - "JBoss Repository" at "http://repository.jboss.org/nexus/content/repositories/releases/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/" - ), libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", @@ -470,7 +472,6 @@ object SparkBuild extends Build { def mqttSettings() = streamingSettings ++ Seq( name := "spark-streaming-mqtt", - resolvers ++= Seq("Eclipse Repo" at "https://repo.eclipse.org/content/repositories/paho-releases/"), libraryDependencies ++= Seq("org.eclipse.paho" % "mqtt-client" % "0.4.0") ) } diff --git a/streaming/pom.xml b/streaming/pom.xml index 2343e381e6f7c..2cb8bde6642bc 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -45,20 +45,6 @@ - - - apache-repo - Apache Repository - https://repository.apache.org/content/repositories/releases - - true - - - false - - - - org.apache.spark From f5486e9f75d62919583da5ecf9a9ad00222b2227 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Sun, 16 Mar 2014 09:57:21 -0700 Subject: [PATCH 094/397] SPARK-1255: Allow user to pass Serializer object instead of class name for shuffle. This is more general than simply passing a string name and leaves more room for performance optimizations. Note that this is technically an API breaking change in the following two ways: 1. The shuffle serializer specification in ShuffleDependency now require an object instead of a String (of the class name), but I suspect nobody else in this world has used this API other than me in GraphX and Shark. 2. Serializer's in Spark from now on are required to be serializable. Author: Reynold Xin Closes #149 from rxin/serializer and squashes the following commits: 5acaccd [Reynold Xin] Properly call serializer's constructors. 2a8d75a [Reynold Xin] Added more documentation for the serializer option in ShuffleDependency. 7420185 [Reynold Xin] Allow user to pass Serializer object instead of class name for shuffle. --- .../scala/org/apache/spark/Dependency.scala | 6 +- .../org/apache/spark/ShuffleFetcher.scala | 2 +- .../scala/org/apache/spark/SparkEnv.scala | 24 +++--- .../org/apache/spark/rdd/CoGroupedRDD.scala | 18 ++--- .../apache/spark/rdd/PairRDDFunctions.scala | 7 +- .../org/apache/spark/rdd/ShuffledRDD.scala | 13 ++-- .../org/apache/spark/rdd/SubtractedRDD.scala | 20 ++--- .../spark/scheduler/ShuffleMapTask.scala | 3 +- .../spark/serializer/JavaSerializer.scala | 27 ++++--- .../spark/serializer/KryoSerializer.scala | 16 ++-- .../apache/spark/serializer/Serializer.scala | 16 +++- .../spark/serializer/SerializerManager.scala | 75 ------------------- .../collection/ExternalAppendOnlyMap.scala | 2 +- .../scala/org/apache/spark/ShuffleSuite.scala | 9 ++- .../apache/spark/graphx/impl/GraphImpl.scala | 2 +- .../graphx/impl/MessageToPartition.scala | 12 +-- .../spark/graphx/impl/Serializers.scala | 14 ++-- .../apache/spark/graphx/SerializerSuite.scala | 30 +++----- 18 files changed, 125 insertions(+), 171 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index cc30105940d1a..448f87b81ef4a 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -18,6 +18,7 @@ package org.apache.spark import org.apache.spark.rdd.RDD +import org.apache.spark.serializer.Serializer /** * Base class for dependencies. @@ -43,12 +44,13 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output - * @param serializerClass class name of the serializer to use + * @param serializer [[Serializer]] to use. If set to null, the default serializer, as specified + * by `spark.serializer` config option, will be used. */ class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, - val serializerClass: String = null) + val serializer: Serializer = null) extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() diff --git a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala index e8f756c408889..a4f69b6b22b2c 100644 --- a/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala +++ b/core/src/main/scala/org/apache/spark/ShuffleFetcher.scala @@ -29,7 +29,7 @@ private[spark] abstract class ShuffleFetcher { shuffleId: Int, reduceId: Int, context: TaskContext, - serializer: Serializer = SparkEnv.get.serializerManager.default): Iterator[T] + serializer: Serializer = SparkEnv.get.serializer): Iterator[T] /** Stop the fetcher */ def stop() {} diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5e43b5198422c..d035d909b7b25 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -28,7 +28,7 @@ import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem import org.apache.spark.storage.{BlockManager, BlockManagerMaster, BlockManagerMasterActor} import org.apache.spark.network.ConnectionManager -import org.apache.spark.serializer.{Serializer, SerializerManager} +import org.apache.spark.serializer.Serializer import org.apache.spark.util.{AkkaUtils, Utils} /** @@ -41,7 +41,6 @@ import org.apache.spark.util.{AkkaUtils, Utils} class SparkEnv private[spark] ( val executorId: String, val actorSystem: ActorSystem, - val serializerManager: SerializerManager, val serializer: Serializer, val closureSerializer: Serializer, val cacheManager: CacheManager, @@ -139,16 +138,22 @@ object SparkEnv extends Logging { // defaultClassName if the property is not set, and return it as a T def instantiateClass[T](propertyName: String, defaultClassName: String): T = { val name = conf.get(propertyName, defaultClassName) - Class.forName(name, true, classLoader).newInstance().asInstanceOf[T] + val cls = Class.forName(name, true, classLoader) + // First try with the constructor that takes SparkConf. If we can't find one, + // use a no-arg constructor instead. + try { + cls.getConstructor(classOf[SparkConf]).newInstance(conf).asInstanceOf[T] + } catch { + case _: NoSuchMethodException => + cls.getConstructor().newInstance().asInstanceOf[T] + } } - val serializerManager = new SerializerManager - val serializer = serializerManager.setDefault( - conf.get("spark.serializer", "org.apache.spark.serializer.JavaSerializer"), conf) + val serializer = instantiateClass[Serializer]( + "spark.serializer", "org.apache.spark.serializer.JavaSerializer") - val closureSerializer = serializerManager.get( - conf.get("spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer"), - conf) + val closureSerializer = instantiateClass[Serializer]( + "spark.closure.serializer", "org.apache.spark.serializer.JavaSerializer") def registerOrLookup(name: String, newActor: => Actor): ActorRef = { if (isDriver) { @@ -220,7 +225,6 @@ object SparkEnv extends Logging { new SparkEnv( executorId, actorSystem, - serializerManager, serializer, closureSerializer, cacheManager, diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 699a10c96c227..8561711931047 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -24,6 +24,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} +import org.apache.spark.serializer.Serializer private[spark] sealed trait CoGroupSplitDep extends Serializable @@ -66,10 +67,10 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: private type CoGroupValue = (Any, Int) // Int is dependency number private type CoGroupCombiner = Seq[CoGroup] - private var serializerClass: String = null + private var serializer: Serializer = null - def setSerializer(cls: String): CoGroupedRDD[K] = { - serializerClass = cls + def setSerializer(serializer: Serializer): CoGroupedRDD[K] = { + this.serializer = serializer this } @@ -80,7 +81,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: new OneToOneDependency(rdd) } else { logDebug("Adding shuffle dependency with " + rdd) - new ShuffleDependency[Any, Any](rdd, part, serializerClass) + new ShuffleDependency[Any, Any](rdd, part, serializer) } } } @@ -113,18 +114,17 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: // A list of (rdd iterator, dependency number) pairs val rddIterators = new ArrayBuffer[(Iterator[Product2[K, Any]], Int)] for ((dep, depNum) <- split.deps.zipWithIndex) dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => // Read them from the parent val it = rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, Any]]] rddIterators += ((it, depNum)) - } - case ShuffleCoGroupSplitDep(shuffleId) => { + + case ShuffleCoGroupSplitDep(shuffleId) => // Read map outputs of shuffle val fetcher = SparkEnv.get.shuffleFetcher - val ser = SparkEnv.get.serializerManager.get(serializerClass, sparkConf) + val ser = Serializer.getSerializer(serializer) val it = fetcher.fetch[Product2[K, Any]](shuffleId, split.index, context, ser) rddIterators += ((it, depNum)) - } } if (!externalSorting) { diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index b20ed99f89f34..b0d322fe27bd5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -44,6 +44,7 @@ import org.apache.spark._ import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} +import org.apache.spark.serializer.Serializer import org.apache.spark.util.SerializableHyperLogLog /** @@ -73,7 +74,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) mergeCombiners: (C, C) => C, partitioner: Partitioner, mapSideCombine: Boolean = true, - serializerClass: String = null): RDD[(K, C)] = { + serializer: Serializer = null): RDD[(K, C)] = { require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0 if (getKeyClass().isArray) { if (mapSideCombine) { @@ -93,13 +94,13 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) aggregator.combineValuesByKey(iter, context) }, preservesPartitioning = true) val partitioned = new ShuffledRDD[K, C, (K, C)](combined, partitioner) - .setSerializer(serializerClass) + .setSerializer(serializer) partitioned.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineCombinersByKey(iter, context)) }, preservesPartitioning = true) } else { // Don't apply map-side combiner. - val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializerClass) + val values = new ShuffledRDD[K, V, (K, V)](self, partitioner).setSerializer(serializer) values.mapPartitionsWithContext((context, iter) => { new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) }, preservesPartitioning = true) diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 0bbda25a905cd..02660ea6a45c5 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { override val index = idx @@ -38,15 +39,15 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( part: Partitioner) extends RDD[P](prev.context, Nil) { - private var serializerClass: String = null + private var serializer: Serializer = null - def setSerializer(cls: String): ShuffledRDD[K, V, P] = { - serializerClass = cls + def setSerializer(serializer: Serializer): ShuffledRDD[K, V, P] = { + this.serializer = serializer this } override def getDependencies: Seq[Dependency[_]] = { - List(new ShuffleDependency(prev, part, serializerClass)) + List(new ShuffleDependency(prev, part, serializer)) } override val partitioner = Some(part) @@ -57,8 +58,8 @@ class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( override def compute(split: Partition, context: TaskContext): Iterator[P] = { val shuffledId = dependencies.head.asInstanceOf[ShuffleDependency[K, V]].shuffleId - SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context, - SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf)) + val ser = Serializer.getSerializer(serializer) + SparkEnv.get.shuffleFetcher.fetch[P](shuffledId, split.index, context, ser) } override def clearDependencies() { diff --git a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala index 5fe9f363db453..9a09c05bbc959 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SubtractedRDD.scala @@ -30,6 +30,7 @@ import org.apache.spark.Partitioner import org.apache.spark.ShuffleDependency import org.apache.spark.SparkEnv import org.apache.spark.TaskContext +import org.apache.spark.serializer.Serializer /** * An optimized version of cogroup for set difference/subtraction. @@ -53,10 +54,10 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( part: Partitioner) extends RDD[(K, V)](rdd1.context, Nil) { - private var serializerClass: String = null + private var serializer: Serializer = null - def setSerializer(cls: String): SubtractedRDD[K, V, W] = { - serializerClass = cls + def setSerializer(serializer: Serializer): SubtractedRDD[K, V, W] = { + this.serializer = serializer this } @@ -67,7 +68,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( new OneToOneDependency(rdd) } else { logDebug("Adding shuffle dependency with " + rdd) - new ShuffleDependency(rdd, part, serializerClass) + new ShuffleDependency(rdd, part, serializer) } } } @@ -92,7 +93,7 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( override def compute(p: Partition, context: TaskContext): Iterator[(K, V)] = { val partition = p.asInstanceOf[CoGroupPartition] - val serializer = SparkEnv.get.serializerManager.get(serializerClass, SparkEnv.get.conf) + val ser = Serializer.getSerializer(serializer) val map = new JHashMap[K, ArrayBuffer[V]] def getSeq(k: K): ArrayBuffer[V] = { val seq = map.get(k) @@ -105,14 +106,13 @@ private[spark] class SubtractedRDD[K: ClassTag, V: ClassTag, W: ClassTag]( } } def integrate(dep: CoGroupSplitDep, op: Product2[K, V] => Unit) = dep match { - case NarrowCoGroupSplitDep(rdd, _, itsSplit) => { + case NarrowCoGroupSplitDep(rdd, _, itsSplit) => rdd.iterator(itsSplit, context).asInstanceOf[Iterator[Product2[K, V]]].foreach(op) - } - case ShuffleCoGroupSplitDep(shuffleId) => { + + case ShuffleCoGroupSplitDep(shuffleId) => val iter = SparkEnv.get.shuffleFetcher.fetch[Product2[K, V]](shuffleId, partition.index, - context, serializer) + context, ser) iter.foreach(op) - } } // the first dep is rdd1; add all values to the map integrate(partition.deps(0), t => getSeq(t._1) += t._2) diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 77789031f464a..2a9edf4a76b97 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -26,6 +26,7 @@ import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics import org.apache.spark.rdd.RDD import org.apache.spark.rdd.RDDCheckpointData +import org.apache.spark.serializer.Serializer import org.apache.spark.storage._ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} @@ -153,7 +154,7 @@ private[spark] class ShuffleMapTask( try { // Obtain all the block writers for shuffle blocks. - val ser = SparkEnv.get.serializerManager.get(dep.serializerClass, SparkEnv.get.conf) + val ser = Serializer.getSerializer(dep.serializer) shuffle = shuffleBlockManager.forMapTask(dep.shuffleId, partitionId, numOutputSplits, ser) // Write the map output to its associated buckets. diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index bfa647f7f0516..18a68b05fa853 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -23,11 +23,10 @@ import java.nio.ByteBuffer import org.apache.spark.SparkConf import org.apache.spark.util.ByteBufferInputStream -private[spark] class JavaSerializationStream(out: OutputStream, conf: SparkConf) +private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) extends SerializationStream { - val objOut = new ObjectOutputStream(out) - var counter = 0 - val counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) + private val objOut = new ObjectOutputStream(out) + private var counter = 0 /** * Calling reset to avoid memory leak: @@ -51,7 +50,7 @@ private[spark] class JavaSerializationStream(out: OutputStream, conf: SparkConf) private[spark] class JavaDeserializationStream(in: InputStream, loader: ClassLoader) extends DeserializationStream { - val objIn = new ObjectInputStream(in) { + private val objIn = new ObjectInputStream(in) { override def resolveClass(desc: ObjectStreamClass) = Class.forName(desc.getName, false, loader) } @@ -60,7 +59,7 @@ extends DeserializationStream { def close() { objIn.close() } } -private[spark] class JavaSerializerInstance(conf: SparkConf) extends SerializerInstance { +private[spark] class JavaSerializerInstance(counterReset: Int) extends SerializerInstance { def serialize[T](t: T): ByteBuffer = { val bos = new ByteArrayOutputStream() val out = serializeStream(bos) @@ -82,7 +81,7 @@ private[spark] class JavaSerializerInstance(conf: SparkConf) extends SerializerI } def serializeStream(s: OutputStream): SerializationStream = { - new JavaSerializationStream(s, conf) + new JavaSerializationStream(s, counterReset) } def deserializeStream(s: InputStream): DeserializationStream = { @@ -97,6 +96,16 @@ private[spark] class JavaSerializerInstance(conf: SparkConf) extends SerializerI /** * A Spark serializer that uses Java's built-in serialization. */ -class JavaSerializer(conf: SparkConf) extends Serializer { - def newInstance(): SerializerInstance = new JavaSerializerInstance(conf) +class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { + private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) + + def newInstance(): SerializerInstance = new JavaSerializerInstance(counterReset) + + override def writeExternal(out: ObjectOutput) { + out.writeInt(counterReset) + } + + override def readExternal(in: ObjectInput) { + counterReset = in.readInt() + } } 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..6b6d814c1fe92 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -34,10 +34,14 @@ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. */ -class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer with Logging { - private val bufferSize = { - conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024 - } +class KryoSerializer(conf: SparkConf) + extends org.apache.spark.serializer.Serializer + with Logging + with Serializable { + + private val bufferSize = conf.getInt("spark.kryoserializer.buffer.mb", 2) * 1024 * 1024 + private val referenceTracking = conf.getBoolean("spark.kryo.referenceTracking", true) + private val registrator = conf.getOption("spark.kryo.registrator") def newKryoOutput() = new KryoOutput(bufferSize) @@ -48,7 +52,7 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial // Allow disabling Kryo reference tracking if user knows their object graphs don't have loops. // Do this before we invoke the user registrator so the user registrator can override this. - kryo.setReferences(conf.getBoolean("spark.kryo.referenceTracking", true)) + kryo.setReferences(referenceTracking) for (cls <- KryoSerializer.toRegister) kryo.register(cls) @@ -58,7 +62,7 @@ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serial // Allow the user to register their own classes by setting spark.kryo.registrator try { - for (regCls <- conf.getOption("spark.kryo.registrator")) { + for (regCls <- registrator) { logDebug("Running user registrator: " + regCls) val reg = Class.forName(regCls, true, classLoader).newInstance() .asInstanceOf[KryoRegistrator] diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 16677ab54be04..099143494b851 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -23,21 +23,31 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream import org.apache.spark.util.{ByteBufferInputStream, NextIterator} +import org.apache.spark.SparkEnv /** * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual * serialization and are guaranteed to only be called from one thread at a time. * - * Implementations of this trait should have a zero-arg constructor or a constructor that accepts a - * [[org.apache.spark.SparkConf]] as parameter. If both constructors are defined, the latter takes - * precedence. + * Implementations of this trait should implement: + * 1. a zero-arg constructor or a constructor that accepts a [[org.apache.spark.SparkConf]] + * as parameter. If both constructors are defined, the latter takes precedence. + * + * 2. Java serialization interface. */ trait Serializer { def newInstance(): SerializerInstance } +object Serializer { + def getSerializer(serializer: Serializer): Serializer = { + if (serializer == null) SparkEnv.get.serializer else serializer + } +} + + /** * An instance of a serializer, for use by one thread at a time. */ diff --git a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala b/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala deleted file mode 100644 index 65ac0155f45e7..0000000000000 --- a/core/src/main/scala/org/apache/spark/serializer/SerializerManager.scala +++ /dev/null @@ -1,75 +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.serializer - -import java.util.concurrent.ConcurrentHashMap - -import org.apache.spark.SparkConf - -/** - * A service that returns a serializer object given the serializer's class name. If a previous - * instance of the serializer object has been created, the get method returns that instead of - * creating a new one. - */ -private[spark] class SerializerManager { - // TODO: Consider moving this into SparkConf itself to remove the global singleton. - - private val serializers = new ConcurrentHashMap[String, Serializer] - private var _default: Serializer = _ - - def default = _default - - def setDefault(clsName: String, conf: SparkConf): Serializer = { - _default = get(clsName, conf) - _default - } - - def get(clsName: String, conf: SparkConf): Serializer = { - if (clsName == null) { - default - } else { - var serializer = serializers.get(clsName) - if (serializer != null) { - // If the serializer has been created previously, reuse that. - serializer - } else this.synchronized { - // Otherwise, create a new one. But make sure no other thread has attempted - // to create another new one at the same time. - serializer = serializers.get(clsName) - if (serializer == null) { - val clsLoader = Thread.currentThread.getContextClassLoader - val cls = Class.forName(clsName, true, clsLoader) - - // First try with the constructor that takes SparkConf. If we can't find one, - // use a no-arg constructor instead. - try { - val constructor = cls.getConstructor(classOf[SparkConf]) - serializer = constructor.newInstance(conf).asInstanceOf[Serializer] - } catch { - case _: NoSuchMethodException => - val constructor = cls.getConstructor() - serializer = constructor.newInstance().asInstanceOf[Serializer] - } - - serializers.put(clsName, serializer) - } - serializer - } - } - } -} diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index ed74a31f05bae..caa06d5b445b4 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -60,7 +60,7 @@ private[spark] class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, - serializer: Serializer = SparkEnv.get.serializerManager.default, + serializer: Serializer = SparkEnv.get.serializer, blockManager: BlockManager = SparkEnv.get.blockManager) extends Iterable[(K, C)] with Serializable with Logging { diff --git a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala index abea36f7c83df..be6508a40ea61 100644 --- a/core/src/test/scala/org/apache/spark/ShuffleSuite.scala +++ b/core/src/test/scala/org/apache/spark/ShuffleSuite.scala @@ -27,6 +27,9 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { + + val conf = new SparkConf(loadDefaults = false) + test("groupByKey without compression") { try { System.setProperty("spark.shuffle.compress", "false") @@ -54,7 +57,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // If the Kryo serializer is not used correctly, the shuffle would fail because the // default Java serializer cannot handle the non serializable class. val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( - b, new HashPartitioner(NUM_BLOCKS)).setSerializer(classOf[KryoSerializer].getName) + b, new HashPartitioner(NUM_BLOCKS)).setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 10) @@ -76,7 +79,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // If the Kryo serializer is not used correctly, the shuffle would fail because the // default Java serializer cannot handle the non serializable class. val c = new ShuffledRDD[Int, NonJavaSerializableClass, (Int, NonJavaSerializableClass)]( - b, new HashPartitioner(3)).setSerializer(classOf[KryoSerializer].getName) + b, new HashPartitioner(3)).setSerializer(new KryoSerializer(conf)) assert(c.count === 10) } @@ -92,7 +95,7 @@ class ShuffleSuite extends FunSuite with ShouldMatchers with LocalSparkContext { // NOTE: The default Java serializer doesn't create zero-sized blocks. // So, use Kryo val c = new ShuffledRDD[Int, Int, (Int, Int)](b, new HashPartitioner(10)) - .setSerializer(classOf[KryoSerializer].getName) + .setSerializer(new KryoSerializer(conf)) val shuffleId = c.dependencies.head.asInstanceOf[ShuffleDependency[Int, Int]].shuffleId assert(c.count === 4) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 1d029bf009e8c..5e9be18990ba3 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -391,6 +391,6 @@ object GraphImpl { // TODO: Consider doing map side distinct before shuffle. new ShuffledRDD[VertexId, Int, (VertexId, Int)]( edges.collectVertexIds.map(vid => (vid, 0)), partitioner) - .setSerializer(classOf[VertexIdMsgSerializer].getName) + .setSerializer(new VertexIdMsgSerializer) } } // end of object GraphImpl diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index e9ee09c3614c1..fe6fe76defdc5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -65,11 +65,11 @@ class VertexBroadcastMsgRDDFunctions[T: ClassTag](self: RDD[VertexBroadcastMsg[T // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { - rdd.setSerializer(classOf[IntVertexBroadcastMsgSerializer].getName) + rdd.setSerializer(new IntVertexBroadcastMsgSerializer) } else if (classTag[T] == ClassTag.Long) { - rdd.setSerializer(classOf[LongVertexBroadcastMsgSerializer].getName) + rdd.setSerializer(new LongVertexBroadcastMsgSerializer) } else if (classTag[T] == ClassTag.Double) { - rdd.setSerializer(classOf[DoubleVertexBroadcastMsgSerializer].getName) + rdd.setSerializer(new DoubleVertexBroadcastMsgSerializer) } rdd } @@ -104,11 +104,11 @@ object MsgRDDFunctions { // Set a custom serializer if the data is of int or double type. if (classTag[T] == ClassTag.Int) { - rdd.setSerializer(classOf[IntAggMsgSerializer].getName) + rdd.setSerializer(new IntAggMsgSerializer) } else if (classTag[T] == ClassTag.Long) { - rdd.setSerializer(classOf[LongAggMsgSerializer].getName) + rdd.setSerializer(new LongAggMsgSerializer) } else if (classTag[T] == ClassTag.Double) { - rdd.setSerializer(classOf[DoubleAggMsgSerializer].getName) + rdd.setSerializer(new DoubleAggMsgSerializer) } rdd } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index c74d487e206db..34a145e01818f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -25,7 +25,7 @@ import org.apache.spark.graphx._ import org.apache.spark.serializer._ private[graphx] -class VertexIdMsgSerializer(conf: SparkConf) extends Serializer { +class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -46,7 +46,7 @@ class VertexIdMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for VertexBroadcastMessage[Int]. */ private[graphx] -class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { +class IntVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -70,7 +70,7 @@ class IntVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for VertexBroadcastMessage[Long]. */ private[graphx] -class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { +class LongVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -94,7 +94,7 @@ class LongVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for VertexBroadcastMessage[Double]. */ private[graphx] -class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { +class DoubleVertexBroadcastMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -118,7 +118,7 @@ class DoubleVertexBroadcastMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for AggregationMessage[Int]. */ private[graphx] -class IntAggMsgSerializer(conf: SparkConf) extends Serializer { +class IntAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -142,7 +142,7 @@ class IntAggMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for AggregationMessage[Long]. */ private[graphx] -class LongAggMsgSerializer(conf: SparkConf) extends Serializer { +class LongAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { @@ -166,7 +166,7 @@ class LongAggMsgSerializer(conf: SparkConf) extends Serializer { /** A special shuffle serializer for AggregationMessage[Double]. */ private[graphx] -class DoubleAggMsgSerializer(conf: SparkConf) extends Serializer { +class DoubleAggMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { override def serializeStream(s: OutputStream) = new ShuffleSerializationStream(s) { diff --git a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala index e5a582b47ba05..73438d9535962 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/SerializerSuite.scala @@ -32,15 +32,14 @@ import org.apache.spark.serializer.SerializationStream class SerializerSuite extends FunSuite with LocalSparkContext { test("IntVertexBroadcastMsgSerializer") { - val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Int](3, 4, 5) val bout = new ByteArrayOutputStream - val outStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new IntVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new IntVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Int] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Int] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -54,15 +53,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongVertexBroadcastMsgSerializer") { - val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Long](3, 4, 5) val bout = new ByteArrayOutputStream - val outStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new LongVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new LongVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Long] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Long] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -76,15 +74,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleVertexBroadcastMsgSerializer") { - val conf = new SparkConf(false) val outMsg = new VertexBroadcastMsg[Double](3, 4, 5.0) val bout = new ByteArrayOutputStream - val outStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleVertexBroadcastMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new DoubleVertexBroadcastMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: VertexBroadcastMsg[Double] = inStrm.readObject() val inMsg2: VertexBroadcastMsg[Double] = inStrm.readObject() assert(outMsg.vid === inMsg1.vid) @@ -98,15 +95,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("IntAggMsgSerializer") { - val conf = new SparkConf(false) val outMsg = (4: VertexId, 5) val bout = new ByteArrayOutputStream - val outStrm = new IntAggMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new IntAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new IntAggMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new IntAggMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: (VertexId, Int) = inStrm.readObject() val inMsg2: (VertexId, Int) = inStrm.readObject() assert(outMsg === inMsg1) @@ -118,15 +114,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("LongAggMsgSerializer") { - val conf = new SparkConf(false) val outMsg = (4: VertexId, 1L << 32) val bout = new ByteArrayOutputStream - val outStrm = new LongAggMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new LongAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new LongAggMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new LongAggMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: (VertexId, Long) = inStrm.readObject() val inMsg2: (VertexId, Long) = inStrm.readObject() assert(outMsg === inMsg1) @@ -138,15 +133,14 @@ class SerializerSuite extends FunSuite with LocalSparkContext { } test("DoubleAggMsgSerializer") { - val conf = new SparkConf(false) val outMsg = (4: VertexId, 5.0) val bout = new ByteArrayOutputStream - val outStrm = new DoubleAggMsgSerializer(conf).newInstance().serializeStream(bout) + val outStrm = new DoubleAggMsgSerializer().newInstance().serializeStream(bout) outStrm.writeObject(outMsg) outStrm.writeObject(outMsg) bout.flush() val bin = new ByteArrayInputStream(bout.toByteArray) - val inStrm = new DoubleAggMsgSerializer(conf).newInstance().deserializeStream(bin) + val inStrm = new DoubleAggMsgSerializer().newInstance().deserializeStream(bin) val inMsg1: (VertexId, Double) = inStrm.readObject() val inMsg2: (VertexId, Double) = inStrm.readObject() assert(outMsg === inMsg1) From dc9654638f1d781ee1e54348fa41436b27793365 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Sun, 16 Mar 2014 22:14:59 -0700 Subject: [PATCH 095/397] SPARK-1240: handle the case of empty RDD when takeSample https://spark-project.atlassian.net/browse/SPARK-1240 It seems that the current implementation does not handle the empty RDD case when run takeSample In this patch, before calling sample() inside takeSample API, I add a checker for this case and returns an empty Array when it's a empty RDD; also in sample(), I add a checker for the invalid fraction value In the test case, I also add several lines for this case Author: CodingCat Closes #135 from CodingCat/SPARK-1240 and squashes the following commits: fef57d4 [CodingCat] fix the same problem in PySpark 36db06b [CodingCat] create new test cases for takeSample from an empty red 810948d [CodingCat] further fix a40e8fb [CodingCat] replace if with require ad483fd [CodingCat] handle the case with empty RDD when take sample --- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 7 ++++++- core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala | 7 +++++++ python/pyspark/rdd.py | 4 ++++ 3 files changed, 17 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index b50c9963b9d2c..f8283fbbb980d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -310,6 +310,7 @@ abstract class RDD[T: ClassTag]( * Return a sampled subset of this RDD. */ def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] = { + require(fraction >= 0.0, "Invalid fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) } else { @@ -344,6 +345,10 @@ abstract class RDD[T: ClassTag]( throw new IllegalArgumentException("Negative number of elements requested") } + if (initialCount == 0) { + return new Array[T](0) + } + if (initialCount > Integer.MAX_VALUE - 1) { maxSelected = Integer.MAX_VALUE - 1 } else { @@ -362,7 +367,7 @@ abstract class RDD[T: ClassTag]( var samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() // If the first sample didn't turn out large enough, keep trying to take samples; - // this shouldn't happen often because we use a big multiplier for thei initial size + // this shouldn't happen often because we use a big multiplier for the initial size while (samples.length < total) { samples = this.sample(withReplacement, fraction, rand.nextInt()).collect() } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 60bcada55245b..9512e0e6eeb14 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -457,6 +457,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("takeSample") { val data = sc.parallelize(1 to 100, 2) + for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) assert(sample.size === 20) // Got exactly 20 elements @@ -488,6 +489,12 @@ class RDDSuite extends FunSuite with SharedSparkContext { } } + test("takeSample from an empty rdd") { + val emptySet = sc.parallelize(Seq.empty[Int], 2) + val sample = emptySet.takeSample(false, 20, 1) + assert(sample.length === 0) + } + test("randomSplit") { val n = 600 val data = sc.parallelize(1 to n, 2) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 6d549b40e5698..f3b432ff248a9 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -268,6 +268,7 @@ def sample(self, withReplacement, fraction, seed): >>> sc.parallelize(range(0, 100)).sample(False, 0.1, 2).collect() #doctest: +SKIP [2, 3, 20, 21, 24, 41, 42, 66, 67, 89, 90, 98] """ + assert fraction >= 0.0, "Invalid fraction value: %s" % fraction return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) # this is ported from scala/spark/RDD.scala @@ -288,6 +289,9 @@ def takeSample(self, withReplacement, num, seed): if (num < 0): raise ValueError + if (initialCount == 0): + return list() + if initialCount > sys.maxint - 1: maxSelected = sys.maxint - 1 else: From 796977acdb5c96ca5c08591657137fb3e44d2e94 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 17 Mar 2014 14:03:32 -0700 Subject: [PATCH 096/397] SPARK-1244: Throw exception if map output status exceeds frame size This is a very small change on top of @andrewor14's patch in #147. Author: Patrick Wendell Author: Andrew Or Closes #152 from pwendell/akka-frame and squashes the following commits: e5fb3ff [Patrick Wendell] Reversing test order 393af4c [Patrick Wendell] Small improvement suggested by Andrew Or 8045103 [Patrick Wendell] Breaking out into two tests 2b4e085 [Patrick Wendell] Consolidate Executor use of akka frame size c9b6109 [Andrew Or] Simplify test + make access to akka frame size more modular 281d7c9 [Andrew Or] Throw exception on spark.akka.frameSize exceeded + Unit tests --- .../org/apache/spark/MapOutputTracker.scala | 19 +++++- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 6 +- .../org/apache/spark/util/AkkaUtils.scala | 9 ++- .../org/apache/spark/AkkaUtilsSuite.scala | 10 ++-- .../apache/spark/MapOutputTrackerSuite.scala | 58 +++++++++++++++++-- 6 files changed, 84 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 5968973132942..80cbf951cb70e 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -35,13 +35,28 @@ private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage -private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster) +private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) extends Actor with Logging { + val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) + def receive = { case GetMapOutputStatuses(shuffleId: Int) => val hostPort = sender.path.address.hostPort logInfo("Asked to send map output locations for shuffle " + shuffleId + " to " + hostPort) - sender ! tracker.getSerializedMapOutputStatuses(shuffleId) + val mapOutputStatuses = tracker.getSerializedMapOutputStatuses(shuffleId) + val serializedSize = mapOutputStatuses.size + if (serializedSize > maxAkkaFrameSize) { + val msg = s"Map output statuses were $serializedSize bytes which " + + s"exceeds spark.akka.frameSize ($maxAkkaFrameSize bytes)." + + /* For SPARK-1244 we'll opt for just logging an error and then throwing an exception. + * Note that on exception the actor will just restart. A bigger refactoring (SPARK-1239) + * will ultimately remove this entire code path. */ + val exception = new SparkException(msg) + logError(msg, exception) + throw exception + } + sender ! mapOutputStatuses case StopMapOutputTracker => logInfo("MapOutputTrackerActor stopped!") diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index d035d909b7b25..774cbd6441a48 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -191,7 +191,7 @@ object SparkEnv extends Logging { } mapOutputTracker.trackerActor = registerOrLookup( "MapOutputTracker", - new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster])) + new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) val shuffleFetcher = instantiateClass[ShuffleFetcher]( "spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher") diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index e69f6f72d3275..2ea2ec29f59f5 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -29,7 +29,7 @@ import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ import org.apache.spark.storage.{StorageLevel, TaskResultBlockId} -import org.apache.spark.util.Utils +import org.apache.spark.util.{AkkaUtils, Utils} /** * Spark executor used with Mesos, YARN, and the standalone scheduler. @@ -120,9 +120,7 @@ private[spark] class Executor( // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. - private val akkaFrameSize = { - env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size") - } + private val akkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) // Start worker thread pool val threadPool = Utils.newDaemonCachedThreadPool("Executor task launch worker") diff --git a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala index a6c9a9aaba8eb..d0ff17db632c1 100644 --- a/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala +++ b/core/src/main/scala/org/apache/spark/util/AkkaUtils.scala @@ -49,7 +49,7 @@ private[spark] object AkkaUtils extends Logging { val akkaTimeout = conf.getInt("spark.akka.timeout", 100) - val akkaFrameSize = conf.getInt("spark.akka.frameSize", 10) + val akkaFrameSize = maxFrameSizeBytes(conf) val akkaLogLifecycleEvents = conf.getBoolean("spark.akka.logLifecycleEvents", false) val lifecycleEvents = if (akkaLogLifecycleEvents) "on" else "off" if (!akkaLogLifecycleEvents) { @@ -92,7 +92,7 @@ private[spark] object AkkaUtils extends Logging { |akka.remote.netty.tcp.port = $port |akka.remote.netty.tcp.tcp-nodelay = on |akka.remote.netty.tcp.connection-timeout = $akkaTimeout s - |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}MiB + |akka.remote.netty.tcp.maximum-frame-size = ${akkaFrameSize}B |akka.remote.netty.tcp.execution-pool-size = $akkaThreads |akka.actor.default-dispatcher.throughput = $akkaBatchSize |akka.log-config-on-start = $logAkkaConfig @@ -121,4 +121,9 @@ private[spark] object AkkaUtils extends Logging { def lookupTimeout(conf: SparkConf): FiniteDuration = { Duration.create(conf.get("spark.akka.lookupTimeout", "30").toLong, "seconds") } + + /** Returns the configured max frame size for Akka messages in bytes. */ + def maxFrameSizeBytes(conf: SparkConf): Int = { + conf.getInt("spark.akka.frameSize", 10) * 1024 * 1024 + } } diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala index cd054c1f684ab..d2e303d81c4c8 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -45,12 +45,12 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "true") badconf.set("spark.authenticate.secret", "bad") - val securityManagerBad = new SecurityManager(badconf); + val securityManagerBad = new SecurityManager(badconf) assert(securityManagerBad.isAuthenticationEnabled() === true) @@ -84,7 +84,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") @@ -136,7 +136,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val goodconf = new SparkConf goodconf.set("spark.authenticate", "true") @@ -189,7 +189,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val badconf = new SparkConf badconf.set("spark.authenticate", "false") diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index 8efa072a97911..a5bd72eb0a122 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark import scala.concurrent.Await import akka.actor._ +import akka.testkit.TestActorRef import org.scalatest.FunSuite import org.apache.spark.scheduler.MapStatus @@ -51,14 +52,16 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master start and stop") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) + tracker.trackerActor = + actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.stop() } test("master register and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) + tracker.trackerActor = + actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -77,7 +80,8 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { test("master register and unregister and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) - tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker))) + tracker.trackerActor = + actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) @@ -100,11 +104,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val hostname = "localhost" val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = new SecurityManager(conf)) - System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext + + // Will be cleared by LocalSparkContext + System.setProperty("spark.driver.port", boundPort.toString) val masterTracker = new MapOutputTrackerMaster(conf) masterTracker.trackerActor = actorSystem.actorOf( - Props(new MapOutputTrackerMasterActor(masterTracker)), "MapOutputTracker") + Props(new MapOutputTrackerMasterActor(masterTracker, conf)), "MapOutputTracker") val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = new SecurityManager(conf)) @@ -126,7 +132,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { masterTracker.incrementEpoch() slaveTracker.updateEpoch(masterTracker.getEpoch) assert(slaveTracker.getServerStatuses(10, 0).toSeq === - Seq((BlockManagerId("a", "hostA", 1000, 0), size1000))) + Seq((BlockManagerId("a", "hostA", 1000, 0), size1000))) masterTracker.unregisterMapOutput(10, 0, BlockManagerId("a", "hostA", 1000, 0)) masterTracker.incrementEpoch() @@ -136,4 +142,44 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { // failure should be cached intercept[FetchFailedException] { slaveTracker.getServerStatuses(10, 0) } } + + test("remote fetch below akka frame size") { + val newConf = new SparkConf + newConf.set("spark.akka.frameSize", "1") + newConf.set("spark.akka.askTimeout", "1") // Fail fast + + val masterTracker = new MapOutputTrackerMaster(conf) + val actorSystem = ActorSystem("test") + val actorRef = TestActorRef[MapOutputTrackerMasterActor]( + new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) + val masterActor = actorRef.underlyingActor + + // Frame size should be ~123B, and no exception should be thrown + masterTracker.registerShuffle(10, 1) + masterTracker.registerMapOutput(10, 0, new MapStatus( + BlockManagerId("88", "mph", 1000, 0), Array.fill[Byte](10)(0))) + masterActor.receive(GetMapOutputStatuses(10)) + } + + test("remote fetch exceeds akka frame size") { + val newConf = new SparkConf + newConf.set("spark.akka.frameSize", "1") + newConf.set("spark.akka.askTimeout", "1") // Fail fast + + val masterTracker = new MapOutputTrackerMaster(conf) + val actorSystem = ActorSystem("test") + val actorRef = TestActorRef[MapOutputTrackerMasterActor]( + new MapOutputTrackerMasterActor(masterTracker, newConf))(actorSystem) + val masterActor = actorRef.underlyingActor + + // Frame size should be ~1.1MB, and MapOutputTrackerMasterActor should throw exception. + // Note that the size is hand-selected here because map output statuses are compressed before + // being sent. + masterTracker.registerShuffle(20, 100) + (0 until 100).foreach { i => + masterTracker.registerMapOutput(20, i, new MapStatus( + BlockManagerId("999", "mps", 1000, 0), Array.fill[Byte](4000000)(0))) + } + intercept[SparkException] { masterActor.receive(GetMapOutputStatuses(20)) } + } } From 087eedca32fd87bfe1629588091bd307d45e4a7c Mon Sep 17 00:00:00 2001 From: Diana Carroll Date: Mon, 17 Mar 2014 17:35:51 -0700 Subject: [PATCH 097/397] [Spark-1261] add instructions for running python examples to doc overview page Author: Diana Carroll Closes #162 from dianacarroll/SPARK-1261 and squashes the following commits: 14ac602 [Diana Carroll] typo in python example text 5121e3e [Diana Carroll] Add explanation of how to run Python examples to main doc overview page --- docs/index.md | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/docs/index.md b/docs/index.md index c4f4d79edbc6c..23311101e1712 100644 --- a/docs/index.md +++ b/docs/index.md @@ -23,10 +23,12 @@ For its Scala API, Spark {{site.SPARK_VERSION}} depends on Scala {{site.SCALA_BI # Running the Examples and Shell -Spark comes with several sample programs in the `examples` directory. -To run one of the samples, use `./bin/run-example ` in the top-level Spark directory +Spark comes with several sample programs. Scala and Java examples are in the `examples` directory, and Python examples are in `python/examples`. +To run one of the Java or Scala sample programs, use `./bin/run-example ` in the top-level Spark directory (the `bin/run-example` script sets up the appropriate paths and launches that program). For example, try `./bin/run-example org.apache.spark.examples.SparkPi local`. +To run a Python sample program, use `./bin/pyspark `. For example, try `./bin/pyspark ./python/examples/pi.py local`. + Each example prints usage help when run with no parameters. Note that all of the sample programs take a `` parameter specifying the cluster URL From e3681f26fae7e87321ac991f5a0fb7517415803a Mon Sep 17 00:00:00 2001 From: Dan McClary Date: Tue, 18 Mar 2014 00:45:47 -0700 Subject: [PATCH 098/397] Spark 1246 add min max to stat counter Here's the addition of min and max to statscounter.py and min and max methods to rdd.py. Author: Dan McClary Closes #144 from dwmclary/SPARK-1246-add-min-max-to-stat-counter and squashes the following commits: fd3fd4b [Dan McClary] fixed error, updated test 82cde0e [Dan McClary] flipped incorrectly assigned inf values in StatCounter 5d96799 [Dan McClary] added max and min to StatCounter repr for pyspark 21dd366 [Dan McClary] added max and min to StatCounter output, updated doc 1a97558 [Dan McClary] added max and min to StatCounter output, updated doc a5c13b0 [Dan McClary] Added min and max to Scala and Java RDD, added min and max to StatCounter ed67136 [Dan McClary] broke min/max out into separate transaction, added to rdd.py 1e7056d [Dan McClary] added underscore to getBucket 37a7dea [Dan McClary] cleaned up boundaries for histogram -- uses real min/max when buckets are derived 29981f2 [Dan McClary] fixed indentation on doctest comment eaf89d9 [Dan McClary] added correct doctest for histogram 4916016 [Dan McClary] added histogram method, added max and min to statscounter --- .../apache/spark/api/java/JavaRDDLike.scala | 20 +++++++++++++++ .../main/scala/org/apache/spark/rdd/RDD.scala | 12 +++++++++ .../org/apache/spark/util/StatCounter.scala | 18 +++++++++++-- .../org/apache/spark/PartitioningSuite.scala | 2 ++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 2 ++ python/pyspark/rdd.py | 19 ++++++++++++++ python/pyspark/statcounter.py | 25 ++++++++++++++++--- 7 files changed, 93 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 2ba4fb8c18b1d..05b89b985736d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -478,6 +478,26 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } + /** + * Returns the maximum element from this RDD as defined by the specified + * Comparator[T]. + * @params comp the comparator that defines ordering + * @return the maximum of the RDD + * */ + def max(comp: Comparator[T]): T = { + rdd.max()(Ordering.comparatorToOrdering(comp)) + } + + /** + * Returns the minimum element from this RDD as defined by the specified + * Comparator[T]. + * @params comp the comparator that defines ordering + * @return the minimum of the RDD + * */ + def min(comp: Comparator[T]): T = { + rdd.min()(Ordering.comparatorToOrdering(comp)) + } + /** * Returns the first K elements from this RDD using the * natural ordering for T while maintain the order. diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index f8283fbbb980d..ddb901246d360 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -956,6 +956,18 @@ abstract class RDD[T: ClassTag]( */ def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse) + /** + * Returns the max of this RDD as defined by the implicit Ordering[T]. + * @return the maximum element of the RDD + * */ + def max()(implicit ord: Ordering[T]):T = this.reduce(ord.max) + + /** + * Returns the min of this RDD as defined by the implicit Ordering[T]. + * @return the minimum element of the RDD + * */ + def min()(implicit ord: Ordering[T]):T = this.reduce(ord.min) + /** * Save this RDD as a text file, using string representations of elements. */ diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala index f837dc7ccc860..732748a7ff82b 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -29,6 +29,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { private var n: Long = 0 // Running count of our values private var mu: Double = 0 // Running mean of our values private var m2: Double = 0 // Running variance numerator (sum of (x - mean)^2) + private var maxValue: Double = Double.NegativeInfinity // Running max of our values + private var minValue: Double = Double.PositiveInfinity // Running min of our values merge(values) @@ -41,6 +43,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { n += 1 mu += delta / n m2 += delta * (value - mu) + maxValue = math.max(maxValue, value) + minValue = math.min(minValue, value) this } @@ -58,7 +62,9 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { if (n == 0) { mu = other.mu m2 = other.m2 - n = other.n + n = other.n + maxValue = other.maxValue + minValue = other.minValue } else if (other.n != 0) { val delta = other.mu - mu if (other.n * 10 < n) { @@ -70,6 +76,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { } m2 += other.m2 + (delta * delta * n * other.n) / (n + other.n) n += other.n + maxValue = math.max(maxValue, other.maxValue) + minValue = math.min(minValue, other.minValue) } this } @@ -81,6 +89,8 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { other.n = n other.mu = mu other.m2 = m2 + other.maxValue = maxValue + other.minValue = minValue other } @@ -90,6 +100,10 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { def sum: Double = n * mu + def max: Double = maxValue + + def min: Double = minValue + /** Return the variance of the values. */ def variance: Double = { if (n == 0) { @@ -121,7 +135,7 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { def sampleStdev: Double = math.sqrt(sampleVariance) override def toString: String = { - "(count: %d, mean: %f, stdev: %f)".format(count, mean, stdev) + "(count: %d, mean: %f, stdev: %f, max: %f, min: %f)".format(count, mean, stdev, max, min) } } diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 4305686d3a6d5..996db70809320 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -171,6 +171,8 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(abs(6.0/2 - rdd.mean) < 0.01) assert(abs(1.0 - rdd.variance) < 0.01) assert(abs(1.0 - rdd.stdev) < 0.01) + assert(stats.max === 4.0) + assert(stats.min === 2.0) // Add other tests here for classes that should be able to handle empty partitions correctly } diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 9512e0e6eeb14..d6b5fdc7984b4 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -47,6 +47,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(nums.glom().map(_.toList).collect().toList === List(List(1, 2), List(3, 4))) assert(nums.collect({ case i if i >= 3 => i.toString }).collect().toList === List("3", "4")) assert(nums.keyBy(_.toString).collect().toList === List(("1", 1), ("2", 2), ("3", 3), ("4", 4))) + assert(nums.max() === 4) + assert(nums.min() === 1) val partitionSums = nums.mapPartitions(iter => Iterator(iter.reduceLeft(_ + _))) assert(partitionSums.collect().toList === List(3, 7)) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index f3b432ff248a9..ae09dbff02a36 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -571,7 +571,26 @@ def func(iterator): return reduce(op, vals, zeroValue) # TODO: aggregate + + + def max(self): + """ + Find the maximum item in this RDD. + + >>> sc.parallelize([1.0, 5.0, 43.0, 10.0]).max() + 43.0 + """ + return self.reduce(max) + def min(self): + """ + Find the maximum item in this RDD. + + >>> sc.parallelize([1.0, 5.0, 43.0, 10.0]).min() + 1.0 + """ + return self.reduce(min) + def sum(self): """ Add up the elements in this RDD. diff --git a/python/pyspark/statcounter.py b/python/pyspark/statcounter.py index 8e1cbd4ad9856..080325061a697 100644 --- a/python/pyspark/statcounter.py +++ b/python/pyspark/statcounter.py @@ -26,7 +26,9 @@ def __init__(self, values=[]): self.n = 0L # Running count of our values self.mu = 0.0 # Running mean of our values self.m2 = 0.0 # Running variance numerator (sum of (x - mean)^2) - + self.maxValue = float("-inf") + self.minValue = float("inf") + for v in values: self.merge(v) @@ -36,6 +38,11 @@ def merge(self, value): self.n += 1 self.mu += delta / self.n self.m2 += delta * (value - self.mu) + if self.maxValue < value: + self.maxValue = value + if self.minValue > value: + self.minValue = value + return self # Merge another StatCounter into this one, adding up the internal statistics. @@ -49,7 +56,10 @@ def mergeStats(self, other): if self.n == 0: self.mu = other.mu self.m2 = other.m2 - self.n = other.n + self.n = other.n + self.maxValue = other.maxValue + self.minValue = other.minValue + elif other.n != 0: delta = other.mu - self.mu if other.n * 10 < self.n: @@ -58,6 +68,9 @@ def mergeStats(self, other): self.mu = other.mu - (delta * self.n) / (self.n + other.n) else: self.mu = (self.mu * self.n + other.mu * other.n) / (self.n + other.n) + + self.maxValue = max(self.maxValue, other.maxValue) + self.minValue = min(self.minValue, other.minValue) self.m2 += other.m2 + (delta * delta * self.n * other.n) / (self.n + other.n) self.n += other.n @@ -76,6 +89,12 @@ def mean(self): def sum(self): return self.n * self.mu + def min(self): + return self.minValue + + def max(self): + return self.maxValue + # Return the variance of the values. def variance(self): if self.n == 0: @@ -105,5 +124,5 @@ def sampleStdev(self): return math.sqrt(self.sampleVariance()) def __repr__(self): - return "(count: %s, mean: %s, stdev: %s)" % (self.count(), self.mean(), self.stdev()) + return "(count: %s, mean: %s, stdev: %s, max: %s, min: %s)" % (self.count(), self.mean(), self.stdev(), self.max(), self.min()) From e7423d4040ebd1ec4105d8d4b9a4a6600b18c2ac Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 18 Mar 2014 00:46:03 -0700 Subject: [PATCH 099/397] Revert "SPARK-1236 - Upgrade Jetty to 9.1.3.v20140225." This reverts commit ca4bf8c572c2f70b484830f1db414b5073744ab6. Jetty 9 requires JDK7 which is probably not a dependency we want to bump right now. Before Spark 1.0 we should consider upgrading to Jetty 8. However, in the mean time to ease some pain let's revert this. Sorry for not catching this during the initial review. cc/ @rxin Author: Patrick Wendell Closes #167 from pwendell/jetty-revert and squashes the following commits: 811b1c5 [Patrick Wendell] Revert "SPARK-1236 - Upgrade Jetty to 9.1.3.v20140225." --- .../scala/org/apache/spark/HttpServer.scala | 21 +++---- .../org/apache/spark/ui/JettyUtils.scala | 58 +++++++++---------- pom.xml | 8 +-- project/SparkBuild.scala | 8 +-- 4 files changed, 46 insertions(+), 49 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index d14693cb78a2d..cb5df25fa48df 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -21,9 +21,10 @@ import java.io.File import org.eclipse.jetty.util.security.{Constraint, Password} import org.eclipse.jetty.security.authentication.DigestAuthenticator -import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService} +import org.eclipse.jetty.security.{ConstraintMapping, ConstraintSecurityHandler, HashLoginService, SecurityHandler} -import org.eclipse.jetty.server.{Server, ServerConnector} +import org.eclipse.jetty.server.Server +import org.eclipse.jetty.server.bio.SocketConnector import org.eclipse.jetty.server.handler.{DefaultHandler, HandlerList, ResourceHandler} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -42,7 +43,7 @@ private[spark] class ServerStateException(message: String) extends Exception(mes */ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityManager) extends Logging { - private var server: Server = _ + private var server: Server = null private var port: Int = -1 def start() { @@ -50,16 +51,16 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan throw new ServerStateException("Server is already started") } else { logInfo("Starting HTTP Server") - val threadPool = new QueuedThreadPool - threadPool.setDaemon(true) - - server = new Server(threadPool) - val connector = new ServerConnector(server) - connector.setIdleTimeout(60 * 1000) + server = new Server() + val connector = new SocketConnector + connector.setMaxIdleTime(60*1000) connector.setSoLingerTime(-1) connector.setPort(0) server.addConnector(connector) + val threadPool = new QueuedThreadPool + threadPool.setDaemon(true) + server.setThreadPool(threadPool) val resHandler = new ResourceHandler resHandler.setResourceBase(resourceBase.getAbsolutePath) @@ -78,7 +79,7 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan } server.start() - port = connector.getLocalPort + port = server.getConnectors()(0).getLocalPort() } } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index cc7324939668b..7c35cd165ad7c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -19,7 +19,6 @@ package org.apache.spark.ui import java.net.InetSocketAddress import java.net.URL -import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletResponse, HttpServletRequest} import scala.annotation.tailrec @@ -29,7 +28,7 @@ import scala.xml.Node import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} -import org.eclipse.jetty.server.{NetworkConnector, Server} +import org.eclipse.jetty.server.{DispatcherType, Server} import org.eclipse.jetty.server.handler.HandlerList import org.eclipse.jetty.servlet.{DefaultServlet, FilterHolder, ServletContextHandler, ServletHolder} import org.eclipse.jetty.util.thread.QueuedThreadPool @@ -61,7 +60,8 @@ private[spark] object JettyUtils extends Logging { def createServlet[T <% AnyRef](servletParams: ServletParams[T], securityMgr: SecurityManager): HttpServlet = { new HttpServlet { - override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + override def doGet(request: HttpServletRequest, + response: HttpServletResponse) { if (securityMgr.checkUIViewPermissions(request.getRemoteUser())) { response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) response.setStatus(HttpServletResponse.SC_OK) @@ -72,7 +72,7 @@ private[spark] object JettyUtils extends Logging { response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.sendError(HttpServletResponse.SC_UNAUTHORIZED, - "User is not authorized to access this page.") + "User is not authorized to access this page."); } } } @@ -120,25 +120,26 @@ private[spark] object JettyUtils extends Logging { private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) - filters.foreach { filter => - if (!filter.isEmpty) { - logInfo("Adding filter: " + filter) - val holder : FilterHolder = new FilterHolder() - holder.setClassName(filter) - // get any parameters for each filter - val paramName = "spark." + filter + ".params" - val params = conf.get(paramName, "").split(',').map(_.trim()).toSet - params.foreach { - case param : String => - if (!param.isEmpty) { - val parts = param.split("=") - if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) - } + filters.foreach { + case filter : String => + if (!filter.isEmpty) { + logInfo("Adding filter: " + filter) + val holder : FilterHolder = new FilterHolder() + holder.setClassName(filter) + // get any parameters for each filter + val paramName = "spark." + filter + ".params" + val params = conf.get(paramName, "").split(',').map(_.trim()).toSet + params.foreach { + case param : String => + if (!param.isEmpty) { + val parts = param.split("=") + if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) + } + } + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, + DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) + handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } } - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, - DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) - handlers.foreach { handler => handler.addFilter(holder, "/*", enumDispatcher) } - } } } @@ -149,10 +150,7 @@ private[spark] object JettyUtils extends Logging { * If the desired port number is contented, continues incrementing ports until a free port is * found. Returns the chosen port and the jetty Server object. */ - def startJettyServer( - hostName: String, - port: Int, - handlers: Seq[ServletContextHandler], + def startJettyServer(hostName: String, port: Int, handlers: Seq[ServletContextHandler], conf: SparkConf): (Server, Int) = { addFilters(handlers, conf) @@ -162,18 +160,16 @@ private[spark] object JettyUtils extends Logging { @tailrec def connect(currentPort: Int): (Server, Int) = { val server = new Server(new InetSocketAddress(hostName, currentPort)) - // Unfortunately Jetty 9 doesn't allow us to set both the thread pool and the port number in - // constructor. But fortunately the pool allocated by Jetty is always a QueuedThreadPool. - val pool = server.getThreadPool.asInstanceOf[QueuedThreadPool] + val pool = new QueuedThreadPool pool.setDaemon(true) - + server.setThreadPool(pool) server.setHandler(handlerList) Try { server.start() } match { case s: Success[_] => - (server, server.getConnectors.head.asInstanceOf[NetworkConnector].getLocalPort) + (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) diff --git a/pom.xml b/pom.xml index 377b6463136dd..524e5daff5388 100644 --- a/pom.xml +++ b/pom.xml @@ -186,22 +186,22 @@ org.eclipse.jetty jetty-util - 9.1.3.v20140225 + 7.6.8.v20121106 org.eclipse.jetty jetty-security - 9.1.3.v20140225 + 7.6.8.v20121106 org.eclipse.jetty jetty-plus - 9.1.3.v20140225 + 7.6.8.v20121106 org.eclipse.jetty jetty-server - 9.1.3.v20140225 + 7.6.8.v20121106 com.google.guava diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index db24f124f3cf8..aff191c98b409 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -238,10 +238,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % "9.1.3.v20140225", - "org.eclipse.jetty" % "jetty-util" % "9.1.3.v20140225", - "org.eclipse.jetty" % "jetty-plus" % "9.1.3.v20140225", - "org.eclipse.jetty" % "jetty-security" % "9.1.3.v20140225", + "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-util" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-plus" % "7.6.8.v20121106", + "org.eclipse.jetty" % "jetty-security" % "7.6.8.v20121106", /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"), "org.scalatest" %% "scalatest" % "1.9.1" % "test", From 2fa26ec02fc2251102f89bb67523419fd7dd3757 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 18 Mar 2014 11:06:18 -0700 Subject: [PATCH 100/397] SPARK-1102: Create a saveAsNewAPIHadoopDataset method https://spark-project.atlassian.net/browse/SPARK-1102 Create a saveAsNewAPIHadoopDataset method By @mateiz: "Right now RDDs can only be saved as files using the new Hadoop API, not as "datasets" with no filename and just a JobConf. See http://codeforhire.com/2014/02/18/using-spark-with-mongodb/ for an example of how you have to give a bogus filename. For the old Hadoop API, we have saveAsHadoopDataset." Author: CodingCat Closes #12 from CodingCat/SPARK-1102 and squashes the following commits: 6ba0c83 [CodingCat] add test cases for saveAsHadoopDataSet (new&old API) a8d11ba [CodingCat] style fix......... 95a6929 [CodingCat] code clean 7643c88 [CodingCat] change the parameter type back to Configuration a8583ee [CodingCat] Create a saveAsNewAPIHadoopDataset method --- .../apache/spark/api/java/JavaPairRDD.scala | 10 +- .../apache/spark/rdd/PairRDDFunctions.scala | 104 ++++++++++-------- .../scala/org/apache/spark/FileSuite.scala | 39 ++++++- 3 files changed, 100 insertions(+), 53 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 0ff428c120353..9596dbaf75488 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -26,7 +26,7 @@ import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ @@ -558,6 +558,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) rdd.saveAsNewAPIHadoopFile(path, keyClass, valueClass, outputFormatClass, conf) } + /** + * Output the RDD to any Hadoop-supported storage system, using + * a Configuration object for that storage system. + */ + def saveAsNewAPIHadoopDataset(conf: Configuration) { + rdd.saveAsNewAPIHadoopDataset(conf) + } + /** Output the RDD to any Hadoop-supported file system. */ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[_, _]]( path: String, diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index b0d322fe27bd5..447deafff53cd 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -30,11 +30,11 @@ import scala.reflect.ClassTag import com.clearspring.analytics.stream.cardinality.HyperLogLog import org.apache.hadoop.conf.{Configurable, Configuration} -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job => NewAPIHadoopJob, RecordWriter => NewRecordWriter, JobContext, SparkHadoopMapReduceUtil} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job => NewAPIHadoopJob, RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} // SparkHadoopWriter and SparkHadoopMapReduceUtil are actually source files defined in Spark. @@ -603,50 +603,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) val job = new NewAPIHadoopJob(conf) job.setOutputKeyClass(keyClass) job.setOutputValueClass(valueClass) - - val wrappedConf = new SerializableWritable(job.getConfiguration) - val outpath = new Path(path) - NewFileOutputFormat.setOutputPath(job, outpath) - val jobFormat = outputFormatClass.newInstance - jobFormat.checkOutputSpecs(job) - val formatter = new SimpleDateFormat("yyyyMMddHHmm") - val jobtrackerID = formatter.format(new Date()) - val stageId = self.id - def writeShard(context: TaskContext, iter: Iterator[(K,V)]): 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 = outputFormatClass.newInstance - format match { - case c: Configurable => c.setConf(wrappedConf.value) - case _ => () - } - val committer = format.getOutputCommitter(hadoopContext) - committer.setupTask(hadoopContext) - val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] - while (iter.hasNext) { - val (k, v) = iter.next() - writer.write(k, v) - } - writer.close(hadoopContext) - committer.commitTask(hadoopContext) - return 1 - } - - /* 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) - self.context.runJob(self, writeShard _) - jobCommitter.commitJob(jobTaskContext) + job.setOutputFormatClass(outputFormatClass) + job.getConfiguration.set("mapred.output.dir", path) + saveAsNewAPIHadoopDataset(job.getConfiguration) } /** @@ -692,6 +651,59 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) saveAsHadoopDataset(conf) } + /** + * Output the RDD to any Hadoop-supported storage system with new Hadoop API, using a Hadoop + * Configuration object for that storage system. The Conf should set an OutputFormat and any + * output paths required (e.g. a table name to write to) in the same way as it would be + * configured for a Hadoop MapReduce job. + */ + def saveAsNewAPIHadoopDataset(conf: Configuration) { + val job = new NewAPIHadoopJob(conf) + val formatter = new SimpleDateFormat("yyyyMMddHHmm") + val jobtrackerID = formatter.format(new Date()) + val stageId = self.id + val wrappedConf = new SerializableWritable(job.getConfiguration) + val outfmt = job.getOutputFormatClass + val jobFormat = outfmt.newInstance + + if (jobFormat.isInstanceOf[NewFileOutputFormat[_, _]]) { + // FileOutputFormat ignores the filesystem parameter + jobFormat.checkOutputSpecs(job) + } + + def writeShard(context: TaskContext, iter: Iterator[(K,V)]): 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 = outfmt.newInstance + format match { + case c: Configurable => c.setConf(wrappedConf.value) + case _ => () + } + val committer = format.getOutputCommitter(hadoopContext) + committer.setupTask(hadoopContext) + val writer = format.getRecordWriter(hadoopContext).asInstanceOf[NewRecordWriter[K,V]] + while (iter.hasNext) { + val (k, v) = iter.next() + writer.write(k, v) + } + writer.close(hadoopContext) + committer.commitTask(hadoopContext) + return 1 + } + + val jobAttemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = true, 0, 0) + val jobTaskContext = newTaskAttemptContext(wrappedConf.value, jobAttemptId) + val jobCommitter = jobFormat.getOutputCommitter(jobTaskContext) + jobCommitter.setupJob(jobTaskContext) + self.context.runJob(self, writeShard _) + jobCommitter.commitJob(jobTaskContext) + } + /** * Output the RDD to any Hadoop-supported storage system, using a Hadoop JobConf object for * that storage system. The JobConf should set an OutputFormat and any output paths required diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 76173608e9f70..01af94077144a 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -24,11 +24,12 @@ import scala.io.Source import com.google.common.io.Files import org.apache.hadoop.io._ import org.apache.hadoop.io.compress.DefaultCodec -import org.apache.hadoop.mapred.FileAlreadyExistsException +import org.apache.hadoop.mapred.{JobConf, FileAlreadyExistsException, TextOutputFormat} +import org.apache.hadoop.mapreduce.lib.output.{TextOutputFormat => NewTextOutputFormat} +import org.apache.hadoop.mapreduce.Job import org.scalatest.FunSuite import org.apache.spark.SparkContext._ -import org.apache.hadoop.mapreduce.lib.output.TextOutputFormat class FileSuite extends FunSuite with LocalSparkContext { @@ -236,7 +237,7 @@ class FileSuite extends FunSuite with LocalSparkContext { val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[TextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) } } @@ -244,10 +245,36 @@ class FileSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val tempdir = Files.createTempDir() val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) - randomRDD.saveAsTextFile(tempdir.getPath + "/output") - assert(new File(tempdir.getPath + "/output/part-00000").exists() === true) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath + "/output") + assert(new File(tempdir.getPath + "/output/part-r-00000").exists() === true) intercept[FileAlreadyExistsException] { - randomRDD.saveAsNewAPIHadoopFile[TextOutputFormat[String, String]](tempdir.getPath) + randomRDD.saveAsNewAPIHadoopFile[NewTextOutputFormat[String, String]](tempdir.getPath) } } + + test ("save Hadoop Dataset through old Hadoop API") { + sc = new SparkContext("local", "test") + val tempdir = Files.createTempDir() + val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val job = new JobConf() + job.setOutputKeyClass(classOf[String]) + job.setOutputValueClass(classOf[String]) + job.set("mapred.output.format.class", classOf[TextOutputFormat[String, String]].getName) + job.set("mapred.output.dir", tempdir.getPath + "/outputDataset_old") + randomRDD.saveAsHadoopDataset(job) + assert(new File(tempdir.getPath + "/outputDataset_old/part-00000").exists() === true) + } + + test ("save Hadoop Dataset through new Hadoop API") { + sc = new SparkContext("local", "test") + val tempdir = Files.createTempDir() + val randomRDD = sc.parallelize(Array(("key1", "a"), ("key2", "a"), ("key3", "b"), ("key4", "c")), 1) + val job = new Job(sc.hadoopConfiguration) + job.setOutputKeyClass(classOf[String]) + job.setOutputValueClass(classOf[String]) + job.setOutputFormatClass(classOf[NewTextOutputFormat[String, String]]) + job.getConfiguration.set("mapred.output.dir", tempdir.getPath + "/outputDataset_new") + randomRDD.saveAsNewAPIHadoopDataset(job.getConfiguration) + assert(new File(tempdir.getPath + "/outputDataset_new/part-r-00000").exists() === true) + } } From 79e547fe5a675a9a10b6acdc73759d67725ad7c6 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 18 Mar 2014 14:34:31 -0700 Subject: [PATCH 101/397] Update copyright year in NOTICE to 2014 Author: Matei Zaharia Closes #174 from mateiz/update-notice and squashes the following commits: 47fc1a5 [Matei Zaharia] Update copyright year in NOTICE to 2014 --- NOTICE | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/NOTICE b/NOTICE index 7cbb114b2ae2d..dce0c4eaf31ed 100644 --- a/NOTICE +++ b/NOTICE @@ -1,5 +1,5 @@ Apache Spark -Copyright 2013 The Apache Software Foundation. +Copyright 2014 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). From e108b9ab94c4310ec56ef0eda99bb904133f942d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 18 Mar 2014 15:14:13 -0700 Subject: [PATCH 102/397] [SPARK-1260]: faster construction of features with intercept The current implementation uses `Array(1.0, features: _*)` to construct a new array with intercept. This is not efficient for big arrays because `Array.apply` uses a for loop that iterates over the arguments. `Array.+:` is a better choice here. Also, I don't see a reason to set initial weights to ones. So I set them to zeros. JIRA: https://spark-project.atlassian.net/browse/SPARK-1260 Author: Xiangrui Meng Closes #161 from mengxr/sgd and squashes the following commits: b5cfc53 [Xiangrui Meng] set default weights to zeros a1439c2 [Xiangrui Meng] faster construction of features with intercept --- .../mllib/regression/GeneralizedLinearAlgorithm.scala | 8 ++++---- 1 file changed, 4 insertions(+), 4 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index f98b0b536deaa..b9621530efa22 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -119,7 +119,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] */ def run(input: RDD[LabeledPoint]) : M = { val nfeatures: Int = input.first().features.length - val initialWeights = Array.fill(nfeatures)(1.0) + val initialWeights = new Array[Double](nfeatures) run(input, initialWeights) } @@ -134,15 +134,15 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] throw new SparkException("Input validation failed.") } - // Add a extra variable consisting of all 1.0's for the intercept. + // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, Array(1.0, labeledPoint.features:_*))) + input.map(labeledPoint => (labeledPoint.label, labeledPoint.features.+:(1.0))) } else { input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) } val initialWeightsWithIntercept = if (addIntercept) { - Array(1.0, initialWeights:_*) + initialWeights.+:(1.0) } else { initialWeights } From f9d8a83c0006bb59c61e8770cd201b72333cb9a4 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 18 Mar 2014 17:20:42 -0700 Subject: [PATCH 103/397] [SPARK-1266] persist factors in implicit ALS In implicit ALS computation, the user or product factor is used twice in each iteration. Caching can certainly help accelerate the computation. I saw the running time decreased by ~70% for implicit ALS on the movielens data. I also made the following changes: 1. Change `YtYb` type from `Broadcast[Option[DoubleMatrix]]` to `Option[Broadcast[DoubleMatrix]]`, so we don't need to broadcast None in explicit computation. 2. Mark methods `computeYtY`, `unblockFactors`, `updateBlock`, and `updateFeatures private`. Users do not need those methods. 3. Materialize the final matrix factors before returning the model. It allows us to clean up other cached RDDs before returning the model. I do not have a better solution here, so I use `RDD.count()`. JIRA: https://spark-project.atlassian.net/browse/SPARK-1266 Author: Xiangrui Meng Closes #165 from mengxr/als and squashes the following commits: c9676a6 [Xiangrui Meng] add a comment about the last products.persist d3a88aa [Xiangrui Meng] change implicitPrefs match to if ... else ... 63862d6 [Xiangrui Meng] persist factors in implicit ALS --- .../spark/mllib/recommendation/ALS.scala | 145 +++++++++++------- 1 file changed, 89 insertions(+), 56 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 777d0db2d6653..0cc9f48769f83 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -148,8 +148,10 @@ class ALS private ( * Returns a MatrixFactorizationModel with feature vectors for each user and product. */ def run(ratings: RDD[Rating]): MatrixFactorizationModel = { + val sc = ratings.context + val numBlocks = if (this.numBlocks == -1) { - math.max(ratings.context.defaultParallelism, ratings.partitions.size / 2) + math.max(sc.defaultParallelism, ratings.partitions.size / 2) } else { this.numBlocks } @@ -187,21 +189,41 @@ class ALS private ( } } - for (iter <- 1 to iterations) { - // perform ALS update - logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) - // YtY / XtX is an Option[DoubleMatrix] and is only required for the implicit feedback model - val YtY = computeYtY(users) - val YtYb = ratings.context.broadcast(YtY) - products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda, - alpha, YtYb) - logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) - val XtX = computeYtY(products) - val XtXb = ratings.context.broadcast(XtX) - users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda, - alpha, XtXb) + if (implicitPrefs) { + for (iter <- 1 to iterations) { + // perform ALS update + logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) + // Persist users because it will be called twice. + users.persist() + val YtY = Some(sc.broadcast(computeYtY(users))) + val previousProducts = products + products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda, + alpha, YtY) + previousProducts.unpersist() + logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) + products.persist() + val XtX = Some(sc.broadcast(computeYtY(products))) + val previousUsers = users + users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda, + alpha, XtX) + previousUsers.unpersist() + } + } else { + for (iter <- 1 to iterations) { + // perform ALS update + logInfo("Re-computing I given U (Iteration %d/%d)".format(iter, iterations)) + products = updateFeatures(users, userOutLinks, productInLinks, partitioner, rank, lambda, + alpha, YtY = None) + logInfo("Re-computing U given I (Iteration %d/%d)".format(iter, iterations)) + users = updateFeatures(products, productOutLinks, userInLinks, partitioner, rank, lambda, + alpha, YtY = None) + } } + // The last `products` will be used twice. One to generate the last `users` and the other to + // generate `productsOut`. So we cache it for better performance. + products.persist() + // Flatten and cache the two final RDDs to un-block them val usersOut = unblockFactors(users, userOutLinks) val productsOut = unblockFactors(products, productOutLinks) @@ -209,31 +231,39 @@ class ALS private ( usersOut.persist() productsOut.persist() + // Materialize usersOut and productsOut. + usersOut.count() + productsOut.count() + + products.unpersist() + + // Clean up. + userInLinks.unpersist() + userOutLinks.unpersist() + productInLinks.unpersist() + productOutLinks.unpersist() + new MatrixFactorizationModel(rank, usersOut, productsOut) } /** * Computes the (`rank x rank`) matrix `YtY`, where `Y` is the (`nui x rank`) matrix of factors - * for each user (or product), in a distributed fashion. Here `reduceByKeyLocally` is used as - * the driver program requires `YtY` to broadcast it to the slaves + * for each user (or product), in a distributed fashion. + * * @param factors the (block-distributed) user or product factor vectors - * @return Option[YtY] - whose value is only used in the implicit preference model + * @return YtY - whose value is only used in the implicit preference model */ - def computeYtY(factors: RDD[(Int, Array[Array[Double]])]) = { - if (implicitPrefs) { - val n = rank * (rank + 1) / 2 - val LYtY = factors.values.aggregate(new DoubleMatrix(n))( seqOp = (L, Y) => { - Y.foreach(y => dspr(1.0, new DoubleMatrix(y), L)) - L - }, combOp = (L1, L2) => { - L1.addi(L2) - }) - val YtY = new DoubleMatrix(rank, rank) - fillFullMatrix(LYtY, YtY) - Option(YtY) - } else { - None - } + private def computeYtY(factors: RDD[(Int, Array[Array[Double]])]) = { + val n = rank * (rank + 1) / 2 + val LYtY = factors.values.aggregate(new DoubleMatrix(n))( seqOp = (L, Y) => { + Y.foreach(y => dspr(1.0, new DoubleMatrix(y), L)) + L + }, combOp = (L1, L2) => { + L1.addi(L2) + }) + val YtY = new DoubleMatrix(rank, rank) + fillFullMatrix(LYtY, YtY) + YtY } /** @@ -264,7 +294,7 @@ class ALS private ( /** * Flatten out blocked user or product factors into an RDD of (id, factor vector) pairs */ - def unblockFactors(blockedFactors: RDD[(Int, Array[Array[Double]])], + private def unblockFactors(blockedFactors: RDD[(Int, Array[Array[Double]])], outLinks: RDD[(Int, OutLinkBlock)]) = { blockedFactors.join(outLinks).flatMap{ case (b, (factors, outLinkBlock)) => for (i <- 0 until factors.length) yield (outLinkBlock.elementIds(i), factors(i)) @@ -332,8 +362,11 @@ class ALS private ( val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) }, true) - links.persist(StorageLevel.MEMORY_AND_DISK) - (links.mapValues(_._1), links.mapValues(_._2)) + val inLinks = links.mapValues(_._1) + val outLinks = links.mapValues(_._2) + inLinks.persist(StorageLevel.MEMORY_AND_DISK) + outLinks.persist(StorageLevel.MEMORY_AND_DISK) + (inLinks, outLinks) } /** @@ -365,7 +398,7 @@ class ALS private ( rank: Int, lambda: Double, alpha: Double, - YtY: Broadcast[Option[DoubleMatrix]]) + YtY: Option[Broadcast[DoubleMatrix]]) : RDD[(Int, Array[Array[Double]])] = { val numBlocks = products.partitions.size @@ -388,8 +421,8 @@ class ALS private ( * Compute the new feature vectors for a block of the users matrix given the list of factors * it received from each product and its InLinkBlock. */ - def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, - rank: Int, lambda: Double, alpha: Double, YtY: Broadcast[Option[DoubleMatrix]]) + private def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, + rank: Int, lambda: Double, alpha: Double, YtY: Option[Broadcast[DoubleMatrix]]) : Array[Array[Double]] = { // Sort the incoming block factor messages by block ID and make them an array @@ -416,21 +449,20 @@ class ALS private ( dspr(1.0, x, tempXtX) val (us, rs) = inLinkBlock.ratingsForBlock(productBlock)(p) for (i <- 0 until us.length) { - implicitPrefs match { - case false => - userXtX(us(i)).addi(tempXtX) - SimpleBlas.axpy(rs(i), x, userXy(us(i))) - case true => - // Extension to the original paper to handle rs(i) < 0. confidence is a function - // of |rs(i)| instead so that it is never negative: - val confidence = 1 + alpha * abs(rs(i)) - SimpleBlas.axpy(confidence - 1.0, tempXtX, userXtX(us(i))) - // For rs(i) < 0, the corresponding entry in P is 0 now, not 1 -- negative rs(i) - // means we try to reconstruct 0. We add terms only where P = 1, so, term below - // is now only added for rs(i) > 0: - if (rs(i) > 0) { - SimpleBlas.axpy(confidence, x, userXy(us(i))) - } + if (implicitPrefs) { + // Extension to the original paper to handle rs(i) < 0. confidence is a function + // of |rs(i)| instead so that it is never negative: + val confidence = 1 + alpha * abs(rs(i)) + SimpleBlas.axpy(confidence - 1.0, tempXtX, userXtX(us(i))) + // For rs(i) < 0, the corresponding entry in P is 0 now, not 1 -- negative rs(i) + // means we try to reconstruct 0. We add terms only where P = 1, so, term below + // is now only added for rs(i) > 0: + if (rs(i) > 0) { + SimpleBlas.axpy(confidence, x, userXy(us(i))) + } + } else { + userXtX(us(i)).addi(tempXtX) + SimpleBlas.axpy(rs(i), x, userXy(us(i))) } } } @@ -443,9 +475,10 @@ class ALS private ( // Add regularization (0 until rank).foreach(i => fullXtX.data(i*rank + i) += lambda) // Solve the resulting matrix, which is symmetric and positive-definite - implicitPrefs match { - case false => Solve.solvePositive(fullXtX, userXy(index)).data - case true => Solve.solvePositive(fullXtX.addi(YtY.value.get), userXy(index)).data + if (implicitPrefs) { + Solve.solvePositive(fullXtX.addi(YtY.get.value), userXy(index)).data + } else { + Solve.solvePositive(fullXtX, userXy(index)).data } } } From cc2655a237442a71c75d4fade99767df7648e55f Mon Sep 17 00:00:00 2001 From: witgo Date: Tue, 18 Mar 2014 21:57:47 -0700 Subject: [PATCH 104/397] Fix SPARK-1256: Master web UI and Worker web UI returns a 404 error Author: witgo Closes #150 from witgo/SPARK-1256 and squashes the following commits: 08044a2 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1256 c99b030 [witgo] Fix SPARK-1256 --- .../org/apache/spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 5 ++++- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 2 +- 4 files changed, 7 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index a7bd01e284c8e..4ad1f95be31c9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -61,7 +61,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { master.applicationMetricsSystem.getServletHandlers val handlers = metricsHandlers ++ Seq[ServletContextHandler]( - createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static/*"), + createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR + "/static", "/static"), createServletHandler("/app/json", createServlet((request: HttpServletRequest) => applicationPage.renderJson(request), master.securityMgr)), diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index ffc05bd30687a..4e33b330ad4e7 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -47,7 +47,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val metricsHandlers = worker.metricsSystem.getServletHandlers val handlers = metricsHandlers ++ Seq[ServletContextHandler]( - createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static/*"), + createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE + "/static", "/static"), createServletHandler("/log", createServlet((request: HttpServletRequest) => log(request), worker.securityMgr)), createServletHandler("/logPage", createServlet((request: HttpServletRequest) => logPage diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 7c35cd165ad7c..e0555ca7ac02f 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -111,10 +111,13 @@ private[spark] object JettyUtils extends Logging { Option(getClass.getClassLoader.getResource(resourceBase)) match { case Some(res) => holder.setInitParameter("resourceBase", res.toString) + holder.setInitParameter("welcomeServlets", "false") + holder.setInitParameter("pathInfoOnly", "false") case None => throw new Exception("Could not find resource path for Web UI: " + resourceBase) } - contextHandler.addServlet(holder, path) + contextHandler.setContextPath(path) + contextHandler.addServlet(holder, "/") contextHandler } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index ca82c3da2fc24..5f0dee64fedb7 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -38,7 +38,7 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { var server: Option[Server] = None val handlers = Seq[ServletContextHandler] ( - createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static/*"), + createStaticHandler(SparkUI.STATIC_RESOURCE_DIR + "/static", "/static"), createRedirectHandler("/stages", "/") ) val storage = new BlockManagerUI(sc) From a18ea00f3af0fa4c6b2c59933e22b6c9f0f636c8 Mon Sep 17 00:00:00 2001 From: Nick Lanham Date: Tue, 18 Mar 2014 22:04:57 -0700 Subject: [PATCH 105/397] Bundle tachyon: SPARK-1269 This should all work as expected with the current version of the tachyon tarball (0.4.1) Author: Nick Lanham Closes #137 from nicklan/bundle-tachyon and squashes the following commits: 2eee15b [Nick Lanham] Put back in exec, start tachyon first 738ba23 [Nick Lanham] Move tachyon out of sbin f2f9bc6 [Nick Lanham] More checks for tachyon script 111e8e1 [Nick Lanham] Only try tachyon operations if tachyon script exists 0561574 [Nick Lanham] Copy over web resources so web interface can run 4dc9809 [Nick Lanham] Update to tachyon 0.4.1 0a1a20c [Nick Lanham] Add scripts using tachyon tarball --- make-distribution.sh | 32 ++++++++++++++++++++++++++++++++ sbin/start-all.sh | 15 +++++++++++++-- sbin/start-master.sh | 21 +++++++++++++++++++++ sbin/start-slaves.sh | 23 +++++++++++++++++++++++ sbin/stop-master.sh | 4 ++++ sbin/stop-slaves.sh | 5 +++++ 6 files changed, 98 insertions(+), 2 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index e6b5956d1e7e2..6bc6819d8da92 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -58,6 +58,7 @@ echo "Version is ${VERSION}" # Initialize defaults SPARK_HADOOP_VERSION=1.0.4 SPARK_YARN=false +SPARK_TACHYON=false MAKE_TGZ=false # Parse arguments @@ -70,6 +71,9 @@ while (( "$#" )); do --with-yarn) SPARK_YARN=true ;; + --with-tachyon) + SPARK_TACHYON=true + ;; --tgz) MAKE_TGZ=true ;; @@ -90,6 +94,12 @@ else echo "YARN disabled" fi +if [ "$SPARK_TACHYON" == "true" ]; then + echo "Tachyon Enabled" +else + echo "Tachyon Disabled" +fi + # Build fat JAR export SPARK_HADOOP_VERSION export SPARK_YARN @@ -113,6 +123,28 @@ cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" +# Download and copy in tachyon, if requested +if [ "$SPARK_TACHYON" == "true" ]; then + TACHYON_VERSION="0.4.1" + TACHYON_URL="https://github.com/amplab/tachyon/releases/download/v${TACHYON_VERSION}/tachyon-${TACHYON_VERSION}-bin.tar.gz" + + TMPD=`mktemp -d` + + pushd $TMPD > /dev/null + echo "Fetchting tachyon tgz" + wget "$TACHYON_URL" + + tar xf "tachyon-${TACHYON_VERSION}-bin.tar.gz" + cp "tachyon-${TACHYON_VERSION}/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/jars" + mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web" + cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon" + cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" + sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" + + popd > /dev/null + rm -rf $TMPD +fi + if [ "$MAKE_TGZ" == "true" ]; then TARDIR="$FWDIR/spark-$VERSION" cp -r "$DISTDIR" "$TARDIR" diff --git a/sbin/start-all.sh b/sbin/start-all.sh index 2daf49db359df..5c89ab4d86b3a 100755 --- a/sbin/start-all.sh +++ b/sbin/start-all.sh @@ -24,11 +24,22 @@ sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` +TACHYON_STR="" + +while (( "$#" )); do +case $1 in + --with-tachyon) + TACHYON_STR="--with-tachyon" + ;; + esac +shift +done + # Load the Spark configuration . "$sbin/spark-config.sh" # Start Master -"$sbin"/start-master.sh +"$sbin"/start-master.sh $TACHYON_STR # Start Workers -"$sbin"/start-slaves.sh +"$sbin"/start-slaves.sh $TACHYON_STR diff --git a/sbin/start-master.sh b/sbin/start-master.sh index ec3dfdb4197ec..03a3428aea9f1 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -22,6 +22,21 @@ sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` +START_TACHYON=false + +while (( "$#" )); do +case $1 in + --with-tachyon) + if [ ! -e "$sbin"/../tachyon/bin/tachyon ]; then + echo "Error: --with-tachyon specified, but tachyon not found." + exit -1 + fi + START_TACHYON=true + ;; + esac +shift +done + . "$sbin/spark-config.sh" if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then @@ -41,3 +56,9 @@ if [ "$SPARK_MASTER_WEBUI_PORT" = "" ]; then fi "$sbin"/spark-daemon.sh start org.apache.spark.deploy.master.Master 1 --ip $SPARK_MASTER_IP --port $SPARK_MASTER_PORT --webui-port $SPARK_MASTER_WEBUI_PORT + +if [ "$START_TACHYON" == "true" ]; then + "$sbin"/../tachyon/bin/tachyon bootstrap-conf $SPARK_MASTER_IP + "$sbin"/../tachyon/bin/tachyon format -s + "$sbin"/../tachyon/bin/tachyon-start.sh master +fi diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index fd5cdeb1e6788..da641cfe3c6fa 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -20,6 +20,22 @@ sbin=`dirname "$0"` sbin=`cd "$sbin"; pwd` + +START_TACHYON=false + +while (( "$#" )); do +case $1 in + --with-tachyon) + if [ ! -e "$sbin"/../tachyon/bin/tachyon ]; then + echo "Error: --with-tachyon specified, but tachyon not found." + exit -1 + fi + START_TACHYON=true + ;; + esac +shift +done + . "$sbin/spark-config.sh" if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then @@ -35,6 +51,13 @@ if [ "$SPARK_MASTER_IP" = "" ]; then SPARK_MASTER_IP=`hostname` fi +if [ "$START_TACHYON" == "true" ]; then + "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon bootstrap-conf $SPARK_MASTER_IP + + # set -t so we can call sudo + SPARK_SSH_OPTS="-o StrictHostKeyChecking=no -t" "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/../tachyon/bin/tachyon-start.sh" worker SudoMount \; sleep 1 +fi + # Launch the slaves if [ "$SPARK_WORKER_INSTANCES" = "" ]; then exec "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin/start-slave.sh" 1 spark://$SPARK_MASTER_IP:$SPARK_MASTER_PORT diff --git a/sbin/stop-master.sh b/sbin/stop-master.sh index 2adabd426563c..b6bdaa4db373c 100755 --- a/sbin/stop-master.sh +++ b/sbin/stop-master.sh @@ -25,3 +25,7 @@ sbin=`cd "$sbin"; pwd` . "$sbin/spark-config.sh" "$sbin"/spark-daemon.sh stop org.apache.spark.deploy.master.Master 1 + +if [ -e "$sbin"/../tachyon/bin/tachyon ]; then + "$sbin"/../tachyon/bin/tachyon killAll tachyon.master.Master +fi diff --git a/sbin/stop-slaves.sh b/sbin/stop-slaves.sh index eb803b4900347..6bf393ccd4b09 100755 --- a/sbin/stop-slaves.sh +++ b/sbin/stop-slaves.sh @@ -26,6 +26,11 @@ if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then . "${SPARK_CONF_DIR}/spark-env.sh" fi +# do before the below calls as they exec +if [ -e "$sbin"/../tachyon/bin/tachyon ]; then + "$sbin/slaves.sh" cd "$SPARK_HOME" \; "$sbin"/../tachyon/bin/tachyon killAll tachyon.worker.Worker +fi + if [ "$SPARK_WORKER_INSTANCES" = "" ]; then "$sbin"/spark-daemons.sh stop org.apache.spark.deploy.worker.Worker 1 else From d55ec86de2e96f7dc9d1dd107daa35c3823791ec Mon Sep 17 00:00:00 2001 From: "shiyun.wxm" Date: Wed, 19 Mar 2014 01:42:34 -0700 Subject: [PATCH 106/397] bugfix: Wrong "Duration" in "Active Stages" in stages page If a stage which has completed once loss parts of data, it will be resubmitted. At this time, it appears that stage.completionTime > stage.submissionTime. Author: shiyun.wxm Closes #170 from BlackNiuza/duration_problem and squashes the following commits: a86d261 [shiyun.wxm] tow space indent c0d7b24 [shiyun.wxm] change the style 3b072e1 [shiyun.wxm] fix scala style f20701e [shiyun.wxm] bugfix: "Duration" in "Active Stages" in stages page --- core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index c5fd3ae16dc9f..99828487f1902 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -106,7 +106,9 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr val description = listener.stageIdToDescription.get(s.stageId) .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) val finishTime = s.completionTime.getOrElse(System.currentTimeMillis()) - val duration = s.submissionTime.map(t => finishTime - t) + val duration = s.submissionTime.map { t => + if (finishTime > t) finishTime - t else System.currentTimeMillis - t + } {s.stageId} From 6112270c94f1a30a461a91f6e56485a5eaec2606 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Wed, 19 Mar 2014 08:09:20 -0500 Subject: [PATCH 107/397] SPARK-1203 fix saving to hdfs from yarn Author: Thomas Graves Closes #173 from tgravescs/SPARK-1203 and squashes the following commits: 4fd5ded [Thomas Graves] adding import 964e3f7 [Thomas Graves] SPARK-1203 fix saving to hdfs from yarn --- core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 2 ++ 1 file changed, 2 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 447deafff53cd..75fc02acd1bce 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -41,6 +41,7 @@ import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutput import org.apache.hadoop.mapred.SparkHadoopWriter import org.apache.spark._ +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} @@ -723,6 +724,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) if (valueClass == null) { throw new SparkException("Output value class not set") } + SparkHadoopUtil.get.addCredentials(conf) logDebug("Saving as hadoop file of type (" + keyClass.getSimpleName + ", " + valueClass.getSimpleName + ")") From ab747d39ddc7c8a314ed2fb26548fc5652af0d74 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 19 Mar 2014 12:46:55 -0700 Subject: [PATCH 108/397] Bugfixes/improvements to scheduler Move the PR#517 of apache-incubator-spark to the apache-spark Author: Mridul Muralidharan Closes #159 from mridulm/master and squashes the following commits: 5ff59c2 [Mridul Muralidharan] Change property in suite also 167fad8 [Mridul Muralidharan] Address review comments 9bda70e [Mridul Muralidharan] Address review comments, akwats add to failedExecutors 270d841 [Mridul Muralidharan] Address review comments fa5d9f1 [Mridul Muralidharan] Bugfixes/improvements to scheduler : PR #517 --- .../spark/scheduler/TaskSchedulerImpl.scala | 3 +- .../spark/scheduler/TaskSetManager.scala | 82 +++++++++++++---- .../spark/scheduler/TaskSetManagerSuite.scala | 87 +++++++++++++++++++ 3 files changed, 152 insertions(+), 20 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 23b06612fd7ab..abff252597e16 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -235,7 +235,8 @@ private[spark] class TaskSchedulerImpl( taskIdToExecutorId(tid) = execId activeExecutorIds += execId executorsByHost(host) += execId - availableCpus(i) -= 1 + availableCpus(i) -= taskSet.CPUS_PER_TASK + assert (availableCpus(i) >= 0) launchedTask = true } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index 5ea4557bbf56a..a73343c1c0826 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -59,6 +59,15 @@ private[spark] class TaskSetManager( // CPUs to request per task val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) + /* + * Sometimes if an executor is dead or in an otherwise invalid state, the driver + * does not realize right away leading to repeated task failures. If enabled, + * this temporarily prevents a task from re-launching on an executor where + * it just failed. + */ + private val EXECUTOR_TASK_BLACKLIST_TIMEOUT = + conf.getLong("spark.scheduler.executorTaskBlacklistTime", 0L) + // Quantile of tasks at which to start speculation val SPECULATION_QUANTILE = conf.getDouble("spark.speculation.quantile", 0.75) val SPECULATION_MULTIPLIER = conf.getDouble("spark.speculation.multiplier", 1.5) @@ -71,7 +80,9 @@ private[spark] class TaskSetManager( val numTasks = tasks.length val copiesRunning = new Array[Int](numTasks) val successful = new Array[Boolean](numTasks) - val numFailures = new Array[Int](numTasks) + private val numFailures = new Array[Int](numTasks) + // key is taskId, value is a Map of executor id to when it failed + private val failedExecutors = new HashMap[Int, HashMap[String, Long]]() val taskAttempts = Array.fill[List[TaskInfo]](numTasks)(Nil) var tasksSuccessful = 0 @@ -228,12 +239,18 @@ private[spark] class TaskSetManager( * This method also cleans up any tasks in the list that have already * been launched, since we want that to happen lazily. */ - private def findTaskFromList(list: ArrayBuffer[Int]): Option[Int] = { - while (!list.isEmpty) { - val index = list.last - list.trimEnd(1) - if (copiesRunning(index) == 0 && !successful(index)) { - return Some(index) + private def findTaskFromList(execId: String, list: ArrayBuffer[Int]): Option[Int] = { + var indexOffset = list.size + + while (indexOffset > 0) { + indexOffset -= 1 + val index = list(indexOffset) + if (!executorIsBlacklisted(execId, index)) { + // This should almost always be list.trimEnd(1) to remove tail + list.remove(indexOffset) + if (copiesRunning(index) == 0 && !successful(index)) { + return Some(index) + } } } None @@ -244,6 +261,21 @@ private[spark] class TaskSetManager( taskAttempts(taskIndex).exists(_.host == host) } + /** + * Is this re-execution of a failed task on an executor it already failed in before + * EXECUTOR_TASK_BLACKLIST_TIMEOUT has elapsed ? + */ + private def executorIsBlacklisted(execId: String, taskId: Int): Boolean = { + if (failedExecutors.contains(taskId)) { + val failed = failedExecutors.get(taskId).get + + return failed.contains(execId) && + clock.getTime() - failed.get(execId).get < EXECUTOR_TASK_BLACKLIST_TIMEOUT + } + + false + } + /** * Return a speculative task for a given executor if any are available. The task should not have * an attempt running on this host, in case the host is slow. In addition, the task should meet @@ -254,10 +286,13 @@ private[spark] class TaskSetManager( { speculatableTasks.retain(index => !successful(index)) // Remove finished tasks from set + def canRunOnHost(index: Int): Boolean = + !hasAttemptOnHost(index, host) && !executorIsBlacklisted(execId, index) + if (!speculatableTasks.isEmpty) { // Check for process-local or preference-less tasks; note that tasks can be process-local // on multiple nodes when we replicate cached blocks, as in Spark Streaming - for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) { + for (index <- speculatableTasks if canRunOnHost(index)) { val prefs = tasks(index).preferredLocations val executors = prefs.flatMap(_.executorId) if (prefs.size == 0 || executors.contains(execId)) { @@ -268,7 +303,7 @@ private[spark] class TaskSetManager( // Check for node-local tasks if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) { - for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) { + for (index <- speculatableTasks if canRunOnHost(index)) { val locations = tasks(index).preferredLocations.map(_.host) if (locations.contains(host)) { speculatableTasks -= index @@ -280,7 +315,7 @@ private[spark] class TaskSetManager( // Check for rack-local tasks if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { for (rack <- sched.getRackForHost(host)) { - for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) { + for (index <- speculatableTasks if canRunOnHost(index)) { val racks = tasks(index).preferredLocations.map(_.host).map(sched.getRackForHost) if (racks.contains(rack)) { speculatableTasks -= index @@ -292,7 +327,7 @@ private[spark] class TaskSetManager( // Check for non-local tasks if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { - for (index <- speculatableTasks if !hasAttemptOnHost(index, host)) { + for (index <- speculatableTasks if canRunOnHost(index)) { speculatableTasks -= index return Some((index, TaskLocality.ANY)) } @@ -309,12 +344,12 @@ private[spark] class TaskSetManager( private def findTask(execId: String, host: String, locality: TaskLocality.Value) : Option[(Int, TaskLocality.Value)] = { - for (index <- findTaskFromList(getPendingTasksForExecutor(execId))) { + for (index <- findTaskFromList(execId, getPendingTasksForExecutor(execId))) { return Some((index, TaskLocality.PROCESS_LOCAL)) } if (TaskLocality.isAllowed(locality, TaskLocality.NODE_LOCAL)) { - for (index <- findTaskFromList(getPendingTasksForHost(host))) { + for (index <- findTaskFromList(execId, getPendingTasksForHost(host))) { return Some((index, TaskLocality.NODE_LOCAL)) } } @@ -322,19 +357,19 @@ private[spark] class TaskSetManager( if (TaskLocality.isAllowed(locality, TaskLocality.RACK_LOCAL)) { for { rack <- sched.getRackForHost(host) - index <- findTaskFromList(getPendingTasksForRack(rack)) + index <- findTaskFromList(execId, getPendingTasksForRack(rack)) } { return Some((index, TaskLocality.RACK_LOCAL)) } } // Look for no-pref tasks after rack-local tasks since they can run anywhere. - for (index <- findTaskFromList(pendingTasksWithNoPrefs)) { + for (index <- findTaskFromList(execId, pendingTasksWithNoPrefs)) { return Some((index, TaskLocality.PROCESS_LOCAL)) } if (TaskLocality.isAllowed(locality, TaskLocality.ANY)) { - for (index <- findTaskFromList(allPendingTasks)) { + for (index <- findTaskFromList(execId, allPendingTasks)) { return Some((index, TaskLocality.ANY)) } } @@ -460,6 +495,7 @@ private[spark] class TaskSetManager( logInfo("Ignorning task-finished event for TID " + tid + " because task " + index + " has already completed successfully") } + failedExecutors.remove(index) maybeFinishTaskSet() } @@ -480,7 +516,7 @@ private[spark] class TaskSetManager( logWarning("Lost TID %s (task %s:%d)".format(tid, taskSet.id, index)) } var taskMetrics : TaskMetrics = null - var failureReason = "unknown" + var failureReason: String = null reason match { case fetchFailed: FetchFailed => logWarning("Loss was due to fetch failure from " + fetchFailed.bmAddress) @@ -488,9 +524,11 @@ private[spark] class TaskSetManager( successful(index) = true tasksSuccessful += 1 } + // Not adding to failed executors for FetchFailed. isZombie = true case TaskKilled => + // Not adding to failed executors for TaskKilled. logWarning("Task %d was killed.".format(tid)) case ef: ExceptionFailure => @@ -504,7 +542,8 @@ private[spark] class TaskSetManager( return } val key = ef.description - failureReason = "Exception failure: %s".format(ef.description) + failureReason = "Exception failure in TID %s on host %s: %s".format( + tid, info.host, ef.description) val now = clock.getTime() val (printFull, dupCount) = { if (recentExceptions.contains(key)) { @@ -533,11 +572,16 @@ private[spark] class TaskSetManager( failureReason = "Lost result for TID %s on host %s".format(tid, info.host) logWarning(failureReason) - case _ => {} + case _ => + failureReason = "TID %s on host %s failed for unknown reason".format(tid, info.host) } + // always add to failed executors + failedExecutors.getOrElseUpdate(index, new HashMap[String, Long]()). + put(info.executorId, clock.getTime()) sched.dagScheduler.taskEnded(tasks(index), reason, null, null, info, taskMetrics) addPendingTask(index) if (!isZombie && state != TaskState.KILLED) { + assert (null != failureReason) numFailures(index) += 1 if (numFailures(index) >= maxTaskFailures) { logError("Task %s:%d failed %d times; aborting job".format( diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 33cc7588b919c..73153d23c4698 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -298,6 +298,93 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { } } + test("executors should be blacklisted after task failure, in spite of locality preferences") { + val rescheduleDelay = 300L + val conf = new SparkConf(). + set("spark.scheduler.executorTaskBlacklistTime", rescheduleDelay.toString). + // dont wait to jump locality levels in this test + set("spark.locality.wait", "0") + + sc = new SparkContext("local", "test", conf) + // two executors on same host, one on different. + val sched = new FakeTaskScheduler(sc, ("exec1", "host1"), + ("exec1.1", "host1"), ("exec2", "host2")) + // affinity to exec1 on host1 - which we will fail. + val taskSet = FakeTask.createTaskSet(1, Seq(TaskLocation("host1", "exec1"))) + val clock = new FakeClock + val manager = new TaskSetManager(sched, taskSet, 4, clock) + + { + val offerResult = manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL) + assert(offerResult.isDefined, "Expect resource offer to return a task") + + assert(offerResult.get.index === 0) + assert(offerResult.get.executorId === "exec1") + + // Cause exec1 to fail : failure 1 + manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) + assert(!sched.taskSetsFailed.contains(taskSet.id)) + + // Ensure scheduling on exec1 fails after failure 1 due to blacklist + assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.RACK_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.ANY).isEmpty) + } + + // Run the task on exec1.1 - should work, and then fail it on exec1.1 + { + val offerResult = manager.resourceOffer("exec1.1", "host1", 1, TaskLocality.NODE_LOCAL) + assert(offerResult.isDefined, + "Expect resource offer to return a task for exec1.1, offerResult = " + offerResult) + + assert(offerResult.get.index === 0) + assert(offerResult.get.executorId === "exec1.1") + + // Cause exec1.1 to fail : failure 2 + manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) + assert(!sched.taskSetsFailed.contains(taskSet.id)) + + // Ensure scheduling on exec1.1 fails after failure 2 due to blacklist + assert(manager.resourceOffer("exec1.1", "host1", 1, TaskLocality.NODE_LOCAL).isEmpty) + } + + // Run the task on exec2 - should work, and then fail it on exec2 + { + val offerResult = manager.resourceOffer("exec2", "host2", 1, TaskLocality.ANY) + assert(offerResult.isDefined, "Expect resource offer to return a task") + + assert(offerResult.get.index === 0) + assert(offerResult.get.executorId === "exec2") + + // Cause exec2 to fail : failure 3 + manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) + assert(!sched.taskSetsFailed.contains(taskSet.id)) + + // Ensure scheduling on exec2 fails after failure 3 due to blacklist + assert(manager.resourceOffer("exec2", "host2", 1, TaskLocality.ANY).isEmpty) + } + + // After reschedule delay, scheduling on exec1 should be possible. + clock.advance(rescheduleDelay) + + { + val offerResult = manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL) + assert(offerResult.isDefined, "Expect resource offer to return a task") + + assert(offerResult.get.index === 0) + assert(offerResult.get.executorId === "exec1") + + assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL).isEmpty) + + // Cause exec1 to fail : failure 4 + manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) + } + + // we have failed the same task 4 times now : task id should now be in taskSetsFailed + assert(sched.taskSetsFailed.contains(taskSet.id)) + } + def createTaskResult(id: Int): DirectTaskResult[Int] = { val valueSer = SparkEnv.get.serializer.newInstance() new DirectTaskResult[Int](valueSer.serialize(id), mutable.Map.empty, new TaskMetrics) From 79d07d66040f206708e14de393ab0b80020ed96a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 19 Mar 2014 13:17:01 -0700 Subject: [PATCH 109/397] [SPARK-1132] Persisting Web UI through refactoring the SparkListener interface The fleeting nature of the Spark Web UI has long been a problem reported by many users: The existing Web UI disappears as soon as the associated application terminates. This is because SparkUI is tightly coupled with SparkContext, and cannot be instantiated independently from it. To solve this, some state must be saved to persistent storage while the application is still running. The approach taken by this PR involves persisting the UI state through SparkListenerEvents. This requires a major refactor of the SparkListener interface because existing events (1) maintain deep references, making de/serialization is difficult, and (2) do not encode all the information displayed on the UI. In this design, each existing listener for the UI (e.g. ExecutorsListener) maintains state that can be fully constructed from SparkListenerEvents. This state is then supplied to the parent UI (e.g. ExecutorsUI), which renders the associated page(s) on demand. This PR introduces two important classes: the **EventLoggingListener**, and the **ReplayListenerBus**. In a live application, SparkUI registers an EventLoggingListener with the SparkContext in addition to the existing listeners. Over the course of the application, this listener serializes and logs all events to persisted storage. Then, after the application has finished, the SparkUI can be revived by replaying all the logged events to the existing UI listeners through the ReplayListenerBus. This feature is currently integrated with the Master Web UI, which optionally rebuilds a SparkUI from event logs as soon as the corresponding application finishes. More details can be found in the commit messages, comments within the code, and the [design doc](https://spark-project.atlassian.net/secure/attachment/12900/PersistingSparkWebUI.pdf). Comments and feedback are most welcome. Author: Andrew Or Author: andrewor14 Closes #42 from andrewor14/master and squashes the following commits: e5f14fa [Andrew Or] Merge github.com:apache/spark a1c5cd9 [Andrew Or] Merge github.com:apache/spark b8ba817 [Andrew Or] Remove UI from map when removing application in Master 83af656 [Andrew Or] Scraps and pieces (no functionality change) 222adcd [Andrew Or] Merge github.com:apache/spark 124429f [Andrew Or] Clarify LiveListenerBus behavior + Add tests for new behavior f80bd31 [Andrew Or] Simplify static handler and BlockManager status update logic 9e14f97 [Andrew Or] Moved around functionality + renamed classes per Patrick 6740e49 [Andrew Or] Fix comment nits 650eb12 [Andrew Or] Add unit tests + Fix bugs found through tests 45fd84c [Andrew Or] Remove now deprecated test c5c2c8f [Andrew Or] Remove list of (TaskInfo, TaskMetrics) from StageInfo 3456090 [Andrew Or] Address Patrick's comments bf80e3d [Andrew Or] Imports, comments, and code formatting, once again (minor) ac69ec8 [Andrew Or] Fix test fail d801d11 [Andrew Or] Merge github.com:apache/spark (major) dc93915 [Andrew Or] Imports, comments, and code formatting (minor) 77ba283 [Andrew Or] Address Kay's and Patrick's comments b6eaea7 [Andrew Or] Treating SparkUI as a handler of MasterUI d59da5f [Andrew Or] Avoid logging all the blocks on each executor d6e3b4a [Andrew Or] Merge github.com:apache/spark ca258a4 [Andrew Or] Master UI - add support for reading compressed event logs 176e68e [Andrew Or] Fix deprecated message for JavaSparkContext (minor) 4f69c4a [Andrew Or] Master UI - Rebuild SparkUI on application finish 291b2be [Andrew Or] Correct directory in log message "INFO: Logging events to " 1ba3407 [Andrew Or] Add a few configurable options to event logging e375431 [Andrew Or] Add new constructors for SparkUI 18b256d [Andrew Or] Refactor out event logging and replaying logic from UI bb4c503 [Andrew Or] Use a more mnemonic path for logging aef411c [Andrew Or] Fix bug: storage status was not reflected on UI in the local case 03eda0b [Andrew Or] Fix HDFS flush behavior 36b3e5d [Andrew Or] Add HDFS support for event logging cceff2b [andrewor14] Fix 100 char format fail 2fee310 [Andrew Or] Address Patrick's comments 2981d61 [Andrew Or] Move SparkListenerBus out of DAGScheduler + Clean up 5d2cec1 [Andrew Or] JobLogger: ID -> Id 0503e4b [Andrew Or] Fix PySpark tests + remove sc.clearFiles/clearJars 4d2fb0c [Andrew Or] Fix format fail faa113e [Andrew Or] General clean up d47585f [Andrew Or] Clean up FileLogger 472fd8a [Andrew Or] Fix a couple of tests 996d7a2 [Andrew Or] Reflect RDD unpersist on UI 7b2f811 [Andrew Or] Guard against TaskMetrics NPE + Fix tests d1f4285 [Andrew Or] Migrate from lift-json to json4s-jackson 28019ca [Andrew Or] Merge github.com:apache/spark bbe3501 [Andrew Or] Embed storage status and RDD info in Task events 6631c02 [Andrew Or] More formatting changes, this time mainly for Json DSL 70e7e7a [Andrew Or] Formatting changes e9e1c6d [Andrew Or] Move all JSON de/serialization logic to JsonProtocol d646df6 [Andrew Or] Completely decouple SparkUI from SparkContext 6814da0 [Andrew Or] Explicitly register each UI listener rather than through some magic 64d2ce1 [Andrew Or] Fix BlockManagerUI bug by introducing new event 4273013 [Andrew Or] Add a gateway SparkListener to simplify event logging 904c729 [Andrew Or] Fix another major bug 5ac906d [Andrew Or] Mostly naming, formatting, and code style changes 3fd584e [Andrew Or] Fix two major bugs f3fc13b [Andrew Or] General refactor 4dfcd22 [Andrew Or] Merge git://git.apache.org/incubator-spark into persist-ui b3976b0 [Andrew Or] Add functionality of reconstructing a persisted UI from SparkContext 8add36b [Andrew Or] JobProgressUI: Add JSON functionality d859efc [Andrew Or] BlockManagerUI: Add JSON functionality c4cd480 [Andrew Or] Also deserialize new events 8a2ebe6 [Andrew Or] Fix bugs for EnvironmentUI and ExecutorsUI de8a1cd [Andrew Or] Serialize events both to and from JSON (rather than just to) bf0b2e9 [Andrew Or] ExecutorUI: Serialize events rather than arbitary executor information bb222b9 [Andrew Or] ExecutorUI: render completely from JSON dcbd312 [Andrew Or] Add JSON Serializability for all SparkListenerEvent's 10ed49d [Andrew Or] Merge github.com:apache/incubator-spark into persist-ui 8e09306 [Andrew Or] Use JSON for ExecutorsUI e3ae35f [Andrew Or] Merge github.com:apache/incubator-spark 3ddeb7e [Andrew Or] Also privatize fields 090544a [Andrew Or] Privatize methods 13920c9 [Andrew Or] Update docs bd5a1d7 [Andrew Or] Typo: phyiscal -> physical 287ef44 [Andrew Or] Avoid reading the entire batch into memory; also simplify streaming logic 3df7005 [Andrew Or] Merge branch 'master' of github.com:andrewor14/incubator-spark a531d2e [Andrew Or] Relax assumptions on compressors and serializers when batching 164489d [Andrew Or] Relax assumptions on compressors and serializers when batching --- .../scala/org/apache/spark/Aggregator.scala | 2 - .../scala/org/apache/spark/CacheManager.scala | 76 +- .../org/apache/spark/SecurityManager.scala | 8 +- .../scala/org/apache/spark/SparkContext.scala | 93 ++- .../scala/org/apache/spark/SparkEnv.scala | 80 +- .../org/apache/spark/TaskEndReason.scala | 3 +- .../spark/api/java/JavaSparkContext.scala | 2 + .../spark/deploy/ApplicationDescription.scala | 5 +- .../apache/spark/deploy/JsonProtocol.scala | 1 - .../spark/deploy/master/ApplicationInfo.scala | 6 - .../apache/spark/deploy/master/Master.scala | 73 +- .../deploy/master/ui/ApplicationPage.scala | 3 +- .../spark/deploy/master/ui/IndexPage.scala | 3 +- .../spark/deploy/master/ui/MasterWebUI.scala | 84 ++- .../apache/spark/deploy/worker/Worker.scala | 18 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 84 +-- .../org/apache/spark/executor/Executor.scala | 8 +- .../apache/spark/executor/TaskMetrics.scala | 13 +- .../apache/spark/io/CompressionCodec.scala | 5 +- .../spark/metrics/sink/MetricsServlet.scala | 8 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../apache/spark/scheduler/DAGScheduler.scala | 306 ++++---- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../spark/scheduler/DAGSchedulerSource.scala | 6 +- .../scheduler/EventLoggingListener.scala | 98 +++ .../apache/spark/scheduler/JobLogger.scala | 266 ++----- .../apache/spark/scheduler/JobResult.scala | 5 +- .../apache/spark/scheduler/JobWaiter.scala | 2 +- .../spark/scheduler/LiveListenerBus.scala | 101 +++ .../org/apache/spark/scheduler/Pool.scala | 2 +- .../spark/scheduler/ReplayListenerBus.scala | 104 +++ .../spark/scheduler/SparkListener.scala | 176 +++-- .../spark/scheduler/SparkListenerBus.scala | 167 ++-- .../apache/spark/scheduler/StageInfo.scala | 29 +- .../apache/spark/scheduler/TaskLocality.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 6 +- .../cluster/SparkDeploySchedulerBackend.scala | 9 +- .../mesos/CoarseMesosSchedulerBackend.scala | 5 +- .../cluster/mesos/MesosSchedulerBackend.scala | 5 +- .../apache/spark/storage/BlockManager.scala | 241 +++--- .../apache/spark/storage/BlockManagerId.scala | 3 +- .../spark/storage/BlockManagerMaster.scala | 6 +- .../storage/BlockManagerMasterActor.scala | 160 ++-- .../apache/spark/storage/MemoryStore.scala | 100 +-- .../org/apache/spark/storage/PutResult.scala | 12 +- .../spark/storage/StorageStatusListener.scala | 94 +++ .../apache/spark/storage/StorageUtils.scala | 119 +-- .../apache/spark/storage/ThreadingTest.scala | 7 +- .../org/apache/spark/ui/JettyUtils.scala | 121 +-- .../scala/org/apache/spark/ui/SparkUI.scala | 101 ++- .../scala/org/apache/spark/ui/UIUtils.scala | 40 +- .../apache/spark/ui/env/EnvironmentUI.scala | 108 ++- .../apache/spark/ui/exec/ExecutorsUI.scala | 224 +++--- .../spark/ui/jobs/ExecutorSummary.scala | 2 +- .../apache/spark/ui/jobs/ExecutorTable.scala | 48 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 90 ++- .../spark/ui/jobs/JobProgressListener.scala | 277 ++++--- .../apache/spark/ui/jobs/JobProgressUI.scala | 32 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 29 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 25 +- .../org/apache/spark/ui/jobs/StagePage.scala | 248 +++--- .../org/apache/spark/ui/jobs/StageTable.scala | 75 +- .../spark/ui/storage/BlockManagerUI.scala | 74 +- .../apache/spark/ui/storage/IndexPage.scala | 41 +- .../org/apache/spark/ui/storage/RDDPage.scala | 86 ++- .../org/apache/spark/util/FileLogger.scala | 165 ++++ .../org/apache/spark/util/JsonProtocol.scala | 710 ++++++++++++++++++ .../scala/org/apache/spark/util/Utils.scala | 38 +- .../java/org/apache/spark/JavaAPISuite.java | 23 +- .../org/apache/spark/CacheManagerSuite.scala | 11 +- .../apache/spark/JobCancellationSuite.scala | 8 +- .../org/apache/spark/PipedRDDSuite.scala | 5 +- .../SparkContextSchedulerCreationSuite.scala | 4 +- .../spark/deploy/JsonProtocolSuite.scala | 11 +- .../spark/scheduler/DAGSchedulerSuite.scala | 15 +- .../spark/scheduler/JobLoggerSuite.scala | 126 ---- .../spark/scheduler/SparkListenerSuite.scala | 150 ++-- .../scheduler/TaskSchedulerImplSuite.scala | 8 +- .../spark/scheduler/TaskSetManagerSuite.scala | 8 +- .../spark/storage/BlockManagerSuite.scala | 69 +- .../scala/org/apache/spark/ui/UISuite.scala | 22 +- .../ui/jobs/JobProgressListenerSuite.scala | 20 +- .../apache/spark/util/JsonProtocolSuite.scala | 559 ++++++++++++++ docs/configuration.md | 25 +- 84 files changed, 4268 insertions(+), 1944 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala create mode 100644 core/src/main/scala/org/apache/spark/util/FileLogger.scala create mode 100644 core/src/main/scala/org/apache/spark/util/JsonProtocol.scala delete mode 100644 core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index c4579cf6ad560..ceead59b79ed6 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,8 +17,6 @@ package org.apache.spark -import scala.{Option, deprecated} - import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index 872e892c04fe6..c7893f288b4b5 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -20,11 +20,12 @@ package org.apache.spark import scala.collection.mutable.{ArrayBuffer, HashSet} import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockManager, RDDBlockId, StorageLevel} +import org.apache.spark.storage.{BlockId, BlockManager, BlockStatus, RDDBlockId, StorageLevel} -/** Spark class responsible for passing RDDs split contents to the BlockManager and making - sure a node doesn't load two copies of an RDD at once. - */ +/** + * Spark class responsible for passing RDDs split contents to the BlockManager and making + * sure a node doesn't load two copies of an RDD at once. + */ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { /** Keys of RDD splits that are being computed/loaded. */ @@ -49,11 +50,11 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { try {loading.wait()} catch {case _ : Throwable =>} } logInfo("Finished waiting for %s".format(key)) - // See whether someone else has successfully loaded it. The main way this would fail - // is for the RDD-level cache eviction policy if someone else has loaded the same RDD - // partition but we didn't want to make space for it. However, that case is unlikely - // because it's unlikely that two threads would work on the same RDD partition. One - // downside of the current code is that threads wait serially if this does happen. + /* See whether someone else has successfully loaded it. The main way this would fail + * is for the RDD-level cache eviction policy if someone else has loaded the same RDD + * partition but we didn't want to make space for it. However, that case is unlikely + * because it's unlikely that two threads would work on the same RDD partition. One + * downside of the current code is that threads wait serially if this does happen. */ blockManager.get(key) match { case Some(values) => return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) @@ -69,32 +70,45 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { // If we got here, we have to load the split logInfo("Partition %s not found, computing it".format(key)) val computedValues = rdd.computeOrReadCheckpoint(split, context) + // Persist the result, so long as the task is not running locally if (context.runningLocally) { return computedValues } - if (storageLevel.useDisk && !storageLevel.useMemory) { - // In the case that this RDD is to be persisted using DISK_ONLY - // the iterator will be passed directly to the blockManager (rather then - // caching it to an ArrayBuffer first), then the resulting block data iterator - // will be passed back to the user. If the iterator generates a lot of data, - // this means that it doesn't all have to be held in memory at one time. - // This could also apply to MEMORY_ONLY_SER storage, but we need to make sure - // blocks aren't dropped by the block store before enabling that. - blockManager.put(key, computedValues, storageLevel, tellMaster = true) - return blockManager.get(key) match { - case Some(values) => - return new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) - case None => - logInfo("Failure to store %s".format(key)) - throw new Exception("Block manager failed to return persisted valued") + + // Keep track of blocks with updated statuses + var updatedBlocks = Seq[(BlockId, BlockStatus)]() + val returnValue: Iterator[T] = { + if (storageLevel.useDisk && !storageLevel.useMemory) { + /* In the case that this RDD is to be persisted using DISK_ONLY + * the iterator will be passed directly to the blockManager (rather then + * caching it to an ArrayBuffer first), then the resulting block data iterator + * will be passed back to the user. If the iterator generates a lot of data, + * this means that it doesn't all have to be held in memory at one time. + * This could also apply to MEMORY_ONLY_SER storage, but we need to make sure + * blocks aren't dropped by the block store before enabling that. */ + updatedBlocks = blockManager.put(key, computedValues, storageLevel, tellMaster = true) + blockManager.get(key) match { + case Some(values) => + new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) + case None => + logInfo("Failure to store %s".format(key)) + throw new Exception("Block manager failed to return persisted valued") + } + } else { + // In this case the RDD is cached to an array buffer. This will save the results + // if we're dealing with a 'one-time' iterator + val elements = new ArrayBuffer[Any] + elements ++= computedValues + updatedBlocks = blockManager.put(key, elements, storageLevel, tellMaster = true) + elements.iterator.asInstanceOf[Iterator[T]] } - } else { - // In this case the RDD is cached to an array buffer. This will save the results - // if we're dealing with a 'one-time' iterator - val elements = new ArrayBuffer[Any] - elements ++= computedValues - blockManager.put(key, elements, storageLevel, tellMaster = true) - return elements.iterator.asInstanceOf[Iterator[T]] } + + // Update task metrics to include any blocks whose storage status is updated + val metrics = context.taskMetrics + metrics.updatedBlocks = Some(updatedBlocks) + + returnValue + } finally { loading.synchronized { loading.remove(key) diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 591978c1d3630..2237ee3bb7aad 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -18,13 +18,13 @@ package org.apache.spark import java.net.{Authenticator, PasswordAuthentication} -import org.apache.hadoop.io.Text -import org.apache.hadoop.security.Credentials -import org.apache.hadoop.security.UserGroupInformation -import org.apache.spark.deploy.SparkHadoopUtil import scala.collection.mutable.ArrayBuffer +import org.apache.hadoop.io.Text + +import org.apache.spark.deploy.SparkHadoopUtil + /** * Spark class responsible for security. * diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 852ed8fe1fb91..a1003b7925715 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -132,6 +132,9 @@ class SparkContext( if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") + // An asynchronous listener bus for Spark events + private[spark] val listenerBus = new LiveListenerBus + // Create the Spark execution environment (cache, map output tracker, etc) private[spark] val env = SparkEnv.create( conf, @@ -139,7 +142,8 @@ class SparkContext( conf.get("spark.driver.host"), conf.get("spark.driver.port").toInt, isDriver = true, - isLocal = isLocal) + isLocal = isLocal, + listenerBus = listenerBus) SparkEnv.set(env) // Used to store a URL for each static file/jar together with the file's local timestamp @@ -151,9 +155,26 @@ class SparkContext( private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) - // Initialize the Spark UI + // Initialize the Spark UI, registering all associated listeners private[spark] val ui = new SparkUI(this) ui.bind() + ui.start() + + // Optionally log Spark events + private[spark] val eventLogger: Option[EventLoggingListener] = { + if (conf.getBoolean("spark.eventLog.enabled", false)) { + val logger = new EventLoggingListener(appName, conf) + listenerBus.addListener(logger) + Some(logger) + } else None + } + + // Information needed to replay logged events, if any + private[spark] val eventLoggingInfo: Option[EventLoggingInfo] = + eventLogger.map { logger => Some(logger.info) }.getOrElse(None) + + // At this point, all relevant SparkListeners have been registered, so begin releasing events + listenerBus.start() val startTime = System.currentTimeMillis() @@ -200,13 +221,13 @@ class SparkContext( executorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler - private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master, appName) + private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) taskScheduler.start() - @volatile private[spark] var dagScheduler = new DAGScheduler(taskScheduler) + @volatile private[spark] var dagScheduler = new DAGScheduler(this) dagScheduler.start() - ui.start() + postEnvironmentUpdate() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration = { @@ -571,7 +592,6 @@ class SparkContext( .flatMap(x => Utils.deserialize[Array[T]](x._2.getBytes)) } - protected[spark] def checkpointFile[T: ClassTag]( path: String ): RDD[T] = { @@ -641,10 +661,11 @@ class SparkContext( Utils.fetchFile(path, new File(SparkFiles.getRootDirectory), conf, env.securityManager) logInfo("Added file " + path + " at " + key + " with timestamp " + addedFiles(key)) + postEnvironmentUpdate() } def addSparkListener(listener: SparkListener) { - dagScheduler.addSparkListener(listener) + listenerBus.addListener(listener) } /** @@ -671,7 +692,7 @@ class SparkContext( */ def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap - def getStageInfo: Map[Stage,StageInfo] = { + def getStageInfo: Map[Stage, StageInfo] = { dagScheduler.stageToInfos } @@ -698,7 +719,7 @@ class SparkContext( } /** - * Return current scheduling mode + * Return current scheduling mode */ def getSchedulingMode: SchedulingMode.SchedulingMode = { taskScheduler.schedulingMode @@ -708,6 +729,7 @@ class SparkContext( * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. */ + @deprecated("adding files no longer creates local copies that need to be deleted", "1.0.0") def clearFiles() { addedFiles.clear() } @@ -722,6 +744,23 @@ class SparkContext( dagScheduler.getPreferredLocs(rdd, partition) } + /** + * Register an RDD to be persisted in memory and/or disk storage + */ + private[spark] def persistRDD(rdd: RDD[_]) { + persistentRdds(rdd.id) = rdd + } + + /** + * Unpersist an RDD from memory and/or disk storage + */ + private[spark] def unpersistRDD(rdd: RDD[_], blocking: Boolean = true) { + val rddId = rdd.id + env.blockManager.master.removeRdd(rddId, blocking) + persistentRdds.remove(rddId) + listenerBus.post(SparkListenerUnpersistRDD(rddId)) + } + /** * Adds a JAR dependency for all tasks to be executed on this SparkContext in the future. * The `path` passed can be either a local file, a file in HDFS (or other Hadoop-supported @@ -744,7 +783,7 @@ class SparkContext( if (SparkHadoopUtil.get.isYarnMode() && (master == "yarn-standalone" || master == "yarn-cluster")) { // In order for this to work in yarn-cluster mode the user must specify the - // --addjars option to the client to upload the file into the distributed cache + // --addjars option to the client to upload the file into the distributed cache // of the AM to make it show up in the current working directory. val fileName = new Path(uri.getPath).getName() try { @@ -752,7 +791,7 @@ class SparkContext( } catch { case e: Exception => { // For now just log an error but allow to go through so spark examples work. - // The spark examples don't really need the jar distributed since its also + // The spark examples don't really need the jar distributed since its also // the app jar. logError("Error adding jar (" + e + "), was the --addJars option used?") null @@ -773,12 +812,14 @@ class SparkContext( logInfo("Added JAR " + path + " at " + key + " with timestamp " + addedJars(key)) } } + postEnvironmentUpdate() } /** * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to * any new nodes. */ + @deprecated("adding jars no longer creates local copies that need to be deleted", "1.0.0") def clearJars() { addedJars.clear() } @@ -786,6 +827,7 @@ class SparkContext( /** Shut down the SparkContext. */ def stop() { ui.stop() + eventLogger.foreach(_.stop()) // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler @@ -793,12 +835,10 @@ class SparkContext( if (dagSchedulerCopy != null) { metadataCleaner.cancel() dagSchedulerCopy.stop() + listenerBus.stop() taskScheduler = null // TODO: Cache.stop()? env.stop() - // Clean up locally linked files - clearFiles() - clearJars() SparkEnv.set(null) ShuffleMapTask.clearCache() ResultTask.clearCache() @@ -1026,6 +1066,19 @@ class SparkContext( /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Post the environment update event once the task scheduler is ready */ + private def postEnvironmentUpdate() { + if (taskScheduler != null) { + val schedulingMode = getSchedulingMode.toString + val addedJarPaths = addedJars.keys.toSeq + val addedFilePaths = addedFiles.keys.toSeq + val environmentDetails = + SparkEnv.environmentDetails(conf, schedulingMode, addedJarPaths, addedFilePaths) + val environmentUpdate = SparkListenerEnvironmentUpdate(environmentDetails) + listenerBus.post(environmentUpdate) + } + } + /** Called by MetadataCleaner to clean up the persistentRdds map periodically */ private[spark] def cleanup(cleanupTime: Long) { persistentRdds.clearOldValues(cleanupTime) @@ -1189,9 +1242,7 @@ object SparkContext extends Logging { } /** Creates a task scheduler based on a given master URL. Extracted for testing. */ - private def createTaskScheduler(sc: SparkContext, master: String, appName: String) - : TaskScheduler = - { + private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { // Regular expression used for local[N] master format val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks @@ -1230,7 +1281,7 @@ object SparkContext extends Logging { case SPARK_REGEX(sparkUrl) => val scheduler = new TaskSchedulerImpl(sc) val masterUrls = sparkUrl.split(",").map("spark://" + _) - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) scheduler @@ -1247,7 +1298,7 @@ object SparkContext extends Logging { val localCluster = new LocalSparkCluster( numSlaves.toInt, coresPerSlave.toInt, memoryPerSlaveInt) val masterUrls = localCluster.start() - val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls, appName) + val backend = new SparkDeploySchedulerBackend(scheduler, sc, masterUrls) scheduler.initialize(backend) backend.shutdownCallback = (backend: SparkDeploySchedulerBackend) => { localCluster.stop() @@ -1307,9 +1358,9 @@ object SparkContext extends Logging { val coarseGrained = sc.conf.getBoolean("spark.mesos.coarse", false) val url = mesosUrl.stripPrefix("mesos://") // strip scheme from raw Mesos URLs val backend = if (coarseGrained) { - new CoarseMesosSchedulerBackend(scheduler, sc, url, appName) + new CoarseMesosSchedulerBackend(scheduler, sc, url) } else { - new MesosSchedulerBackend(scheduler, sc, url, appName) + new MesosSchedulerBackend(scheduler, sc, url) } scheduler.initialize(backend) scheduler diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 774cbd6441a48..a1af63fa4a391 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -17,8 +17,10 @@ package org.apache.spark +import scala.collection.JavaConversions._ import scala.collection.mutable import scala.concurrent.Await +import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker @@ -26,9 +28,10 @@ import com.google.common.collect.MapMaker import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.storage.{BlockManager, BlockManagerMaster, BlockManagerMasterActor} import org.apache.spark.network.ConnectionManager +import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.Serializer +import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** @@ -49,11 +52,11 @@ class SparkEnv private[spark] ( val broadcastManager: BroadcastManager, val blockManager: BlockManager, val connectionManager: ConnectionManager, + val securityManager: SecurityManager, val httpFileServer: HttpFileServer, val sparkFilesDir: String, val metricsSystem: MetricsSystem, - val conf: SparkConf, - val securityManager: SecurityManager) extends Logging { + val conf: SparkConf) extends Logging { // A mapping of thread ID to amount of memory used for shuffle in bytes // All accesses should be manually synchronized @@ -120,9 +123,16 @@ object SparkEnv extends Logging { hostname: String, port: Int, isDriver: Boolean, - isLocal: Boolean): SparkEnv = { + isLocal: Boolean, + listenerBus: LiveListenerBus = null): SparkEnv = { + + // Listener bus is only used on the driver + if (isDriver) { + assert(listenerBus != null, "Attempted to create driver SparkEnv with null listener bus!") + } val securityManager = new SecurityManager(conf) + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, port, conf = conf, securityManager = securityManager) @@ -172,8 +182,9 @@ object SparkEnv extends Logging { val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", - new BlockManagerMasterActor(isLocal, conf)), conf) - val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, + new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) + + val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, serializer, conf, securityManager) val connectionManager = blockManager.connectionManager @@ -233,10 +244,63 @@ object SparkEnv extends Logging { broadcastManager, blockManager, connectionManager, + securityManager, httpFileServer, sparkFilesDir, metricsSystem, - conf, - securityManager) + conf) + } + + /** + * Return a map representation of jvm information, Spark properties, system properties, and + * class paths. Map keys define the category, and map values represent the corresponding + * attributes as a sequence of KV pairs. This is used mainly for SparkListenerEnvironmentUpdate. + */ + private[spark] + def environmentDetails( + conf: SparkConf, + schedulingMode: String, + addedJars: Seq[String], + addedFiles: Seq[String]): Map[String, Seq[(String, String)]] = { + + val jvmInformation = Seq( + ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), + ("Java Home", Properties.javaHome), + ("Scala Version", Properties.versionString), + ("Scala Home", Properties.scalaHome) + ).sorted + + // Spark properties + // This includes the scheduling mode whether or not it is configured (used by SparkUI) + val schedulerMode = + if (!conf.contains("spark.scheduler.mode")) { + Seq(("spark.scheduler.mode", schedulingMode)) + } else { + Seq[(String, String)]() + } + val sparkProperties = (conf.getAll ++ schedulerMode).sorted + + // System properties that are not java classpaths + val systemProperties = System.getProperties.iterator.toSeq + val otherProperties = systemProperties.filter { case (k, v) => + k != "java.class.path" && !k.startsWith("spark.") + }.sorted + + // Class paths including all added jars and files + val classPathProperty = systemProperties.find { case (k, v) => + k == "java.class.path" + }.getOrElse(("", "")) + val classPathEntries = classPathProperty._2 + .split(conf.get("path.separator", ":")) + .filterNot(e => e.isEmpty) + .map(e => (e, "System Classpath")) + val addedJarsAndFiles = (addedJars ++ addedFiles).map((_, "Added By User")) + val classPaths = (addedJarsAndFiles ++ classPathEntries).sorted + + Map[String, Seq[(String, String)]]( + "JVM Information" -> jvmInformation, + "Spark Properties" -> sparkProperties, + "System Properties" -> otherProperties, + "Classpath Entries" -> classPaths) } } diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index 3fd6f5eb472f4..f1a753b6ab8a9 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -29,7 +29,7 @@ private[spark] sealed trait TaskEndReason private[spark] case object Success extends TaskEndReason -private[spark] +private[spark] case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it private[spark] case class FetchFailed( @@ -65,4 +65,3 @@ private[spark] case object ExecutorLostFailure extends TaskEndReason * deserializing the task result. */ private[spark] case object UnknownReason extends TaskEndReason - diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 8e0eab56a3dcf..35508b6e5acba 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -434,6 +434,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * Clear the job's list of JARs added by `addJar` so that they do not get downloaded to * any new nodes. */ + @deprecated("adding jars no longer creates local copies that need to be deleted", "1.0.0") def clearJars() { sc.clearJars() } @@ -442,6 +443,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * Clear the job's list of files added by `addFile` so that they do not get downloaded to * any new nodes. */ + @deprecated("adding files no longer creates local copies that need to be deleted", "1.0.0") def clearFiles() { sc.clearFiles() } diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 449b953530ff9..15fa8a7679874 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,13 +17,16 @@ package org.apache.spark.deploy +import org.apache.spark.scheduler.EventLoggingInfo + private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], val memoryPerSlave: Int, val command: Command, val sparkHome: Option[String], - val appUiUrl: String) + var appUiUrl: String, + val eventLogInfo: Option[EventLoggingInfo] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala index cefb1ff97e83c..c4f5e294a393e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/deploy/JsonProtocol.scala @@ -43,7 +43,6 @@ private[spark] object JsonProtocol { ("starttime" -> obj.startTime) ~ ("id" -> obj.id) ~ ("name" -> obj.desc.name) ~ - ("appuiurl" -> obj.appUiUrl) ~ ("cores" -> obj.desc.maxCores) ~ ("user" -> obj.desc.user) ~ ("memoryperslave" -> obj.desc.memoryPerSlave) ~ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala index e8867bc1691d3..46b9f4dc7d3ba 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ApplicationInfo.scala @@ -31,7 +31,6 @@ private[spark] class ApplicationInfo( val desc: ApplicationDescription, val submitDate: Date, val driver: ActorRef, - val appUiUrl: String, defaultCores: Int) extends Serializable { @@ -45,11 +44,6 @@ private[spark] class ApplicationInfo( init() - private def readObject(in: java.io.ObjectInputStream) : Unit = { - in.defaultReadObject() - init() - } - private def init() { state = ApplicationState.WAITING executors = new mutable.HashMap[Int, ExecutorInfo] diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index b8dfa44102583..1fd211416976e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -37,10 +37,16 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem +import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} -private[spark] class Master(host: String, port: Int, webUiPort: Int, +private[spark] class Master( + host: String, + port: Int, + webUiPort: Int, val securityMgr: SecurityManager) extends Actor with Logging { + import context.dispatcher // to use Akka's scheduler.schedule() val conf = new SparkConf @@ -64,6 +70,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, val completedApps = new ArrayBuffer[ApplicationInfo] var nextAppNumber = 0 + val appIdToUI = new HashMap[String, SparkUI] + val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] val waitingDrivers = new ArrayBuffer[DriverInfo] // Drivers currently spooled for scheduling @@ -107,8 +115,8 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, logInfo("Starting Spark master at " + masterUrl) // Listen for remote client disconnection events, since they don't go through Akka's watch() context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - webUi.start() - masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort.get + webUi.bind() + masterWebUiUrl = "http://" + masterPublicAddress + ":" + webUi.boundPort context.system.scheduler.schedule(0 millis, WORKER_TIMEOUT millis, self, CheckForWorkerTimeOut) masterMetricsSystem.registerSource(masterSource) @@ -141,6 +149,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, override def postStop() { webUi.stop() + appIdToUI.values.foreach(_.stop()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -373,7 +382,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, } case RequestWebUIPort => { - sender ! WebUIPortResponse(webUi.boundPort.getOrElse(-1)) + sender ! WebUIPortResponse(webUi.boundPort) } } @@ -581,8 +590,7 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, def createApplication(desc: ApplicationDescription, driver: ActorRef): ApplicationInfo = { val now = System.currentTimeMillis() val date = new Date(now) - new ApplicationInfo( - now, newApplicationId(date), desc, date, driver, desc.appUiUrl, defaultCores) + new ApplicationInfo(now, newApplicationId(date), desc, date, driver, defaultCores) } def registerApplication(app: ApplicationInfo): Unit = { @@ -614,12 +622,27 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach( a => { + appIdToUI.remove(a.id).foreach { ui => + ui.stop() + webUi.detachUI(ui) + } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) } completedApps += app // Remember it in our history waitingApps -= app + + // If application events are logged, use them to rebuild the UI + startPersistedSparkUI(app).map { ui => + app.desc.appUiUrl = ui.basePath + appIdToUI(app.id) = ui + webUi.attachUI(ui) + }.getOrElse { + // Avoid broken links if the UI is not reconstructed + app.desc.appUiUrl = "" + } + for (exec <- app.executors.values) { exec.worker.removeExecutor(exec) exec.worker.actor ! KillExecutor(masterUrl, exec.application.id, exec.id) @@ -634,6 +657,36 @@ private[spark] class Master(host: String, port: Int, webUiPort: Int, } } + /** + * Start a new SparkUI rendered from persisted storage. If this is unsuccessful for any reason, + * return None. Otherwise return the reconstructed UI. + */ + def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { + val appName = app.desc.name + val eventLogInfo = app.desc.eventLogInfo.getOrElse { return None } + val eventLogDir = eventLogInfo.logDir + val eventCompressionCodec = eventLogInfo.compressionCodec + val appConf = new SparkConf + eventCompressionCodec.foreach { codec => + appConf.set("spark.eventLog.compress", "true") + appConf.set("spark.io.compression.codec", codec) + } + val replayerBus = new ReplayListenerBus(appConf) + val ui = new SparkUI( + appConf, + replayerBus, + "%s (finished)".format(appName), + "/history/%s".format(app.id)) + + // Do not call ui.bind() to avoid creating a new server for each application + ui.start() + val success = replayerBus.replay(eventLogDir) + if (!success) { + ui.stop() + None + } else Some(ui) + } + /** Generate a new app ID given a app's submission date */ def newApplicationId(submitDate: Date): String = { val appId = "app-%s-%04d".format(DATE_FORMAT.format(submitDate), nextAppNumber) @@ -717,9 +770,11 @@ private[spark] object Master { } } - def startSystemAndActor(host: String, port: Int, webUiPort: Int, conf: SparkConf) - : (ActorSystem, Int, Int) = - { + def startSystemAndActor( + host: String, + port: Int, + webUiPort: Int, + conf: SparkConf): (ActorSystem, Int, Int) = { val securityMgr = new SecurityManager(conf) val (actorSystem, boundPort) = AkkaUtils.createActorSystem(systemName, host, port, conf = conf, securityManager = securityMgr) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index 90cad3c37fda6..cb092cb5d576b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -23,7 +23,6 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import javax.servlet.http.HttpServletRequest import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol @@ -83,7 +82,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) {
  • Submit Date: {app.submitDate}
  • State: {app.state}
  • -
  • Application Detail UI
  • +
  • Application Detail UI
  • diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 3233cd97f7bd0..7ec71eb80bfc0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -23,7 +23,6 @@ import scala.concurrent.Await import scala.xml.Node import akka.pattern.ask -import javax.servlet.http.HttpServletRequest import org.json4s.JValue import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} @@ -162,7 +161,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { {app.id} - {app.desc.name} + {app.desc.name} {app.coresGranted} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 4ad1f95be31c9..bd75b2dfd0e07 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -18,12 +18,12 @@ package org.apache.spark.deploy.master.ui import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Server + import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.JettyUtils +import org.apache.spark.ui.{ServerInfo, SparkUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -32,24 +32,35 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { - val timeout = AkkaUtils.askTimeout(master.conf) - val host = Utils.localHostName() - val port = requestedPort - val masterActorRef = master.self + val timeout = AkkaUtils.askTimeout(master.conf) - var server: Option[Server] = None - var boundPort: Option[Int] = None + private val host = Utils.localHostName() + private val port = requestedPort + private val applicationPage = new ApplicationPage(this) + private val indexPage = new IndexPage(this) + private var serverInfo: Option[ServerInfo] = None - val applicationPage = new ApplicationPage(this) - val indexPage = new IndexPage(this) + private val handlers: Seq[ServletContextHandler] = { + master.masterMetricsSystem.getServletHandlers ++ + master.applicationMetricsSystem.getServletHandlers ++ + Seq[ServletContextHandler]( + createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"), + createServletHandler("/app/json", + (request: HttpServletRequest) => applicationPage.renderJson(request), master.securityMgr), + createServletHandler("/app", + (request: HttpServletRequest) => applicationPage.render(request), master.securityMgr), + createServletHandler("/json", + (request: HttpServletRequest) => indexPage.renderJson(request), master.securityMgr), + createServletHandler("/", + (request: HttpServletRequest) => indexPage.render(request), master.securityMgr) + ) + } - def start() { + def bind() { try { - val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers, master.conf) - server = Some(srv) - boundPort = Some(bPort) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort.get)) + serverInfo = Some(startJettyServer(host, port, handlers, master.conf)) + logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create Master JettyUtils", e) @@ -57,27 +68,38 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { } } - val metricsHandlers = master.masterMetricsSystem.getServletHandlers ++ - master.applicationMetricsSystem.getServletHandlers + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - val handlers = metricsHandlers ++ Seq[ServletContextHandler]( - createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR + "/static", "/static"), - createServletHandler("/app/json", - createServlet((request: HttpServletRequest) => applicationPage.renderJson(request), - master.securityMgr)), - createServletHandler("/app", createServlet((request: HttpServletRequest) => applicationPage - .render(request), master.securityMgr)), - createServletHandler("/json", createServlet((request: HttpServletRequest) => indexPage - .renderJson(request), master.securityMgr)), - createServletHandler("*", createServlet((request: HttpServletRequest) => indexPage.render - (request), master.securityMgr)) - ) + /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ + def attachUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } + + /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ + def detachUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + if (handler.isStarted) { + handler.stop() + } + rootHandler.removeHandler(handler) + } + } def stop() { - server.foreach(_.stop()) + assert(serverInfo.isDefined, "Attempted to stop a Master UI that was not bound to a server!") + serverInfo.get.server.stop() } } private[spark] object MasterWebUI { - val STATIC_RESOURCE_DIR = "org/apache/spark/ui" + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index afaabedffefea..5e0fc31fff22f 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -124,7 +124,7 @@ private[spark] class Worker( createWorkDir() webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) - webUi.start() + webUi.bind() registerWithMaster() metricsSystem.registerSource(workerSource) @@ -150,8 +150,7 @@ private[spark] class Worker( for (masterUrl <- masterUrls) { logInfo("Connecting to master " + masterUrl + "...") val actor = context.actorSelection(Master.toAkkaUrl(masterUrl)) - actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort.get, - publicAddress) + actor ! RegisterWorker(workerId, host, port, cores, memory, webUi.boundPort, publicAddress) } } @@ -340,10 +339,15 @@ private[spark] object Worker { actorSystem.awaitTermination() } - def startSystemAndActor(host: String, port: Int, webUiPort: Int, cores: Int, memory: Int, - masterUrls: Array[String], workDir: String, workerNumber: Option[Int] = None) - : (ActorSystem, Int) = - { + def startSystemAndActor( + host: String, + port: Int, + webUiPort: Int, + cores: Int, + memory: Int, + masterUrls: Array[String], + workDir: String, workerNumber: Option[Int] = None): (ActorSystem, Int) = { + // The LocalSparkCluster runs multiple local sparkWorkerX actor systems val conf = new SparkConf val systemName = "sparkWorker" + workerNumber.map(_.toString).getOrElse("") diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 4e33b330ad4e7..de76a5d5eb7bc 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -19,12 +19,12 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.server.Server + import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{JettyUtils, UIUtils} +import org.apache.spark.ui.{JettyUtils, ServerInfo, SparkUI, UIUtils} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -33,37 +33,35 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends Logging { - val timeout = AkkaUtils.askTimeout(worker.conf) - val host = Utils.localHostName() - val port = requestedPort.getOrElse( - worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) - - var server: Option[Server] = None - var boundPort: Option[Int] = None - - val indexPage = new IndexPage(this) + extends Logging { - val metricsHandlers = worker.metricsSystem.getServletHandlers + val timeout = AkkaUtils.askTimeout(worker.conf) - val handlers = metricsHandlers ++ Seq[ServletContextHandler]( - createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE + "/static", "/static"), - createServletHandler("/log", createServlet((request: HttpServletRequest) => log(request), - worker.securityMgr)), - createServletHandler("/logPage", createServlet((request: HttpServletRequest) => logPage - (request), worker.securityMgr)), - createServletHandler("/json", createServlet((request: HttpServletRequest) => indexPage - .renderJson(request), worker.securityMgr)), - createServletHandler("*", createServlet((request: HttpServletRequest) => indexPage.render - (request), worker.securityMgr)) - ) + private val host = Utils.localHostName() + private val port = requestedPort.getOrElse( + worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + private val indexPage = new IndexPage(this) + private var serverInfo: Option[ServerInfo] = None + + private val handlers: Seq[ServletContextHandler] = { + worker.metricsSystem.getServletHandlers ++ + Seq[ServletContextHandler]( + createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static"), + createServletHandler("/log", + (request: HttpServletRequest) => log(request), worker.securityMgr), + createServletHandler("/logPage", + (request: HttpServletRequest) => logPage(request), worker.securityMgr), + createServletHandler("/json", + (request: HttpServletRequest) => indexPage.renderJson(request), worker.securityMgr), + createServletHandler("/", + (request: HttpServletRequest) => indexPage.render(request), worker.securityMgr) + ) + } - def start() { + def bind() { try { - val (srv, bPort) = JettyUtils.startJettyServer(host, port, handlers, worker.conf) - server = Some(srv) - boundPort = Some(bPort) - logInfo("Started Worker web UI at http://%s:%d".format(host, bPort)) + serverInfo = Some(JettyUtils.startJettyServer(host, port, handlers, worker.conf)) + logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create Worker JettyUtils", e) @@ -71,7 +69,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } } - def log(request: HttpServletRequest): String = { + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + + private def log(request: HttpServletRequest): String = { val defaultBytes = 100 * 1024 val appId = Option(request.getParameter("appId")) @@ -98,7 +98,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I pre + Utils.offsetBytes(path, startByte, endByte) } - def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { + private def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { val defaultBytes = 100 * 1024 val appId = Option(request.getParameter("appId")) val executorId = Option(request.getParameter("executorId")) @@ -119,17 +119,14 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I val (startByte, endByte) = getByteRange(path, offset, byteLength) val file = new File(path) val logLength = file.length - val logText = {Utils.offsetBytes(path, startByte, endByte)} - val linkToMaster =

    Back to Master

    - val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} val backButton = if (startByte > 0) { + .format(params, logType, math.max(startByte - byteLength, 0), byteLength)}> @@ -146,7 +143,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } @@ -175,33 +172,28 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I } /** Determine the byte range for a log or log page. */ - def getByteRange(path: String, offset: Option[Long], byteLength: Int) - : (Long, Long) = { + private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { val defaultBytes = 100 * 1024 val maxBytes = 1024 * 1024 - val file = new File(path) val logLength = file.length() - val getOffset = offset.getOrElse(logLength-defaultBytes) - + val getOffset = offset.getOrElse(logLength - defaultBytes) val startByte = if (getOffset < 0) 0L else if (getOffset > logLength) logLength else getOffset - val logPageLength = math.min(byteLength, maxBytes) - val endByte = math.min(startByte + logPageLength, logLength) - (startByte, endByte) } def stop() { - server.foreach(_.stop()) + assert(serverInfo.isDefined, "Attempted to stop a Worker UI that was not bound to a server!") + serverInfo.get.server.stop() } } private[spark] object WorkerWebUI { - val STATIC_RESOURCE_BASE = "org/apache/spark/ui" + val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 2ea2ec29f59f5..8fe9b848ba145 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -224,10 +224,10 @@ private[spark] class Executor( for (m <- task.metrics) { m.hostname = Utils.localHostName() - m.executorDeserializeTime = (taskStart - startTime).toInt - m.executorRunTime = (taskFinish - taskStart).toInt + m.executorDeserializeTime = taskStart - startTime + m.executorRunTime = taskFinish - taskStart m.jvmGCTime = gcTime - startGCTime - m.resultSerializationTime = (afterSerialization - beforeSerialization).toInt + m.resultSerializationTime = afterSerialization - beforeSerialization } val accumUpdates = Accumulators.values @@ -263,7 +263,7 @@ private[spark] class Executor( } case t: Throwable => { - val serviceTime = (System.currentTimeMillis() - taskStart).toInt + val serviceTime = System.currentTimeMillis() - taskStart val metrics = attemptedTask.flatMap(t => t.metrics) for (m <- metrics) { m.executorRunTime = serviceTime diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 760458cb02a9b..88625e79a5c68 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,21 +17,23 @@ package org.apache.spark.executor +import org.apache.spark.storage.{BlockId, BlockStatus} + class TaskMetrics extends Serializable { /** - * Host's name the task runs on + * Host's name the task runs on */ var hostname: String = _ /** * Time taken on the executor to deserialize this task */ - var executorDeserializeTime: Int = _ + var executorDeserializeTime: Long = _ /** * Time the executor spends actually running the task (including fetching shuffle data) */ - var executorRunTime: Int = _ + var executorRunTime: Long = _ /** * The number of bytes this task transmitted back to the driver as the TaskResult @@ -68,6 +70,11 @@ class TaskMetrics extends Serializable { * here */ var shuffleWriteMetrics: Option[ShuffleWriteMetrics] = None + + /** + * Storage statuses of any blocks that have been updated as a result of this task. + */ + var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } object TaskMetrics { diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 848b5c439bb5b..059e58824c39b 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -38,8 +38,7 @@ trait CompressionCodec { private[spark] object CompressionCodec { def createCodec(conf: SparkConf): CompressionCodec = { - createCodec(conf, conf.get( - "spark.io.compression.codec", classOf[LZFCompressionCodec].getName)) + createCodec(conf, conf.get("spark.io.compression.codec", DEFAULT_COMPRESSION_CODEC)) } def createCodec(conf: SparkConf, codecName: String): CompressionCodec = { @@ -47,6 +46,8 @@ private[spark] object CompressionCodec { .getConstructor(classOf[SparkConf]) ctor.newInstance(conf).asInstanceOf[CompressionCodec] } + + val DEFAULT_COMPRESSION_CODEC = classOf[LZFCompressionCodec].getName } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 3110eccdee4fc..854b52c510e3d 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -28,7 +28,7 @@ import com.fasterxml.jackson.databind.ObjectMapper import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SecurityManager -import org.apache.spark.ui.JettyUtils +import org.apache.spark.ui.JettyUtils._ class MetricsServlet(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { @@ -46,10 +46,8 @@ class MetricsServlet(val property: Properties, val registry: MetricRegistry, new MetricsModule(TimeUnit.SECONDS, TimeUnit.MILLISECONDS, servletShowSample)) def getHandlers = Array[ServletContextHandler]( - JettyUtils.createServletHandler(servletPath, - JettyUtils.createServlet( - new JettyUtils.ServletParams(request => getMetricsSnapshot(request), "text/json"), - securityMgr) ) + createServletHandler(servletPath, + new ServletParams(request => getMetricsSnapshot(request), "text/json"), securityMgr) ) def getMetricsSnapshot(request: HttpServletRequest): String = { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index ddb901246d360..1b43040c6d918 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -137,9 +137,8 @@ abstract class RDD[T: ClassTag]( throw new UnsupportedOperationException( "Cannot change storage level of an RDD after it was already assigned a level") } + sc.persistRDD(this) storageLevel = newLevel - // Register the RDD with the SparkContext - sc.persistentRdds(id) = this this } @@ -157,8 +156,7 @@ abstract class RDD[T: ClassTag]( */ def unpersist(blocking: Boolean = true): RDD[T] = { logInfo("Removing RDD " + id + " from persistence list") - sc.env.blockManager.master.removeRdd(id, blocking) - sc.persistentRdds.remove(id) + sc.unpersistRDD(this, blocking) storageLevel = StorageLevel.NONE this } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index d83d0341c61ab..77c558ac46f6f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -32,7 +32,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -54,87 +54,53 @@ import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedH */ private[spark] class DAGScheduler( - taskSched: TaskScheduler, + taskScheduler: TaskScheduler, + listenerBus: LiveListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, env: SparkEnv) extends Logging { - def this(taskSched: TaskScheduler) { - this(taskSched, SparkEnv.get.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], - SparkEnv.get.blockManager.master, SparkEnv.get) - } - taskSched.setDAGScheduler(this) + import DAGScheduler._ - // Called by TaskScheduler to report task's starting. - def taskStarted(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! BeginEvent(task, taskInfo) - } - - // Called to report that a task has completed and results are being fetched remotely. - def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! GettingResultEvent(task, taskInfo) + def this(sc: SparkContext, taskScheduler: TaskScheduler) = { + this( + taskScheduler, + sc.listenerBus, + sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], + sc.env.blockManager.master, + sc.env) } - // Called by TaskScheduler to report task completions or failures. - def taskEnded( - task: Task[_], - reason: TaskEndReason, - result: Any, - accumUpdates: Map[Long, Any], - taskInfo: TaskInfo, - taskMetrics: TaskMetrics) { - eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics) - } - - // Called by TaskScheduler when an executor fails. - def executorLost(execId: String) { - eventProcessActor ! ExecutorLost(execId) - } - - // Called by TaskScheduler when a host is added - def executorGained(execId: String, host: String) { - eventProcessActor ! ExecutorGained(execId, host) - } - - // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or - // cancellation of the job itself. - def taskSetFailed(taskSet: TaskSet, reason: String) { - eventProcessActor ! TaskSetFailed(taskSet, reason) - } - - // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; - // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one - // as more failure events come in - val RESUBMIT_TIMEOUT = 200.milliseconds - - // The time, in millis, to wake up between polls of the completion queue in order to potentially - // resubmit failed stages - val POLL_TIMEOUT = 10L - - // Warns the user if a stage contains a task with size greater than this value (in KB) - val TASK_SIZE_TO_WARN = 100 + def this(sc: SparkContext) = this(sc, sc.taskScheduler) private var eventProcessActor: ActorRef = _ private[scheduler] val nextJobId = new AtomicInteger(0) - - def numTotalJobs: Int = nextJobId.get() - + private[scheduler] def numTotalJobs: Int = nextJobId.get() private val nextStageId = new AtomicInteger(0) private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage] - private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] - + private[scheduler] val stageIdToActiveJob = new HashMap[Int, ActiveJob] + private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] - // An async scheduler event bus. The bus should be stopped when DAGSCheduler is stopped. - private[spark] val listenerBus = new SparkListenerBus + // Stages we need to run whose parents aren't done + private[scheduler] val waitingStages = new HashSet[Stage] + + // Stages we are running right now + private[scheduler] val runningStages = new HashSet[Stage] + + // Stages that must be resubmitted due to fetch failures + private[scheduler] val failedStages = new HashSet[Stage] + + // Missing tasks from each stage + private[scheduler] val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] + + private[scheduler] val activeJobs = new HashSet[ActiveJob] // Contains the locations that each RDD's partitions are cached on private val cacheLocs = new HashMap[Int, Array[Seq[TaskLocation]]] @@ -145,22 +111,12 @@ class DAGScheduler( // // TODO: Garbage collect information about failure epochs when we know there are no more // stray messages to detect. - val failedEpoch = new HashMap[String, Long] + private val failedEpoch = new HashMap[String, Long] - // stage id to the active job - val idToActiveJob = new HashMap[Int, ActiveJob] + private val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) - val waiting = new HashSet[Stage] // Stages we need to run whose parents aren't done - val running = new HashSet[Stage] // Stages we are running right now - val failed = new HashSet[Stage] // Stages that must be resubmitted due to fetch failures - // Missing tasks from each stage - val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] - - val activeJobs = new HashSet[ActiveJob] - val resultStageToJob = new HashMap[Stage, ActiveJob] - - val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) + taskScheduler.setDAGScheduler(this) /** * Starts the event processing actor. The actor has two responsibilities: @@ -196,13 +152,46 @@ class DAGScheduler( })) } - def addSparkListener(listener: SparkListener) { - listenerBus.addListener(listener) + // Called by TaskScheduler to report task's starting. + def taskStarted(task: Task[_], taskInfo: TaskInfo) { + eventProcessActor ! BeginEvent(task, taskInfo) + } + + // Called to report that a task has completed and results are being fetched remotely. + def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { + eventProcessActor ! GettingResultEvent(task, taskInfo) + } + + // Called by TaskScheduler to report task completions or failures. + def taskEnded( + task: Task[_], + reason: TaskEndReason, + result: Any, + accumUpdates: Map[Long, Any], + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) { + eventProcessActor ! CompletionEvent(task, reason, result, accumUpdates, taskInfo, taskMetrics) + } + + // Called by TaskScheduler when an executor fails. + def executorLost(execId: String) { + eventProcessActor ! ExecutorLost(execId) + } + + // Called by TaskScheduler when a host is added + def executorAdded(execId: String, host: String) { + eventProcessActor ! ExecutorAdded(execId, host) + } + + // Called by TaskScheduler to cancel an entire TaskSet due to either repeated failures or + // cancellation of the job itself. + def taskSetFailed(taskSet: TaskSet, reason: String) { + eventProcessActor ! TaskSetFailed(taskSet, reason) } private def getCacheLocs(rdd: RDD[_]): Array[Seq[TaskLocation]] = { if (!cacheLocs.contains(rdd.id)) { - val blockIds = rdd.partitions.indices.map(index=> RDDBlockId(rdd.id, index)).toArray[BlockId] + val blockIds = rdd.partitions.indices.map(index => RDDBlockId(rdd.id, index)).toArray[BlockId] val locs = BlockManager.blockIdsToBlockManagers(blockIds, env, blockManagerMaster) cacheLocs(rdd.id) = blockIds.map { id => locs.getOrElse(id, Nil).map(bm => TaskLocation(bm.host, bm.executorId)) @@ -250,7 +239,7 @@ class DAGScheduler( new Stage(id, rdd, numTasks, shuffleDep, getParentStages(rdd, jobId), jobId, callSite) stageIdToStage(id) = stage updateJobIdStageIdMaps(jobId, stage) - stageToInfos(stage) = new StageInfo(stage) + stageToInfos(stage) = StageInfo.fromStage(stage) stage } @@ -376,9 +365,9 @@ class DAGScheduler( def removeStage(stageId: Int) { // data structures based on Stage for (stage <- stageIdToStage.get(stageId)) { - if (running.contains(stage)) { + if (runningStages.contains(stage)) { logDebug("Removing running stage %d".format(stageId)) - running -= stage + runningStages -= stage } stageToInfos -= stage for ((k, v) <- shuffleToMapStage.find(_._2 == stage)) { @@ -388,13 +377,13 @@ class DAGScheduler( logDebug("Removing pending status for stage %d".format(stageId)) } pendingTasks -= stage - if (waiting.contains(stage)) { + if (waitingStages.contains(stage)) { logDebug("Removing stage %d from waiting set.".format(stageId)) - waiting -= stage + waitingStages -= stage } - if (failed.contains(stage)) { + if (failedStages.contains(stage)) { logDebug("Removing stage %d from failed set.".format(stageId)) - failed -= stage + failedStages -= stage } } // data structures based on StageId @@ -544,13 +533,14 @@ class DAGScheduler( logInfo("Missing parents: " + getMissingParentStages(finalStage)) if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job, Array(), properties)) + listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) runLocally(job) } else { - idToActiveJob(jobId) = job + stageIdToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job - listenerBus.post(SparkListenerJobStart(job, jobIdToStageIds(jobId).toArray, properties)) + listenerBus.post( + SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) submitStage(finalStage) } @@ -563,23 +553,23 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(activeJob => groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach { handleJobCancellation } + jobIds.foreach(handleJobCancellation) case AllJobsCancelled => // Cancel all running jobs. - running.map(_.jobId).foreach { handleJobCancellation } + runningStages.map(_.jobId).foreach(handleJobCancellation) activeJobs.clear() // These should already be empty by this point, - idToActiveJob.clear() // but just in case we lost track of some jobs... + stageIdToActiveJob.clear() // but just in case we lost track of some jobs... - case ExecutorGained(execId, host) => - handleExecutorGained(execId, host) + case ExecutorAdded(execId, host) => + handleExecutorAdded(execId, host) case ExecutorLost(execId) => handleExecutorLost(execId) case BeginEvent(task, taskInfo) => for ( - job <- idToActiveJob.get(task.stageId); + job <- stageIdToActiveJob.get(task.stageId); stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage) ) { @@ -591,20 +581,22 @@ class DAGScheduler( task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) } } - listenerBus.post(SparkListenerTaskStart(task, taskInfo)) + listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) case GettingResultEvent(task, taskInfo) => - listenerBus.post(SparkListenerTaskGettingResult(task, taskInfo)) + listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => - listenerBus.post(SparkListenerTaskEnd(task, reason, taskInfo, taskMetrics)) + val stageId = task.stageId + val taskType = Utils.getFormattedClassName(task) + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) handleTaskCompletion(completion) case TaskSetFailed(taskSet, reason) => stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } case ResubmitFailedStages => - if (failed.size > 0) { + if (failedStages.size > 0) { // Failed stages may be removed by job cancellation, so failed might be empty even if // the ResubmitFailedStages event has been scheduled. resubmitFailedStages() @@ -615,7 +607,7 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, None))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) } return true } @@ -629,9 +621,9 @@ class DAGScheduler( private[scheduler] def resubmitFailedStages() { logInfo("Resubmitting failed stages") clearCacheLocs() - val failed2 = failed.toArray - failed.clear() - for (stage <- failed2.sortBy(_.jobId)) { + val failedStagesCopy = failedStages.toArray + failedStages.clear() + for (stage <- failedStagesCopy.sortBy(_.jobId)) { submitStage(stage) } } @@ -644,12 +636,12 @@ class DAGScheduler( // TODO: We might want to run this less often, when we are sure that something has become // runnable that wasn't before. logTrace("Checking for newly runnable parent stages") - logTrace("running: " + running) - logTrace("waiting: " + waiting) - logTrace("failed: " + failed) - val waiting2 = waiting.toArray - waiting.clear() - for (stage <- waiting2.sortBy(_.jobId)) { + logTrace("running: " + runningStages) + logTrace("waiting: " + waitingStages) + logTrace("failed: " + failedStages) + val waitingStagesCopy = waitingStages.toArray + waitingStages.clear() + for (stage <- waitingStagesCopy.sortBy(_.jobId)) { submitStage(stage) } } @@ -685,7 +677,7 @@ class DAGScheduler( } } catch { case e: Exception => - jobResult = JobFailed(e, Some(job.finalStage)) + jobResult = JobFailed(e, job.finalStage.id) job.listener.jobFailed(e) } finally { val s = job.finalStage @@ -693,7 +685,7 @@ class DAGScheduler( stageIdToStage -= s.id // but that won't get cleaned up via the normal paths through stageToInfos -= s // completion events or stage abort jobIdToStageIds -= job.jobId - listenerBus.post(SparkListenerJobEnd(job, jobResult)) + listenerBus.post(SparkListenerJobEnd(job.jobId, jobResult)) } } @@ -705,7 +697,7 @@ class DAGScheduler( private def activeJobForStage(stage: Stage): Option[Int] = { if (stageIdToJobIds.contains(stage.id)) { val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted - jobsThatUseStage.find(idToActiveJob.contains(_)) + jobsThatUseStage.find(stageIdToActiveJob.contains) } else { None } @@ -716,18 +708,18 @@ class DAGScheduler( val jobId = activeJobForStage(stage) if (jobId.isDefined) { logDebug("submitStage(" + stage + ")") - if (!waiting(stage) && !running(stage) && !failed(stage)) { + if (!waitingStages(stage) && !runningStages(stage) && !failedStages(stage)) { val missing = getMissingParentStages(stage).sortBy(_.id) logDebug("missing: " + missing) if (missing == Nil) { logInfo("Submitting " + stage + " (" + stage.rdd + "), which has no missing parents") submitMissingTasks(stage, jobId.get) - running += stage + runningStages += stage } else { for (parent <- missing) { submitStage(parent) } - waiting += stage + waitingStages += stage } } } else { @@ -758,8 +750,8 @@ class DAGScheduler( } } - val properties = if (idToActiveJob.contains(jobId)) { - idToActiveJob(stage.jobId).properties + val properties = if (stageIdToActiveJob.contains(jobId)) { + stageIdToActiveJob(stage.jobId).properties } else { //this stage will be assigned to "default" pool null @@ -779,20 +771,20 @@ class DAGScheduler( } catch { case e: NotSerializableException => abortStage(stage, "Task not serializable: " + e.toString) - running -= stage + runningStages -= stage return } logInfo("Submitting " + tasks.size + " missing tasks from " + stage + " (" + stage.rdd + ")") myPending ++= tasks logDebug("New pending tasks: " + myPending) - taskSched.submitTasks( + taskScheduler.submitTasks( new TaskSet(tasks.toArray, stage.id, stage.newAttemptId(), stage.jobId, properties)) stageToInfos(stage).submissionTime = Some(System.currentTimeMillis()) } else { logDebug("Stage " + stage + " is actually done; %b %d %d".format( stage.isAvailable, stage.numAvailableOutputs, stage.numPartitions)) - running -= stage + runningStages -= stage } } @@ -817,7 +809,7 @@ class DAGScheduler( logInfo("%s (%s) finished in %s s".format(stage, stage.name, serviceTime)) stageToInfos(stage).completionTime = Some(System.currentTimeMillis()) listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) - running -= stage + runningStages -= stage } event.reason match { case Success => @@ -826,7 +818,6 @@ class DAGScheduler( Accumulators.add(event.accumUpdates) // TODO: do this only if task wasn't resubmitted } pendingTasks(stage) -= task - stageToInfos(stage).taskInfos += event.taskInfo -> event.taskMetrics task match { case rt: ResultTask[_, _] => resultStageToJob.get(stage) match { @@ -836,12 +827,12 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - idToActiveJob -= stage.jobId + stageIdToActiveJob -= stage.jobId activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) jobIdToStageIdsRemove(job.jobId) - listenerBus.post(SparkListenerJobEnd(job, JobSucceeded)) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) } @@ -858,12 +849,12 @@ class DAGScheduler( } else { stage.addOutputLoc(smt.partitionId, status) } - if (running.contains(stage) && pendingTasks(stage).isEmpty) { + if (runningStages.contains(stage) && pendingTasks(stage).isEmpty) { markStageAsFinished(stage) logInfo("looking for newly runnable stages") - logInfo("running: " + running) - logInfo("waiting: " + waiting) - logInfo("failed: " + failed) + logInfo("running: " + runningStages) + logInfo("waiting: " + waitingStages) + logInfo("failed: " + failedStages) if (stage.shuffleDep.isDefined) { // We supply true to increment the epoch number here in case this is a // recomputation of the map outputs. In that case, some nodes may have cached @@ -886,14 +877,14 @@ class DAGScheduler( submitStage(stage) } else { val newlyRunnable = new ArrayBuffer[Stage] - for (stage <- waiting) { + for (stage <- waitingStages) { logInfo("Missing parents for " + stage + ": " + getMissingParentStages(stage)) } - for (stage <- waiting if getMissingParentStages(stage) == Nil) { + for (stage <- waitingStages if getMissingParentStages(stage) == Nil) { newlyRunnable += stage } - waiting --= newlyRunnable - running ++= newlyRunnable + waitingStages --= newlyRunnable + runningStages ++= newlyRunnable for { stage <- newlyRunnable.sortBy(_.id) jobId <- activeJobForStage(stage) @@ -912,7 +903,7 @@ class DAGScheduler( case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => // Mark the stage that the reducer was in as unrunnable val failedStage = stageIdToStage(task.stageId) - running -= failedStage + runningStages -= failedStage // TODO: Cancel running tasks in the stage logInfo("Marking " + failedStage + " (" + failedStage.name + ") for resubmision due to a fetch failure") @@ -924,7 +915,7 @@ class DAGScheduler( } logInfo("The failed fetch was from " + mapStage + " (" + mapStage.name + "); marking it for resubmission") - if (failed.isEmpty && eventProcessActor != null) { + if (failedStages.isEmpty && eventProcessActor != null) { // Don't schedule an event to resubmit failed stages if failed isn't empty, because // in that case the event will already have been scheduled. eventProcessActor may be // null during unit tests. @@ -932,8 +923,8 @@ class DAGScheduler( env.actorSystem.scheduler.scheduleOnce( RESUBMIT_TIMEOUT, eventProcessActor, ResubmitFailedStages) } - failed += failedStage - failed += mapStage + failedStages += failedStage + failedStages += mapStage // TODO: mark the executor as failed only if there were lots of fetch failures on it if (bmAddress != null) { handleExecutorLost(bmAddress.executorId, Some(task.epoch)) @@ -980,10 +971,10 @@ class DAGScheduler( } } - private def handleExecutorGained(execId: String, host: String) { + private def handleExecutorAdded(execId: String, host: String) { // remove from failedEpoch(execId) ? if (failedEpoch.contains(execId)) { - logInfo("Host gained which was in lost list earlier: " + host) + logInfo("Host added was in lost list earlier: " + host) failedEpoch -= execId } } @@ -993,14 +984,14 @@ class DAGScheduler( logDebug("Trying to cancel unregistered job " + jobId) } else { val independentStages = removeJobAndIndependentStages(jobId) - independentStages.foreach { taskSched.cancelTasks } + independentStages.foreach(taskScheduler.cancelTasks) val error = new SparkException("Job %d cancelled".format(jobId)) - val job = idToActiveJob(jobId) + val job = stageIdToActiveJob(jobId) job.listener.jobFailed(error) jobIdToStageIds -= jobId activeJobs -= job - idToActiveJob -= jobId - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(job.finalStage)))) + stageIdToActiveJob -= jobId + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) } } @@ -1020,10 +1011,10 @@ class DAGScheduler( val error = new SparkException("Job aborted: " + reason) job.listener.jobFailed(error) jobIdToStageIdsRemove(job.jobId) - idToActiveJob -= resultStage.jobId + stageIdToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage - listenerBus.post(SparkListenerJobEnd(job, JobFailed(error, Some(failedStage)))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") @@ -1102,11 +1093,11 @@ class DAGScheduler( "stageToInfos" -> stageToInfos, "jobIdToStageIds" -> jobIdToStageIds, "stageIdToJobIds" -> stageIdToJobIds). - foreach { case(s, t) => { - val sizeBefore = t.size - t.clearOldValues(cleanupTime) - logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) - }} + foreach { case (s, t) => + val sizeBefore = t.size + t.clearOldValues(cleanupTime) + logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) + } } def stop() { @@ -1114,7 +1105,20 @@ class DAGScheduler( eventProcessActor ! StopDAGScheduler } metadataCleaner.cancel() - taskSched.stop() - listenerBus.stop() + taskScheduler.stop() } } + +private[spark] object DAGScheduler { + // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; + // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one + // as more failure events come in + val RESUBMIT_TIMEOUT = 200.milliseconds + + // The time, in millis, to wake up between polls of the completion queue in order to potentially + // resubmit failed stages + val POLL_TIMEOUT = 10L + + // Warns the user if a stage contains a task with size greater than this value (in KB) + val TASK_SIZE_TO_WARN = 100 +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 39cd98e2d74e4..04c53d468465a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -65,7 +65,7 @@ private[scheduler] case class CompletionEvent( taskMetrics: TaskMetrics) extends DAGSchedulerEvent -private[scheduler] case class ExecutorGained(execId: String, host: String) extends DAGSchedulerEvent +private[scheduler] case class ExecutorAdded(execId: String, host: String) extends DAGSchedulerEvent private[scheduler] case class ExecutorLost(execId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala index b52fe2410abde..5878e733908f5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerSource.scala @@ -28,15 +28,15 @@ private[spark] class DAGSchedulerSource(val dagScheduler: DAGScheduler, sc: Spar val sourceName = "%s.DAGScheduler".format(sc.appName) metricRegistry.register(MetricRegistry.name("stage", "failedStages"), new Gauge[Int] { - override def getValue: Int = dagScheduler.failed.size + override def getValue: Int = dagScheduler.failedStages.size }) metricRegistry.register(MetricRegistry.name("stage", "runningStages"), new Gauge[Int] { - override def getValue: Int = dagScheduler.running.size + override def getValue: Int = dagScheduler.runningStages.size }) metricRegistry.register(MetricRegistry.name("stage", "waitingStages"), new Gauge[Int] { - override def getValue: Int = dagScheduler.waiting.size + override def getValue: Int = dagScheduler.waitingStages.size }) metricRegistry.register(MetricRegistry.name("job", "allJobs"), new Gauge[Int] { diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala new file mode 100644 index 0000000000000..217f8825c2ae9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -0,0 +1,98 @@ +/* + * 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.scheduler + +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, FileLogger} + +/** + * A SparkListener that logs events to persistent storage. + * + * Event logging is specified by the following configurable parameters: + * spark.eventLog.enabled - Whether event logging is enabled. + * spark.eventLog.compress - Whether to compress logged events + * spark.eventLog.overwrite - Whether to overwrite any existing files. + * spark.eventLog.dir - Path to the directory in which events are logged. + * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams + */ +private[spark] class EventLoggingListener(appName: String, conf: SparkConf) + extends SparkListener with Logging { + + private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) + private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) + private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 + private val logBaseDir = conf.get("spark.eventLog.dir", "/tmp/spark-events").stripSuffix("/") + private val name = appName.replaceAll("[ :/]", "-").toLowerCase + "-" + System.currentTimeMillis + val logDir = logBaseDir + "/" + name + + private val logger = + new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) + + // Information needed to replay the events logged by this listener later + val info = { + val compressionCodec = if (shouldCompress) { + Some(conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)) + } else None + EventLoggingInfo(logDir, compressionCodec) + } + + logInfo("Logging events to %s".format(logDir)) + + /** Log the event as JSON */ + private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { + val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) + logger.logLine(eventJson) + if (flushLogger) { + logger.flush() + } + } + + // Events that do not trigger a flush + override def onStageSubmitted(event: SparkListenerStageSubmitted) = + logEvent(event) + override def onTaskStart(event: SparkListenerTaskStart) = + logEvent(event) + override def onTaskGettingResult(event: SparkListenerTaskGettingResult) = + logEvent(event) + override def onTaskEnd(event: SparkListenerTaskEnd) = + logEvent(event) + override def onEnvironmentUpdate(event: SparkListenerEnvironmentUpdate) = + logEvent(event) + + // Events that trigger a flush + override def onStageCompleted(event: SparkListenerStageCompleted) = + logEvent(event, flushLogger = true) + override def onJobStart(event: SparkListenerJobStart) = + logEvent(event, flushLogger = true) + override def onJobEnd(event: SparkListenerJobEnd) = + logEvent(event, flushLogger = true) + override def onBlockManagerAdded(event: SparkListenerBlockManagerAdded) = + logEvent(event, flushLogger = true) + override def onBlockManagerRemoved(event: SparkListenerBlockManagerRemoved) = + logEvent(event, flushLogger = true) + override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = + logEvent(event, flushLogger = true) + + def stop() = logger.stop() +} + +// If compression is not enabled, compressionCodec is None +private[spark] case class EventLoggingInfo(logDir: String, compressionCodec: Option[String]) diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 01cbcc390c6cd..b3a67d7e17976 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -22,24 +22,25 @@ import java.text.SimpleDateFormat import java.util.{Date, Properties} import java.util.concurrent.LinkedBlockingQueue -import scala.collection.mutable.{HashMap, HashSet, ListBuffer} +import scala.collection.mutable.HashMap import org.apache.spark._ import org.apache.spark.executor.TaskMetrics -import org.apache.spark.rdd.RDD -import org.apache.spark.storage.StorageLevel /** * A logger class to record runtime information for jobs in Spark. This class outputs one log file - * for each Spark job, containing RDD graph, tasks start/stop, shuffle information. - * JobLogger is a subclass of SparkListener, use addSparkListener to add JobLogger to a SparkContext - * after the SparkContext is created. - * Note that each JobLogger only works for one SparkContext - * @param logDirName The base directory for the log files. + * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass + * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext + * is created. Note that each JobLogger only works for one SparkContext + * + * NOTE: The functionality of this class is heavily stripped down to accommodate for a general + * refactor of the SparkListener interface. In its place, the EventLoggingListener is introduced + * to log application information as SparkListenerEvents. To enable this functionality, set + * spark.eventLog.enabled to true. */ -class JobLogger(val user: String, val logDirName: String) - extends SparkListener with Logging { +@deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") +class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { def this() = this(System.getProperty("user.name", ""), String.valueOf(System.currentTimeMillis())) @@ -51,19 +52,19 @@ class JobLogger(val user: String, val logDirName: String) "/tmp/spark-%s".format(user) } - private val jobIDToPrintWriter = new HashMap[Int, PrintWriter] - private val stageIDToJobID = new HashMap[Int, Int] - private val jobIDToStages = new HashMap[Int, ListBuffer[Stage]] + private val jobIdToPrintWriter = new HashMap[Int, PrintWriter] + private val stageIdToJobId = new HashMap[Int, Int] + private val jobIdToStageIds = new HashMap[Int, Seq[Int]] private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents] + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent] createLogDir() // The following 5 functions are used only in testing. private[scheduler] def getLogDir = logDir - private[scheduler] def getJobIDtoPrintWriter = jobIDToPrintWriter - private[scheduler] def getStageIDToJobID = stageIDToJobID - private[scheduler] def getJobIDToStages = jobIDToStages + private[scheduler] def getJobIdToPrintWriter = jobIdToPrintWriter + private[scheduler] def getStageIdToJobId = stageIdToJobId + private[scheduler] def getJobIdToStageIds = jobIdToStageIds private[scheduler] def getEventQueue = eventQueue /** Create a folder for log files, the folder's name is the creation time of jobLogger */ @@ -80,187 +81,78 @@ class JobLogger(val user: String, val logDirName: String) /** * Create a log file for one job - * @param jobID ID of the job + * @param jobId ID of the job * @throws FileNotFoundException Fail to create log file */ - protected def createLogWriter(jobID: Int) { + protected def createLogWriter(jobId: Int) { try { - val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobID) - jobIDToPrintWriter += (jobID -> fileWriter) + val fileWriter = new PrintWriter(logDir + "/" + logDirName + "/" + jobId) + jobIdToPrintWriter += (jobId -> fileWriter) } catch { case e: FileNotFoundException => e.printStackTrace() } } /** - * Close log file, and clean the stage relationship in stageIDToJobID - * @param jobID ID of the job + * Close log file, and clean the stage relationship in stageIdToJobId + * @param jobId ID of the job */ - protected def closeLogWriter(jobID: Int) { - jobIDToPrintWriter.get(jobID).foreach { fileWriter => + protected def closeLogWriter(jobId: Int) { + jobIdToPrintWriter.get(jobId).foreach { fileWriter => fileWriter.close() - jobIDToStages.get(jobID).foreach(_.foreach{ stage => - stageIDToJobID -= stage.id + jobIdToStageIds.get(jobId).foreach(_.foreach { stageId => + stageIdToJobId -= stageId }) - jobIDToPrintWriter -= jobID - jobIDToStages -= jobID + jobIdToPrintWriter -= jobId + jobIdToStageIds -= jobId } } + /** + * Build up the maps that represent stage-job relationships + * @param jobId ID of the job + * @param stageIds IDs of the associated stages + */ + protected def buildJobStageDependencies(jobId: Int, stageIds: Seq[Int]) = { + jobIdToStageIds(jobId) = stageIds + stageIds.foreach { stageId => stageIdToJobId(stageId) = jobId } + } + /** * Write info into log file - * @param jobID ID of the job + * @param jobId ID of the job * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def jobLogInfo(jobID: Int, info: String, withTime: Boolean = true) { + protected def jobLogInfo(jobId: Int, info: String, withTime: Boolean = true) { var writeInfo = info if (withTime) { val date = new Date(System.currentTimeMillis()) writeInfo = DATE_FORMAT.format(date) + ": " + info } - jobIDToPrintWriter.get(jobID).foreach(_.println(writeInfo)) + jobIdToPrintWriter.get(jobId).foreach(_.println(writeInfo)) } /** * Write info into log file - * @param stageID ID of the stage + * @param stageId ID of the stage * @param info Info to be recorded * @param withTime Controls whether to record time stamp before the info, default is true */ - protected def stageLogInfo(stageID: Int, info: String, withTime: Boolean = true) { - stageIDToJobID.get(stageID).foreach(jobID => jobLogInfo(jobID, info, withTime)) - } - - /** - * Build stage dependency for a job - * @param jobID ID of the job - * @param stage Root stage of the job - */ - protected def buildJobDep(jobID: Int, stage: Stage) { - if (stage.jobId == jobID) { - jobIDToStages.get(jobID) match { - case Some(stageList) => stageList += stage - case None => val stageList = new ListBuffer[Stage] - stageList += stage - jobIDToStages += (jobID -> stageList) - } - stageIDToJobID += (stage.id -> jobID) - stage.parents.foreach(buildJobDep(jobID, _)) - } - } - - /** - * Record stage dependency and RDD dependency for a stage - * @param jobID Job ID of the stage - */ - protected def recordStageDep(jobID: Int) { - def getRddsInStage(rdd: RDD[_]): ListBuffer[RDD[_]] = { - var rddList = new ListBuffer[RDD[_]] - rddList += rdd - rdd.dependencies.foreach { - case shufDep: ShuffleDependency[_, _] => - case dep: Dependency[_] => rddList ++= getRddsInStage(dep.rdd) - } - rddList - } - jobIDToStages.get(jobID).foreach {_.foreach { stage => - var depRddDesc: String = "" - getRddsInStage(stage.rdd).foreach { rdd => - depRddDesc += rdd.id + "," - } - var depStageDesc: String = "" - stage.parents.foreach { stage => - depStageDesc += "(" + stage.id + "," + stage.shuffleDep.get.shuffleId + ")" - } - jobLogInfo(jobID, "STAGE_ID=" + stage.id + " RDD_DEP=(" + - depRddDesc.substring(0, depRddDesc.length - 1) + ")" + - " STAGE_DEP=" + depStageDesc, false) - } - } - } - - /** - * Generate indents and convert to String - * @param indent Number of indents - * @return string of indents - */ - protected def indentString(indent: Int): String = { - val sb = new StringBuilder() - for (i <- 1 to indent) { - sb.append(" ") - } - sb.toString() - } - - /** - * Get RDD's name - * @param rdd Input RDD - * @return String of RDD's name - */ - protected def getRddName(rdd: RDD[_]): String = { - var rddName = rdd.getClass.getSimpleName - if (rdd.name != null) { - rddName = rdd.name - } - rddName - } - - /** - * Record RDD dependency graph in a stage - * @param jobID Job ID of the stage - * @param rdd Root RDD of the stage - * @param indent Indent number before info - */ - protected def recordRddInStageGraph(jobID: Int, rdd: RDD[_], indent: Int) { - val cacheStr = if (rdd.getStorageLevel != StorageLevel.NONE) "CACHED" else "NONE" - val rddInfo = - s"RDD_ID=$rdd.id ${getRddName(rdd)} $cacheStr " + - s"${rdd.getCreationSite} ${rdd.creationSiteInfo.firstUserClass}" - jobLogInfo(jobID, indentString(indent) + rddInfo, false) - rdd.dependencies.foreach { - case shufDep: ShuffleDependency[_, _] => - val depInfo = "SHUFFLE_ID=" + shufDep.shuffleId - jobLogInfo(jobID, indentString(indent + 1) + depInfo, false) - case dep: Dependency[_] => recordRddInStageGraph(jobID, dep.rdd, indent + 1) - } - } - - /** - * Record stage dependency graph of a job - * @param jobID Job ID of the stage - * @param stage Root stage of the job - * @param indent Indent number before info, default is 0 - */ - protected def recordStageDepGraph(jobID: Int, stage: Stage, idSet: HashSet[Int], indent: Int = 0) - { - val stageInfo = if (stage.isShuffleMap) { - "STAGE_ID=" + stage.id + " MAP_STAGE SHUFFLE_ID=" + stage.shuffleDep.get.shuffleId - } else { - "STAGE_ID=" + stage.id + " RESULT_STAGE" - } - if (stage.jobId == jobID) { - jobLogInfo(jobID, indentString(indent) + stageInfo, false) - if (!idSet.contains(stage.id)) { - idSet += stage.id - recordRddInStageGraph(jobID, stage.rdd, indent) - stage.parents.foreach(recordStageDepGraph(jobID, _, idSet, indent + 2)) - } - } else { - jobLogInfo(jobID, indentString(indent) + stageInfo + " JOB_ID=" + stage.jobId, false) - } + protected def stageLogInfo(stageId: Int, info: String, withTime: Boolean = true) { + stageIdToJobId.get(stageId).foreach(jobId => jobLogInfo(jobId, info, withTime)) } /** * Record task metrics into job log files, including execution info and shuffle metrics - * @param stageID Stage ID of the task + * @param stageId Stage ID of the task * @param status Status info of the task * @param taskInfo Task description info * @param taskMetrics Task running metrics */ - protected def recordTaskMetrics(stageID: Int, status: String, + protected def recordTaskMetrics(stageId: Int, status: String, taskInfo: TaskInfo, taskMetrics: TaskMetrics) { - val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageID + + val info = " TID=" + taskInfo.taskId + " STAGE_ID=" + stageId + " START_TIME=" + taskInfo.launchTime + " FINISH_TIME=" + taskInfo.finishTime + " EXECUTOR_ID=" + taskInfo.executorId + " HOST=" + taskMetrics.hostname val executorRunTime = " EXECUTOR_RUN_TIME=" + taskMetrics.executorRunTime @@ -278,7 +170,7 @@ class JobLogger(val user: String, val logDirName: String) case Some(metrics) => " SHUFFLE_BYTES_WRITTEN=" + metrics.shuffleBytesWritten case None => "" } - stageLogInfo(stageID, status + info + executorRunTime + readMetrics + writeMetrics) + stageLogInfo(stageId, status + info + executorRunTime + readMetrics + writeMetrics) } /** @@ -286,8 +178,9 @@ class JobLogger(val user: String, val logDirName: String) * @param stageSubmitted Stage submitted event */ override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) { - stageLogInfo(stageSubmitted.stage.stageId,"STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format( - stageSubmitted.stage.stageId, stageSubmitted.stage.numTasks)) + val stageInfo = stageSubmitted.stageInfo + stageLogInfo(stageInfo.stageId, "STAGE_ID=%d STATUS=SUBMITTED TASK_SIZE=%d".format( + stageInfo.stageId, stageInfo.numTasks)) } /** @@ -295,36 +188,30 @@ class JobLogger(val user: String, val logDirName: String) * @param stageCompleted Stage completed event */ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - stageLogInfo(stageCompleted.stage.stageId, "STAGE_ID=%d STATUS=COMPLETED".format( - stageCompleted.stage.stageId)) + val stageId = stageCompleted.stageInfo.stageId + stageLogInfo(stageId, "STAGE_ID=%d STATUS=COMPLETED".format(stageId)) } - override def onTaskStart(taskStart: SparkListenerTaskStart) { } - /** * When task ends, record task completion status and metrics * @param taskEnd Task end event */ override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val task = taskEnd.task val taskInfo = taskEnd.taskInfo - var taskStatus = "" - task match { - case resultTask: ResultTask[_, _] => taskStatus = "TASK_TYPE=RESULT_TASK" - case shuffleMapTask: ShuffleMapTask => taskStatus = "TASK_TYPE=SHUFFLE_MAP_TASK" - } + var taskStatus = "TASK_TYPE=%s".format(taskEnd.taskType) + val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty() taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" - recordTaskMetrics(task.stageId, taskStatus, taskInfo, taskEnd.taskMetrics) + recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskMetrics) case Resubmitted => taskStatus += " STATUS=RESUBMITTED TID=" + taskInfo.taskId + - " STAGE_ID=" + task.stageId - stageLogInfo(task.stageId, taskStatus) + " STAGE_ID=" + taskEnd.stageId + stageLogInfo(taskEnd.stageId, taskStatus) case FetchFailed(bmAddress, shuffleId, mapId, reduceId) => taskStatus += " STATUS=FETCHFAILED TID=" + taskInfo.taskId + " STAGE_ID=" + - task.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + + taskEnd.stageId + " SHUFFLE_ID=" + shuffleId + " MAP_ID=" + mapId + " REDUCE_ID=" + reduceId - stageLogInfo(task.stageId, taskStatus) + stageLogInfo(taskEnd.stageId, taskStatus) case _ => } } @@ -334,8 +221,8 @@ class JobLogger(val user: String, val logDirName: String) * @param jobEnd Job end event */ override def onJobEnd(jobEnd: SparkListenerJobEnd) { - val job = jobEnd.job - var info = "JOB_ID=" + job.jobId + val jobId = jobEnd.jobId + var info = "JOB_ID=" + jobId jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" case JobFailed(exception, _) => @@ -343,19 +230,19 @@ class JobLogger(val user: String, val logDirName: String) exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => } - jobLogInfo(job.jobId, info.substring(0, info.length - 1).toUpperCase) - closeLogWriter(job.jobId) + jobLogInfo(jobId, info.substring(0, info.length - 1).toUpperCase) + closeLogWriter(jobId) } /** * Record job properties into job log file - * @param jobID ID of the job + * @param jobId ID of the job * @param properties Properties of the job */ - protected def recordJobProperties(jobID: Int, properties: Properties) { - if(properties != null) { + protected def recordJobProperties(jobId: Int, properties: Properties) { + if (properties != null) { val description = properties.getProperty(SparkContext.SPARK_JOB_DESCRIPTION, "") - jobLogInfo(jobID, description, false) + jobLogInfo(jobId, description, false) } } @@ -364,14 +251,11 @@ class JobLogger(val user: String, val logDirName: String) * @param jobStart Job start event */ override def onJobStart(jobStart: SparkListenerJobStart) { - val job = jobStart.job + val jobId = jobStart.jobId val properties = jobStart.properties - createLogWriter(job.jobId) - recordJobProperties(job.jobId, properties) - buildJobDep(job.jobId, job.finalStage) - recordStageDep(job.jobId) - recordStageDepGraph(job.jobId, job.finalStage, new HashSet[Int]) - jobLogInfo(job.jobId, "JOB_ID=" + job.jobId + " STATUS=STARTED") + createLogWriter(jobId) + recordJobProperties(jobId, properties) + buildJobStageDependencies(jobId, jobStart.stageIds) + jobLogInfo(jobId, "JOB_ID=" + jobId + " STATUS=STARTED") } } - diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index d94f6ad924260..3cf4e3077e4a4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -23,5 +23,6 @@ package org.apache.spark.scheduler private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult -private[spark] case class JobFailed(exception: Exception, failedStage: Option[Stage]) - extends JobResult + +// A failed stage ID of -1 means there is not a particular stage that caused the failure +private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index b026f860a8cd8..8007b5418741e 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -64,7 +64,7 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true - jobResult = JobFailed(exception, None) + jobResult = JobFailed(exception, -1) this.notifyAll() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala new file mode 100644 index 0000000000000..353a48661b0f7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -0,0 +1,101 @@ +/* + * 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.scheduler + +import java.util.concurrent.LinkedBlockingQueue + +import org.apache.spark.Logging + +/** + * Asynchronously passes SparkListenerEvents to registered SparkListeners. + * + * Until start() is called, all posted events are only buffered. Only after this listener bus + * has started will events be actually propagated to all attached listeners. This listener bus + * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). + */ +private[spark] class LiveListenerBus extends SparkListenerBus with Logging { + + /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ + private val EVENT_QUEUE_CAPACITY = 10000 + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) + private var queueFullErrorMessageLogged = false + private var started = false + + /** + * Start sending events to attached listeners. + * + * This first sends out all buffered events posted before this listener bus has started, then + * listens for any additional events asynchronously while the listener bus is still running. + * This should only be called once. + */ + def start() { + if (started) { + throw new IllegalStateException("Listener bus already started!") + } + started = true + new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + postToAll(event) + } + } + }.start() + } + + def post(event: SparkListenerEvent) { + val eventAdded = eventQueue.offer(event) + if (!eventAdded && !queueFullErrorMessageLogged) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + + "rate at which tasks are being started by the scheduler.") + queueFullErrorMessageLogged = true + } + } + + /** + * Waits until there are no more events in the queue, or until the specified time has elapsed. + * Used for testing only. Returns true if the queue has emptied and false is the specified time + * elapsed before the queue emptied. + */ + def waitUntilEmpty(timeoutMillis: Int): Boolean = { + val finishTime = System.currentTimeMillis + timeoutMillis + while (!eventQueue.isEmpty) { + if (System.currentTimeMillis > finishTime) { + return false + } + /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify + * add overhead in the general case. */ + Thread.sleep(10) + } + true + } + + def stop() { + if (!started) { + throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") + } + post(SparkListenerShutdown) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala index 4bc13c23d980b..187672c4e19e7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Pool.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Pool.scala @@ -62,7 +62,7 @@ private[spark] class Pool( override def addSchedulable(schedulable: Schedulable) { schedulableQueue += schedulable schedulableNameToSchedulable(schedulable.name) = schedulable - schedulable.parent= this + schedulable.parent = this } override def removeSchedulable(schedulable: Schedulable) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala new file mode 100644 index 0000000000000..db76178b65501 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -0,0 +1,104 @@ +/* + * 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.scheduler + +import java.io.InputStream +import java.net.URI + +import scala.io.Source + +import it.unimi.dsi.fastutil.io.FastBufferedInputStream +import org.apache.hadoop.fs.{Path, FileSystem} +import org.json4s.jackson.JsonMethods._ + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.io.CompressionCodec +import org.apache.spark.util.{JsonProtocol, Utils} + +/** + * An EventBus that replays logged events from persisted storage + */ +private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus with Logging { + private val compressed = conf.getBoolean("spark.eventLog.compress", false) + + // Only used if compression is enabled + private lazy val compressionCodec = CompressionCodec.createCodec(conf) + + /** + * Return a list of paths representing log files in the given directory. + */ + private def getLogFilePaths(logDir: String, fileSystem: FileSystem): Array[Path] = { + val path = new Path(logDir) + if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { + logWarning("Log path provided is not a valid directory: %s".format(logDir)) + return Array[Path]() + } + val logStatus = fileSystem.listStatus(path) + if (logStatus == null || !logStatus.exists(!_.isDir)) { + logWarning("Log path provided contains no log files: %s".format(logDir)) + return Array[Path]() + } + logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + } + + /** + * Replay each event in the order maintained in the given logs. + */ + def replay(logDir: String): Boolean = { + val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + val logPaths = getLogFilePaths(logDir, fileSystem) + if (logPaths.length == 0) { + return false + } + + logPaths.foreach { path => + // Keep track of input streams at all levels to close them later + // This is necessary because an exception can occur in between stream initializations + var fileStream: Option[InputStream] = None + var bufferedStream: Option[InputStream] = None + var compressStream: Option[InputStream] = None + var currentLine = "" + try { + currentLine = "" + fileStream = Some(fileSystem.open(path)) + bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) + compressStream = + if (compressed) { + Some(compressionCodec.compressedInputStream(bufferedStream.get)) + } else bufferedStream + + // Parse each line as an event and post it to all attached listeners + val lines = Source.fromInputStream(compressStream.get).getLines() + lines.foreach { line => + currentLine = line + postToAll(JsonProtocol.sparkEventFromJson(parse(line))) + } + } catch { + case e: Exception => + logError("Exception in parsing Spark event log %s".format(path), e) + logError("Malformed line: %s\n".format(currentLine)) + } finally { + fileStream.foreach(_.close()) + bufferedStream.foreach(_.close()) + compressStream.foreach(_.close()) + } + } + fileSystem.close() + true + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index 9590c03f10632..d4eb0ac88d8e8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -19,33 +19,52 @@ package org.apache.spark.scheduler import java.util.Properties +import scala.collection.Map +import scala.collection.mutable + import org.apache.spark.{Logging, TaskEndReason} import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} -sealed trait SparkListenerEvents +sealed trait SparkListenerEvent + +case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) + extends SparkListenerEvent + +case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent + +case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent -case class SparkListenerStageSubmitted(stage: StageInfo, properties: Properties) - extends SparkListenerEvents +case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent -case class SparkListenerStageCompleted(stage: StageInfo) extends SparkListenerEvents +case class SparkListenerTaskEnd( + stageId: Int, + taskType: String, + reason: TaskEndReason, + taskInfo: TaskInfo, + taskMetrics: TaskMetrics) + extends SparkListenerEvent -case class SparkListenerTaskStart(task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents +case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) + extends SparkListenerEvent -case class SparkListenerTaskGettingResult( - task: Task[_], taskInfo: TaskInfo) extends SparkListenerEvents +case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent -case class SparkListenerTaskEnd(task: Task[_], reason: TaskEndReason, taskInfo: TaskInfo, - taskMetrics: TaskMetrics) extends SparkListenerEvents +case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) + extends SparkListenerEvent -case class SparkListenerJobStart(job: ActiveJob, stageIds: Array[Int], - properties: Properties = null) extends SparkListenerEvents +case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) + extends SparkListenerEvent -case class SparkListenerJobEnd(job: ActiveJob, jobResult: JobResult) - extends SparkListenerEvents +case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) + extends SparkListenerEvent + +case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ -private[scheduler] case object SparkListenerShutdown extends SparkListenerEvents +private[spark] case object SparkListenerShutdown extends SparkListenerEvent + /** * Interface for listening to events from the Spark scheduler. @@ -87,97 +106,134 @@ trait SparkListener { */ def onJobEnd(jobEnd: SparkListenerJobEnd) { } + /** + * Called when environment properties have been updated + */ + def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { } + + /** + * Called when a new block manager has joined + */ + def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { } + + /** + * Called when an existing block manager has been removed + */ + def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { } + + /** + * Called when an RDD is manually unpersisted by the application + */ + def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } } /** * Simple SparkListener that logs a few summary statistics when each stage completes */ class StatsReportListener extends SparkListener with Logging { + + import org.apache.spark.scheduler.StatsReportListener._ + + private val taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { + val info = taskEnd.taskInfo + val metrics = taskEnd.taskMetrics + if (info != null && metrics != null) { + taskInfoMetrics += ((info, metrics)) + } + } + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { - import org.apache.spark.scheduler.StatsReportListener._ implicit val sc = stageCompleted - this.logInfo("Finished stage: " + stageCompleted.stage) - showMillisDistribution("task runtime:", (info, _) => Some(info.duration)) + this.logInfo("Finished stage: " + stageCompleted.stageInfo) + showMillisDistribution("task runtime:", (info, _) => Some(info.duration), taskInfoMetrics) - //shuffle write + // Shuffle write showBytesDistribution("shuffle bytes written:", - (_,metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten)) + (_, metric) => metric.shuffleWriteMetrics.map(_.shuffleBytesWritten), taskInfoMetrics) - //fetch & io + // Fetch & I/O showMillisDistribution("fetch wait time:", - (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime)) + (_, metric) => metric.shuffleReadMetrics.map(_.fetchWaitTime), taskInfoMetrics) showBytesDistribution("remote bytes read:", - (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead)) - showBytesDistribution("task result size:", (_, metric) => Some(metric.resultSize)) - - //runtime breakdown + (_, metric) => metric.shuffleReadMetrics.map(_.remoteBytesRead), taskInfoMetrics) + showBytesDistribution("task result size:", + (_, metric) => Some(metric.resultSize), taskInfoMetrics) - val runtimePcts = stageCompleted.stage.taskInfos.map{ - case (info, metrics) => RuntimePercentage(info.duration, metrics) + // Runtime breakdown + val runtimePcts = taskInfoMetrics.map { case (info, metrics) => + RuntimePercentage(info.duration, metrics) } showDistribution("executor (non-fetch) time pct: ", - Distribution(runtimePcts.map{_.executorPct * 100}), "%2.0f %%") + Distribution(runtimePcts.map(_.executorPct * 100)), "%2.0f %%") showDistribution("fetch wait time pct: ", - Distribution(runtimePcts.flatMap{_.fetchPct.map{_ * 100}}), "%2.0f %%") - showDistribution("other time pct: ", Distribution(runtimePcts.map{_.other * 100}), "%2.0f %%") + Distribution(runtimePcts.flatMap(_.fetchPct.map(_ * 100))), "%2.0f %%") + showDistribution("other time pct: ", Distribution(runtimePcts.map(_.other * 100)), "%2.0f %%") + taskInfoMetrics.clear() } } private[spark] object StatsReportListener extends Logging { - //for profiling, the extremes are more interesting + // For profiling, the extremes are more interesting val percentiles = Array[Int](0,5,10,25,50,75,90,95,100) - val probabilities = percentiles.map{_ / 100.0} + val probabilities = percentiles.map(_ / 100.0) val percentilesHeader = "\t" + percentiles.mkString("%\t") + "%" - def extractDoubleDistribution(stage: SparkListenerStageCompleted, - getMetric: (TaskInfo,TaskMetrics) => Option[Double]) - : Option[Distribution] = { - Distribution(stage.stage.taskInfos.flatMap { - case ((info,metric)) => getMetric(info, metric)}) + def extractDoubleDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Option[Double]): Option[Distribution] = { + Distribution(taskInfoMetrics.flatMap { case (info, metric) => getMetric(info, metric) }) } - //is there some way to setup the types that I can get rid of this completely? - def extractLongDistribution(stage: SparkListenerStageCompleted, - getMetric: (TaskInfo,TaskMetrics) => Option[Long]) - : Option[Distribution] = { - extractDoubleDistribution(stage, (info, metric) => getMetric(info,metric).map{_.toDouble}) + // Is there some way to setup the types that I can get rid of this completely? + def extractLongDistribution( + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)], + getMetric: (TaskInfo, TaskMetrics) => Option[Long]): Option[Distribution] = { + extractDoubleDistribution( + taskInfoMetrics, + (info, metric) => { getMetric(info, metric).map(_.toDouble) }) } def showDistribution(heading: String, d: Distribution, formatNumber: Double => String) { val stats = d.statCounter - val quantiles = d.getQuantiles(probabilities).map{formatNumber} + val quantiles = d.getQuantiles(probabilities).map(formatNumber) logInfo(heading + stats) logInfo(percentilesHeader) logInfo("\t" + quantiles.mkString("\t")) } - def showDistribution(heading: String, dOpt: Option[Distribution], formatNumber: Double => String) - { + def showDistribution( + heading: String, + dOpt: Option[Distribution], + formatNumber: Double => String) { dOpt.foreach { d => showDistribution(heading, d, formatNumber)} } def showDistribution(heading: String, dOpt: Option[Distribution], format:String) { - def f(d:Double) = format.format(d) + def f(d: Double) = format.format(d) showDistribution(heading, dOpt, f _) } def showDistribution( heading: String, format: String, - getMetric: (TaskInfo, TaskMetrics) => Option[Double]) - (implicit stage: SparkListenerStageCompleted) { - showDistribution(heading, extractDoubleDistribution(stage, getMetric), format) + getMetric: (TaskInfo, TaskMetrics) => Option[Double], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showDistribution(heading, extractDoubleDistribution(taskInfoMetrics, getMetric), format) } - def showBytesDistribution(heading:String, getMetric: (TaskInfo,TaskMetrics) => Option[Long]) - (implicit stage: SparkListenerStageCompleted) { - showBytesDistribution(heading, extractLongDistribution(stage, getMetric)) + def showBytesDistribution( + heading:String, + getMetric: (TaskInfo, TaskMetrics) => Option[Long], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showBytesDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) } def showBytesDistribution(heading: String, dOpt: Option[Distribution]) { - dOpt.foreach{dist => showBytesDistribution(heading, dist)} + dOpt.foreach { dist => showBytesDistribution(heading, dist) } } def showBytesDistribution(heading: String, dist: Distribution) { @@ -189,9 +245,11 @@ private[spark] object StatsReportListener extends Logging { (d => StatsReportListener.millisToString(d.toLong)): Double => String) } - def showMillisDistribution(heading: String, getMetric: (TaskInfo, TaskMetrics) => Option[Long]) - (implicit stage: SparkListenerStageCompleted) { - showMillisDistribution(heading, extractLongDistribution(stage, getMetric)) + def showMillisDistribution( + heading: String, + getMetric: (TaskInfo, TaskMetrics) => Option[Long], + taskInfoMetrics: Seq[(TaskInfo, TaskMetrics)]) { + showMillisDistribution(heading, extractLongDistribution(taskInfoMetrics, getMetric)) } val seconds = 1000L @@ -199,7 +257,7 @@ private[spark] object StatsReportListener extends Logging { val hours = minutes * 60 /** - * reformat a time interval in milliseconds to a prettier format for output + * Reformat a time interval in milliseconds to a prettier format for output */ def millisToString(ms: Long) = { val (size, units) = @@ -221,8 +279,8 @@ private case class RuntimePercentage(executorPct: Double, fetchPct: Option[Doubl private object RuntimePercentage { def apply(totalTime: Long, metrics: TaskMetrics): RuntimePercentage = { val denom = totalTime.toDouble - val fetchTime = metrics.shuffleReadMetrics.map{_.fetchWaitTime} - val fetch = fetchTime.map{_ / denom} + val fetchTime = metrics.shuffleReadMetrics.map(_.fetchWaitTime) + val fetch = fetchTime.map(_ / denom) val exec = (metrics.executorRunTime - fetchTime.getOrElse(0L)) / denom val other = 1.0 - (exec + fetch.getOrElse(0d)) RuntimePercentage(exec, fetch, other) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 17b1328b86788..729e120497571 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -1,100 +1,67 @@ -/* - * 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.scheduler - -import java.util.concurrent.LinkedBlockingQueue - -import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} - -import org.apache.spark.Logging - -/** Asynchronously passes SparkListenerEvents to registered SparkListeners. */ -private[spark] class SparkListenerBus extends Logging { - private val sparkListeners = new ArrayBuffer[SparkListener] with SynchronizedBuffer[SparkListener] - - /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ - private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvents](EVENT_QUEUE_CAPACITY) - private var queueFullErrorMessageLogged = false - - // Create a new daemon thread to listen for events. This thread is stopped when it receives - // a SparkListenerShutdown event, using the stop method. - new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - event match { - case stageSubmitted: SparkListenerStageSubmitted => - sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) - case stageCompleted: SparkListenerStageCompleted => - sparkListeners.foreach(_.onStageCompleted(stageCompleted)) - case jobStart: SparkListenerJobStart => - sparkListeners.foreach(_.onJobStart(jobStart)) - case jobEnd: SparkListenerJobEnd => - sparkListeners.foreach(_.onJobEnd(jobEnd)) - case taskStart: SparkListenerTaskStart => - sparkListeners.foreach(_.onTaskStart(taskStart)) - case taskGettingResult: SparkListenerTaskGettingResult => - sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) - case taskEnd: SparkListenerTaskEnd => - sparkListeners.foreach(_.onTaskEnd(taskEnd)) - case SparkListenerShutdown => - // Get out of the while loop and shutdown the daemon thread - return - case _ => - } - } - } - }.start() - - def addListener(listener: SparkListener) { - sparkListeners += listener - } - - def post(event: SparkListenerEvents) { - val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true - } - } - - /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time - * elapsed before the queue emptied. - */ - def waitUntilEmpty(timeoutMillis: Int): Boolean = { - val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { - if (System.currentTimeMillis > finishTime) { - return false - } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ - Thread.sleep(10) - } - true - } - - def stop(): Unit = post(SparkListenerShutdown) -} +/* + * 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.scheduler + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + +/** + * A SparkListenerEvent bus that relays events to its listeners + */ +private[spark] trait SparkListenerBus { + + // SparkListeners attached to this event bus + protected val sparkListeners = new ArrayBuffer[SparkListener] + with mutable.SynchronizedBuffer[SparkListener] + + def addListener(listener: SparkListener) { + sparkListeners += listener + } + + /** + * Post an event to all attached listeners. This does nothing if the event is + * SparkListenerShutdown. + */ + protected def postToAll(event: SparkListenerEvent) { + event match { + case stageSubmitted: SparkListenerStageSubmitted => + sparkListeners.foreach(_.onStageSubmitted(stageSubmitted)) + case stageCompleted: SparkListenerStageCompleted => + sparkListeners.foreach(_.onStageCompleted(stageCompleted)) + case jobStart: SparkListenerJobStart => + sparkListeners.foreach(_.onJobStart(jobStart)) + case jobEnd: SparkListenerJobEnd => + sparkListeners.foreach(_.onJobEnd(jobEnd)) + case taskStart: SparkListenerTaskStart => + sparkListeners.foreach(_.onTaskStart(taskStart)) + case taskGettingResult: SparkListenerTaskGettingResult => + sparkListeners.foreach(_.onTaskGettingResult(taskGettingResult)) + case taskEnd: SparkListenerTaskEnd => + sparkListeners.foreach(_.onTaskEnd(taskEnd)) + case environmentUpdate: SparkListenerEnvironmentUpdate => + sparkListeners.foreach(_.onEnvironmentUpdate(environmentUpdate)) + case blockManagerAdded: SparkListenerBlockManagerAdded => + sparkListeners.foreach(_.onBlockManagerAdded(blockManagerAdded)) + case blockManagerRemoved: SparkListenerBlockManagerRemoved => + sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) + case unpersistRDD: SparkListenerUnpersistRDD => + sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + case SparkListenerShutdown => + } + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 8f320e5c7a74b..8115a7ed7896d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,28 +17,25 @@ package org.apache.spark.scheduler -import scala.collection._ - -import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage.RDDInfo /** * Stores information about a stage to pass from the scheduler to SparkListeners. - * - * taskInfos stores the metrics for all tasks that have completed, including redundant, speculated - * tasks. */ -class StageInfo( - stage: Stage, - val taskInfos: mutable.Buffer[(TaskInfo, TaskMetrics)] = - mutable.Buffer[(TaskInfo, TaskMetrics)]() -) { - val stageId = stage.id +private[spark] +class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None var completionTime: Option[Long] = None - val rddName = stage.rdd.name - val name = stage.name - val numPartitions = stage.numPartitions - val numTasks = stage.numTasks var emittedTaskSizeWarning = false } + +private[spark] +object StageInfo { + def fromStage(stage: Stage): StageInfo = { + val rdd = stage.rdd + val rddName = Option(rdd.name).getOrElse(rdd.id.toString) + val rddInfo = new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) + new StageInfo(stage.id, stage.name, stage.numTasks, rddInfo) + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index ea3229b75be36..308edb12edd5c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler private[spark] object TaskLocality extends Enumeration { - // process local is expected to be used ONLY within tasksetmanager for now. + // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value type TaskLocality = Value diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index abff252597e16..30bceb47b9e7d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -204,7 +204,7 @@ private[spark] class TaskSchedulerImpl( executorIdToHost(o.executorId) = o.host if (!executorsByHost.contains(o.host)) { executorsByHost(o.host) = new HashSet[String]() - executorGained(o.executorId, o.host) + executorAdded(o.executorId, o.host) } } @@ -400,8 +400,8 @@ private[spark] class TaskSchedulerImpl( rootPool.executorLost(executorId, host) } - def executorGained(execId: String, host: String) { - dagScheduler.executorGained(execId, host) + def executorAdded(execId: String, host: String) { + dagScheduler.executorAdded(execId, host) } def getExecutorsAliveOnHost(host: String): Option[Set[String]] = synchronized { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index ee4b65e312abc..25b7472a99cdb 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler.cluster import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.deploy.{Command, ApplicationDescription} +import org.apache.spark.deploy.{ApplicationDescription, Command} import org.apache.spark.deploy.client.{AppClient, AppClientListener} import org.apache.spark.scheduler.{ExecutorExited, ExecutorLossReason, SlaveLost, TaskSchedulerImpl} import org.apache.spark.util.Utils @@ -26,8 +26,7 @@ import org.apache.spark.util.Utils private[spark] class SparkDeploySchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - masters: Array[String], - appName: String) + masters: Array[String]) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with AppClientListener with Logging { @@ -49,8 +48,8 @@ private[spark] class SparkDeploySchedulerBackend( val command = Command( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() - val appDesc = new ApplicationDescription(appName, maxCores, sc.executorMemory, command, - sparkHome, "http://" + sc.ui.appUIAddress) + val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, + sparkHome, sc.ui.appUIAddress, sc.eventLoggingInfo) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 28b019d9fd495..06b041e1fd9a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -45,8 +45,7 @@ import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend private[spark] class CoarseMesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - master: String, - appName: String) + master: String) extends CoarseGrainedSchedulerBackend(scheduler, sc.env.actorSystem) with MScheduler with Logging { @@ -94,7 +93,7 @@ private[spark] class CoarseMesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = CoarseMesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { { val ret = driver.run() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index bcf0ce19a54cd..4092dd04b112b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -41,8 +41,7 @@ import org.apache.spark.util.Utils private[spark] class MesosSchedulerBackend( scheduler: TaskSchedulerImpl, sc: SparkContext, - master: String, - appName: String) + master: String) extends SchedulerBackend with MScheduler with Logging { @@ -71,7 +70,7 @@ private[spark] class MesosSchedulerBackend( setDaemon(true) override def run() { val scheduler = MesosSchedulerBackend.this - val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(appName).build() + val fwInfo = FrameworkInfo.newBuilder().setUser("").setName(sc.appName).build() driver = new MesosSchedulerDriver(scheduler, fwInfo, master) try { val ret = driver.run() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 1bf3f4db32ea7..71584b6eb102a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -29,7 +29,7 @@ import akka.actor.{ActorSystem, Cancellable, Props} import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer -import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException, SecurityManager} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer @@ -92,7 +92,7 @@ private[spark] class BlockManager( val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) - // Pending reregistration action being executed asynchronously or null if none + // Pending re-registration action being executed asynchronously or null if none // is pending. Accesses should synchronize on asyncReregisterLock. var asyncReregisterTask: Future[Unit] = null val asyncReregisterLock = new Object @@ -122,10 +122,15 @@ private[spark] class BlockManager( /** * Construct a BlockManager with a memory limit set based on system properties. */ - def this(execId: String, actorSystem: ActorSystem, master: BlockManagerMaster, - serializer: Serializer, conf: SparkConf, securityManager: SecurityManager) = { - this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), conf, - securityManager) + def this( + execId: String, + actorSystem: ActorSystem, + master: BlockManagerMaster, + serializer: Serializer, + conf: SparkConf, + securityManager: SecurityManager) = { + this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), + conf, securityManager) } /** @@ -148,14 +153,15 @@ private[spark] class BlockManager( * an executor crash. * * This function deliberately fails silently if the master returns false (indicating that - * the slave needs to reregister). The error condition will be detected again by the next - * heart beat attempt or new block registration and another try to reregister all blocks + * the slave needs to re-register). The error condition will be detected again by the next + * heart beat attempt or new block registration and another try to re-register all blocks * will be made then. */ private def reportAllBlocks() { logInfo("Reporting " + blockInfo.size + " blocks to the master.") for ((blockId, info) <- blockInfo) { - if (!tryToReportBlockStatus(blockId, info)) { + val status = getCurrentBlockStatus(blockId, info) + if (!tryToReportBlockStatus(blockId, info, status)) { logError("Failed to report " + blockId + " to master; giving up.") return } @@ -163,20 +169,20 @@ private[spark] class BlockManager( } /** - * Reregister with the master and report all blocks to it. This will be called by the heart beat + * Re-register with the master and report all blocks to it. This will be called by the heart beat * thread if our heartbeat to the block manager indicates that we were not registered. * * Note that this method must be called without any BlockInfo locks held. */ def reregister() { - // TODO: We might need to rate limit reregistering. - logInfo("BlockManager reregistering with master") + // TODO: We might need to rate limit re-registering. + logInfo("BlockManager re-registering with master") master.registerBlockManager(blockManagerId, maxMemory, slaveActor) reportAllBlocks() } /** - * Reregister with the master sometime soon. + * Re-register with the master sometime soon. */ def asyncReregister() { asyncReregisterLock.synchronized { @@ -192,7 +198,7 @@ private[spark] class BlockManager( } /** - * For testing. Wait for any pending asynchronous reregistration; otherwise, do nothing. + * For testing. Wait for any pending asynchronous re-registration; otherwise, do nothing. */ def waitForAsyncReregister() { val task = asyncReregisterTask @@ -211,15 +217,19 @@ private[spark] class BlockManager( * message reflecting the current status, *not* the desired storage level in its block info. * For example, a block with MEMORY_AND_DISK set might have fallen out to be only on disk. * - * droppedMemorySize exists to account for when block is dropped from memory to disk (so it - * is still valid). This ensures that update in master will compensate for the increase in + * droppedMemorySize exists to account for when the block is dropped from memory to disk (so + * it is still valid). This ensures that update in master will compensate for the increase in * memory on slave. */ - def reportBlockStatus(blockId: BlockId, info: BlockInfo, droppedMemorySize: Long = 0L) { - val needReregister = !tryToReportBlockStatus(blockId, info, droppedMemorySize) + def reportBlockStatus( + blockId: BlockId, + info: BlockInfo, + status: BlockStatus, + droppedMemorySize: Long = 0L) { + val needReregister = !tryToReportBlockStatus(blockId, info, status, droppedMemorySize) if (needReregister) { - logInfo("Got told to reregister updating block " + blockId) - // Reregistering will report our new block for free. + logInfo("Got told to re-register updating block " + blockId) + // Re-registering will report our new block for free. asyncReregister() } logDebug("Told master about block " + blockId) @@ -230,27 +240,41 @@ private[spark] class BlockManager( * which will be true if the block was successfully recorded and false if * the slave needs to re-register. */ - private def tryToReportBlockStatus(blockId: BlockId, info: BlockInfo, + private def tryToReportBlockStatus( + blockId: BlockId, + info: BlockInfo, + status: BlockStatus, droppedMemorySize: Long = 0L): Boolean = { - val (curLevel, inMemSize, onDiskSize, tellMaster) = info.synchronized { + if (info.tellMaster) { + val storageLevel = status.storageLevel + val inMemSize = Math.max(status.memSize, droppedMemorySize) + val onDiskSize = status.diskSize + master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) + } else true + } + + /** + * Return the updated storage status of the block with the given ID. More specifically, if + * the block is dropped from memory and possibly added to disk, return the new storage level + * and the updated in-memory and on-disk sizes. + */ + private def getCurrentBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { + val (newLevel, inMemSize, onDiskSize) = info.synchronized { info.level match { case null => - (StorageLevel.NONE, 0L, 0L, false) + (StorageLevel.NONE, 0L, 0L) case level => val inMem = level.useMemory && memoryStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) - val storageLevel = StorageLevel(onDisk, inMem, level.deserialized, level.replication) - val memSize = if (inMem) memoryStore.getSize(blockId) else droppedMemorySize + val deserialized = if (inMem) level.deserialized else false + val replication = if (inMem || onDisk) level.replication else 1 + val storageLevel = StorageLevel(onDisk, inMem, deserialized, replication) + val memSize = if (inMem) memoryStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - (storageLevel, memSize, diskSize, info.tellMaster) + (storageLevel, memSize, diskSize) } } - - if (tellMaster) { - master.updateBlockInfo(blockManagerId, blockId, curLevel, inMemSize, onDiskSize) - } else { - true - } + BlockStatus(newLevel, inMemSize, onDiskSize) } /** @@ -398,10 +422,10 @@ private[spark] class BlockManager( /** * Get block from remote block managers as serialized bytes. */ - def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { + def getRemoteBytes(blockId: BlockId): Option[ByteBuffer] = { logDebug("Getting remote block " + blockId + " as bytes") doGetRemote(blockId, asValues = false).asInstanceOf[Option[ByteBuffer]] - } + } private def doGetRemote(blockId: BlockId, asValues: Boolean): Option[Any] = { require(blockId != null, "BlockId is null") @@ -447,9 +471,8 @@ private[spark] class BlockManager( * so that we can control the maxMegabytesInFlight for the fetch. */ def getMultiple( - blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], serializer: Serializer) - : BlockFetcherIterator = { - + blocksByAddress: Seq[(BlockManagerId, Seq[(BlockId, Long)])], + serializer: Serializer): BlockFetcherIterator = { val iter = if (conf.getBoolean("spark.shuffle.use.netty", false)) { new BlockFetcherIterator.NettyBlockFetcherIterator(this, blocksByAddress, serializer) @@ -461,8 +484,11 @@ private[spark] class BlockManager( iter } - def put(blockId: BlockId, values: Iterator[Any], level: StorageLevel, tellMaster: Boolean) - : Long = { + def put( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + tellMaster: Boolean): Seq[(BlockId, BlockStatus)] = { doPut(blockId, IteratorValues(values), level, tellMaster) } @@ -472,41 +498,58 @@ private[spark] class BlockManager( * This is currently used for writing shuffle files out. Callers should handle error * cases. */ - def getDiskWriter(blockId: BlockId, file: File, serializer: Serializer, bufferSize: Int) - : BlockObjectWriter = { + def getDiskWriter( + blockId: BlockId, + file: File, + serializer: Serializer, + bufferSize: Int): BlockObjectWriter = { val compressStream: OutputStream => OutputStream = wrapForCompression(blockId, _) val syncWrites = conf.getBoolean("spark.shuffle.sync", false) new DiskBlockObjectWriter(blockId, file, serializer, bufferSize, compressStream, syncWrites) } /** - * Put a new block of values to the block manager. Returns its (estimated) size in bytes. + * Put a new block of values to the block manager. Return a list of blocks updated as a + * result of this put. */ - def put(blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - tellMaster: Boolean = true) : Long = { + def put( + blockId: BlockId, + values: ArrayBuffer[Any], + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { require(values != null, "Values is null") doPut(blockId, ArrayBufferValues(values), level, tellMaster) } /** - * Put a new block of serialized bytes to the block manager. + * Put a new block of serialized bytes to the block manager. Return a list of blocks updated + * as a result of this put. */ - def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel, - tellMaster: Boolean = true) { + def putBytes( + blockId: BlockId, + bytes: ByteBuffer, + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { require(bytes != null, "Bytes is null") doPut(blockId, ByteBufferValues(bytes), level, tellMaster) } - private def doPut(blockId: BlockId, - data: Values, - level: StorageLevel, tellMaster: Boolean = true): Long = { + private def doPut( + blockId: BlockId, + data: Values, + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { + require(blockId != null, "BlockId is null") require(level != null && level.isValid, "StorageLevel is null or invalid") + // Return value + val updatedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + // Remember the block's storage level so that we can correctly drop it to disk if it needs // to be dropped right after it got put into memory. Note, however, that other threads will // not be able to get() this block until we call markReady on its BlockInfo. - val myInfo = { + val putBlockInfo = { val tinfo = new BlockInfo(level, tellMaster) // Do atomically ! val oldBlockOpt = blockInfo.putIfAbsent(blockId, tinfo) @@ -514,7 +557,7 @@ private[spark] class BlockManager( if (oldBlockOpt.isDefined) { if (oldBlockOpt.get.waitForReady()) { logWarning("Block " + blockId + " already exists on this machine; not re-adding it") - return oldBlockOpt.get.size + return updatedBlocks } // TODO: So the block info exists - but previous attempt to load it (?) failed. @@ -536,7 +579,7 @@ private[spark] class BlockManager( // Ditto for the bytes after the put var bytesAfterPut: ByteBuffer = null - // Size of the block in bytes (to return to caller) + // Size of the block in bytes var size = 0L // If we're storing bytes, then initiate the replication before storing them locally. @@ -551,7 +594,7 @@ private[spark] class BlockManager( null } - myInfo.synchronized { + putBlockInfo.synchronized { logTrace("Put for block " + blockId + " took " + Utils.getUsedTimeMs(startTimeMs) + " to get into synchronized block") @@ -566,7 +609,7 @@ private[spark] class BlockManager( case ArrayBufferValues(array) => memoryStore.putValues(blockId, array, level, true) case ByteBufferValues(bytes) => { - bytes.rewind(); + bytes.rewind() memoryStore.putBytes(blockId, bytes, level) } } @@ -575,6 +618,8 @@ private[spark] class BlockManager( case Right(newBytes) => bytesAfterPut = newBytes case Left(newIterator) => valuesAfterPut = newIterator } + // Keep track of which blocks are dropped from memory + res.droppedBlocks.foreach { block => updatedBlocks += block } } else { // Save directly to disk. // Don't get back the bytes unless we replicate them. @@ -586,7 +631,7 @@ private[spark] class BlockManager( case ArrayBufferValues(array) => diskStore.putValues(blockId, array, level, askForBytes) case ByteBufferValues(bytes) => { - bytes.rewind(); + bytes.rewind() diskStore.putBytes(blockId, bytes, level) } } @@ -597,21 +642,25 @@ private[spark] class BlockManager( } } - // Now that the block is in either the memory or disk store, let other threads read it, - // and tell the master about it. - marked = true - myInfo.markReady(size) - if (tellMaster) { - reportBlockStatus(blockId, myInfo) + val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) + if (putBlockStatus.storageLevel != StorageLevel.NONE) { + // Now that the block is in either the memory or disk store, let other threads read it, + // and tell the master about it. + marked = true + putBlockInfo.markReady(size) + if (tellMaster) { + reportBlockStatus(blockId, putBlockInfo, putBlockStatus) + } + updatedBlocks += ((blockId, putBlockStatus)) } } finally { - // If we failed at putting the block to memory/disk, notify other possible readers + // If we failed in putting the block to memory/disk, notify other possible readers // that it has failed, and then remove it from the block info map. - if (! marked) { + if (!marked) { // Note that the remove must happen before markFailure otherwise another thread // could've inserted a new BlockInfo before we remove it. blockInfo.remove(blockId) - myInfo.markFailure() + putBlockInfo.markFailure() logWarning("Putting block " + blockId + " failed") } } @@ -650,7 +699,7 @@ private[spark] class BlockManager( Utils.getUsedTimeMs(startTimeMs)) } - size + updatedBlocks } /** @@ -687,28 +736,42 @@ private[spark] class BlockManager( /** * Write a block consisting of a single object. */ - def putSingle(blockId: BlockId, value: Any, level: StorageLevel, tellMaster: Boolean = true) { + def putSingle( + blockId: BlockId, + value: Any, + level: StorageLevel, + tellMaster: Boolean = true): Seq[(BlockId, BlockStatus)] = { put(blockId, Iterator(value), level, tellMaster) } /** * Drop a block from memory, possibly putting it on disk if applicable. Called when the memory * store reaches its limit and needs to free up space. + * + * Return the block status if the given block has been updated, else None. */ - def dropFromMemory(blockId: BlockId, data: Either[ArrayBuffer[Any], ByteBuffer]) { + def dropFromMemory( + blockId: BlockId, + data: Either[ArrayBuffer[Any], ByteBuffer]): Option[BlockStatus] = { + logInfo("Dropping block " + blockId + " from memory") val info = blockInfo.get(blockId).orNull + + // If the block has not already been dropped if (info != null) { info.synchronized { // required ? As of now, this will be invoked only for blocks which are ready // But in case this changes in future, adding for consistency sake. - if (! info.waitForReady() ) { + if (!info.waitForReady()) { // If we get here, the block write failed. logWarning("Block " + blockId + " was marked as failure. Nothing to drop") - return + return None } + var blockIsUpdated = false val level = info.level + + // Drop to disk, if storage level requires if (level.useDisk && !diskStore.contains(blockId)) { logInfo("Writing block " + blockId + " to disk") data match { @@ -717,24 +780,33 @@ private[spark] class BlockManager( case Right(bytes) => diskStore.putBytes(blockId, bytes, level) } + blockIsUpdated = true } + + // Actually drop from memory store val droppedMemorySize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L - val blockWasRemoved = memoryStore.remove(blockId) - if (!blockWasRemoved) { + val blockIsRemoved = memoryStore.remove(blockId) + if (blockIsRemoved) { + blockIsUpdated = true + } else { logWarning("Block " + blockId + " could not be dropped from memory as it does not exist") } + + val status = getCurrentBlockStatus(blockId, info) if (info.tellMaster) { - reportBlockStatus(blockId, info, droppedMemorySize) + reportBlockStatus(blockId, info, status, droppedMemorySize) } if (!level.useDisk) { // The block is completely gone from this node; forget it so we can put() it again later. blockInfo.remove(blockId) } + if (blockIsUpdated) { + return Some(status) + } } - } else { - // The block has already been dropped } + None } /** @@ -766,7 +838,8 @@ private[spark] class BlockManager( } blockInfo.remove(blockId) if (tellMaster && info.tellMaster) { - reportBlockStatus(blockId, info) + val status = getCurrentBlockStatus(blockId, info) + reportBlockStatus(blockId, info, status) } } else { // The block has already been removed; do nothing. @@ -801,7 +874,8 @@ private[spark] class BlockManager( iterator.remove() logInfo("Dropped block " + id) } - reportBlockStatus(id, info) + val status = getCurrentBlockStatus(id, info) + reportBlockStatus(id, info, status) } } } @@ -911,9 +985,8 @@ private[spark] object BlockManager extends Logging { def blockIdsToBlockManagers( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null) - : Map[BlockId, Seq[BlockManagerId]] = - { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[BlockManagerId]] = { + // blockManagerMaster != null is used in tests assert (env != null || blockManagerMaster != null) val blockLocations: Seq[Seq[BlockManagerId]] = if (blockManagerMaster == null) { @@ -932,18 +1005,14 @@ private[spark] object BlockManager extends Logging { def blockIdsToExecutorIds( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null) - : Map[BlockId, Seq[String]] = - { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.executorId)) } def blockIdsToHosts( blockIds: Array[BlockId], env: SparkEnv, - blockManagerMaster: BlockManagerMaster = null) - : Map[BlockId, Seq[String]] = - { + blockManagerMaster: BlockManagerMaster = null): Map[BlockId, Seq[String]] = { blockIdsToBlockManagers(blockIds, env, blockManagerMaster).mapValues(s => s.map(_.host)) } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala index 98cd6e68fa724..be537d77309bc 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerId.scala @@ -50,7 +50,6 @@ private[spark] class BlockManagerId private ( // DEBUG code Utils.checkHost(host) assert (port > 0) - host + ":" + port } @@ -93,7 +92,7 @@ private[spark] class BlockManagerId private ( private[spark] object BlockManagerId { /** - * Returns a [[org.apache.spark.storage.BlockManagerId]] for the given configuraiton. + * Returns a [[org.apache.spark.storage.BlockManagerId]] for the given configuration. * * @param execId ID of the executor. * @param host Host name of the block manager. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index e531467cccb40..ed6937851b836 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -28,8 +28,7 @@ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.AkkaUtils private[spark] -class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Logging { - +class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Logging { val AKKA_RETRY_ATTEMPTS: Int = conf.getInt("spark.akka.num.retries", 3) val AKKA_RETRY_INTERVAL_MS: Int = conf.getInt("spark.akka.retry.wait", 3000) @@ -53,8 +52,7 @@ class BlockManagerMaster(var driverActor : ActorRef, conf: SparkConf) extends Lo } /** Register the BlockManager's id with the driver. */ - def registerBlockManager( - blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { + def registerBlockManager(blockManagerId: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { logInfo("Trying to register BlockManager") tell(RegisterBlockManager(blockManagerId, maxMemSize, slaveActor)) logInfo("Registered BlockManager") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index a999d76a326a6..ff2652b640272 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -28,6 +28,7 @@ import akka.actor.{Actor, ActorRef, Cancellable} import akka.pattern.ask import org.apache.spark.{Logging, SparkConf, SparkException} +import org.apache.spark.scheduler._ import org.apache.spark.storage.BlockManagerMessages._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -36,11 +37,11 @@ import org.apache.spark.util.{AkkaUtils, Utils} * all slaves' block managers. */ private[spark] -class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Actor with Logging { +class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus: LiveListenerBus) + extends Actor with Logging { // Mapping from block manager id to the block manager's information. - private val blockManagerInfo = - new mutable.HashMap[BlockManagerId, BlockManagerMasterActor.BlockManagerInfo] + private val blockManagerInfo = new mutable.HashMap[BlockManagerId, BlockManagerInfo] // Mapping from executor ID to block manager ID. private val blockManagerIdByExecutor = new mutable.HashMap[String, BlockManagerId] @@ -160,6 +161,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act blockLocations.remove(locations) } } + listenerBus.post(SparkListenerBlockManagerRemoved(blockManagerId)) } private def expireDeadHosts() { @@ -217,8 +219,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act private def storageStatus: Array[StorageStatus] = { blockManagerInfo.map { case(blockManagerId, info) => - import collection.JavaConverters._ - StorageStatus(blockManagerId, info.maxMem, info.blocks.asScala.toMap) + val blockMap = mutable.Map[BlockId, BlockStatus](info.blocks.toSeq: _*) + new StorageStatus(blockManagerId, info.maxMem, blockMap) }.toArray } @@ -233,9 +235,10 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act case None => blockManagerIdByExecutor(id.executorId) = id } - blockManagerInfo(id) = new BlockManagerMasterActor.BlockManagerInfo( - id, System.currentTimeMillis(), maxMemSize, slaveActor) + blockManagerInfo(id) = + new BlockManagerInfo(id, System.currentTimeMillis(), maxMemSize, slaveActor) } + listenerBus.post(SparkListenerBlockManagerAdded(id, maxMemSize)) } private def updateBlockInfo( @@ -307,97 +310,96 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf) extends Act } -private[spark] -object BlockManagerMasterActor { - - case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) +private[spark] case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) - class BlockManagerInfo( - val blockManagerId: BlockManagerId, - timeMs: Long, - val maxMem: Long, - val slaveActor: ActorRef) - extends Logging { +private[spark] class BlockManagerInfo( + val blockManagerId: BlockManagerId, + timeMs: Long, + val maxMem: Long, + val slaveActor: ActorRef) + extends Logging { - private var _lastSeenMs: Long = timeMs - private var _remainingMem: Long = maxMem + private var _lastSeenMs: Long = timeMs + private var _remainingMem: Long = maxMem - // Mapping from block id to its status. - private val _blocks = new JHashMap[BlockId, BlockStatus] + // Mapping from block id to its status. + private val _blocks = new JHashMap[BlockId, BlockStatus] - logInfo("Registering block manager %s with %s RAM".format( - blockManagerId.hostPort, Utils.bytesToString(maxMem))) + logInfo("Registering block manager %s with %s RAM".format( + blockManagerId.hostPort, Utils.bytesToString(maxMem))) - def updateLastSeenMs() { - _lastSeenMs = System.currentTimeMillis() - } + def updateLastSeenMs() { + _lastSeenMs = System.currentTimeMillis() + } - def updateBlockInfo(blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + def updateBlockInfo( + blockId: BlockId, + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long) { - updateLastSeenMs() + updateLastSeenMs() - if (_blocks.containsKey(blockId)) { - // The block exists on the slave already. - val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel + if (_blocks.containsKey(blockId)) { + // The block exists on the slave already. + val originalLevel: StorageLevel = _blocks.get(blockId).storageLevel - if (originalLevel.useMemory) { - _remainingMem += memSize - } + if (originalLevel.useMemory) { + _remainingMem += memSize } + } - if (storageLevel.isValid) { - // isValid means it is either stored in-memory or on-disk. - // But the memSize here indicates the data size in or dropped from memory, - // and the diskSize here indicates the data size in or dropped to disk. - // They can be both larger than 0, when a block is dropped from memory to disk. - // Therefore, a safe way to set BlockStatus is to set its info in accurate modes. - if (storageLevel.useMemory) { - _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) - _remainingMem -= memSize - logInfo("Added %s in memory on %s (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), - Utils.bytesToString(_remainingMem))) - } - if (storageLevel.useDisk) { - _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize)) - logInfo("Added %s on disk on %s (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) - } - } else if (_blocks.containsKey(blockId)) { - // If isValid is not true, drop the block. - val blockStatus: BlockStatus = _blocks.get(blockId) - _blocks.remove(blockId) - if (blockStatus.storageLevel.useMemory) { - _remainingMem += blockStatus.memSize - logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), - Utils.bytesToString(_remainingMem))) - } - if (blockStatus.storageLevel.useDisk) { - logInfo("Removed %s on %s on disk (size: %s)".format( - blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) - } + if (storageLevel.isValid) { + /* isValid means it is either stored in-memory or on-disk. + * But the memSize here indicates the data size in or dropped from memory, + * and the diskSize here indicates the data size in or dropped to disk. + * They can be both larger than 0, when a block is dropped from memory to disk. + * Therefore, a safe way to set BlockStatus is to set its info in accurate modes. */ + if (storageLevel.useMemory) { + _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) + _remainingMem -= memSize + logInfo("Added %s in memory on %s (size: %s, free: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), + Utils.bytesToString(_remainingMem))) + } + if (storageLevel.useDisk) { + _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize)) + logInfo("Added %s on disk on %s (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) + } + } else if (_blocks.containsKey(blockId)) { + // If isValid is not true, drop the block. + val blockStatus: BlockStatus = _blocks.get(blockId) + _blocks.remove(blockId) + if (blockStatus.storageLevel.useMemory) { + _remainingMem += blockStatus.memSize + logInfo("Removed %s on %s in memory (size: %s, free: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.memSize), + Utils.bytesToString(_remainingMem))) + } + if (blockStatus.storageLevel.useDisk) { + logInfo("Removed %s on %s on disk (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } } + } - def removeBlock(blockId: BlockId) { - if (_blocks.containsKey(blockId)) { - _remainingMem += _blocks.get(blockId).memSize - _blocks.remove(blockId) - } + def removeBlock(blockId: BlockId) { + if (_blocks.containsKey(blockId)) { + _remainingMem += _blocks.get(blockId).memSize + _blocks.remove(blockId) } + } - def remainingMem: Long = _remainingMem + def remainingMem: Long = _remainingMem - def lastSeenMs: Long = _lastSeenMs + def lastSeenMs: Long = _lastSeenMs - def blocks: JHashMap[BlockId, BlockStatus] = _blocks + def blocks: JHashMap[BlockId, BlockStatus] = _blocks - override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem + override def toString: String = "BlockManagerInfo " + timeMs + " " + _remainingMem - def clear() { - _blocks.clear() - } + def clear() { + _blocks.clear() } } diff --git a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala index 38836d44b04e8..488f1ea9628f5 100644 --- a/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala +++ b/core/src/main/scala/org/apache/spark/storage/MemoryStore.scala @@ -49,7 +49,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } - override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel) : PutResult = { + override def putBytes(blockId: BlockId, _bytes: ByteBuffer, level: StorageLevel): PutResult = { // Work on a duplicate - since the original input might be used elsewhere. val bytes = _bytes.duplicate() bytes.rewind() @@ -70,16 +70,15 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) blockId: BlockId, values: ArrayBuffer[Any], level: StorageLevel, - returnValues: Boolean) - : PutResult = { + returnValues: Boolean): PutResult = { if (level.deserialized) { val sizeEstimate = SizeEstimator.estimate(values.asInstanceOf[AnyRef]) - tryToPut(blockId, values, sizeEstimate, true) - PutResult(sizeEstimate, Left(values.toIterator)) + val putAttempt = tryToPut(blockId, values, sizeEstimate, deserialized = true) + PutResult(sizeEstimate, Left(values.iterator), putAttempt.droppedBlocks) } else { - val bytes = blockManager.dataSerialize(blockId, values.toIterator) - tryToPut(blockId, bytes, bytes.limit, false) - PutResult(bytes.limit(), Right(bytes.duplicate())) + val bytes = blockManager.dataSerialize(blockId, values.iterator) + val putAttempt = tryToPut(blockId, bytes, bytes.limit, deserialized = false) + PutResult(bytes.limit(), Right(bytes.duplicate()), putAttempt.droppedBlocks) } } @@ -87,20 +86,10 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) blockId: BlockId, values: Iterator[Any], level: StorageLevel, - returnValues: Boolean) - : PutResult = { - - if (level.deserialized) { - val valueEntries = new ArrayBuffer[Any]() - valueEntries ++= values - val sizeEstimate = SizeEstimator.estimate(valueEntries.asInstanceOf[AnyRef]) - tryToPut(blockId, valueEntries, sizeEstimate, true) - PutResult(sizeEstimate, Left(valueEntries.toIterator)) - } else { - val bytes = blockManager.dataSerialize(blockId, values) - tryToPut(blockId, bytes, bytes.limit, false) - PutResult(bytes.limit(), Right(bytes.duplicate())) - } + returnValues: Boolean): PutResult = { + val valueEntries = new ArrayBuffer[Any]() + valueEntries ++= values + putValues(blockId, valueEntries, level, returnValues) } override def getBytes(blockId: BlockId): Option[ByteBuffer] = { @@ -164,19 +153,34 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) * an ArrayBuffer if deserialized is true or a ByteBuffer otherwise. Its (possibly estimated) * size must also be passed by the caller. * - * Locks on the object putLock to ensure that all the put requests and its associated block + * Lock on the object putLock to ensure that all the put requests and its associated block * dropping is done by only on thread at a time. Otherwise while one thread is dropping * blocks to free memory for one block, another thread may use up the freed space for * another block. + * + * Return whether put was successful, along with the blocks dropped in the process. */ - private def tryToPut(blockId: BlockId, value: Any, size: Long, deserialized: Boolean): Boolean = { - // TODO: Its possible to optimize the locking by locking entries only when selecting blocks - // to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has been - // released, it must be ensured that those to-be-dropped blocks are not double counted for - // freeing up more space for another block that needs to be put. Only then the actually dropping - // of blocks (and writing to disk if necessary) can proceed in parallel. + private def tryToPut( + blockId: BlockId, + value: Any, + size: Long, + deserialized: Boolean): ResultWithDroppedBlocks = { + + /* TODO: Its possible to optimize the locking by locking entries only when selecting blocks + * to be dropped. Once the to-be-dropped blocks have been selected, and lock on entries has + * been released, it must be ensured that those to-be-dropped blocks are not double counted + * for freeing up more space for another block that needs to be put. Only then the actually + * dropping of blocks (and writing to disk if necessary) can proceed in parallel. */ + + var putSuccess = false + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + putLock.synchronized { - if (ensureFreeSpace(blockId, size)) { + val freeSpaceResult = ensureFreeSpace(blockId, size) + val enoughFreeSpace = freeSpaceResult.success + droppedBlocks ++= freeSpaceResult.droppedBlocks + + if (enoughFreeSpace) { val entry = new Entry(value, size, deserialized) entries.synchronized { entries.put(blockId, entry) @@ -189,7 +193,7 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) logInfo("Block %s stored as bytes to memory (size %s, free %s)".format( blockId, Utils.bytesToString(size), Utils.bytesToString(freeMemory))) } - true + putSuccess = true } else { // Tell the block manager that we couldn't put it in memory so that it can drop it to // disk if the block allows disk storage. @@ -198,29 +202,33 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { Right(value.asInstanceOf[ByteBuffer].duplicate()) } - blockManager.dropFromMemory(blockId, data) - false + val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) + droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } } } + ResultWithDroppedBlocks(putSuccess, droppedBlocks) } /** - * Tries to free up a given amount of space to store a particular block, but can fail and return - * false if either the block is bigger than our memory or it would require replacing another - * block from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that + * Try to free up a given amount of space to store a particular block, but can fail if + * either the block is bigger than our memory or it would require replacing another block + * from the same RDD (which leads to a wasteful cyclic replacement pattern for RDDs that * don't fit into memory that we want to avoid). * - * Assumes that a lock is held by the caller to ensure only one thread is dropping blocks. + * Assume that a lock is held by the caller to ensure only one thread is dropping blocks. * Otherwise, the freed space may fill up before the caller puts in their new value. + * + * Return whether there is enough free space, along with the blocks dropped in the process. */ - private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): Boolean = { - + private def ensureFreeSpace(blockIdToAdd: BlockId, space: Long): ResultWithDroppedBlocks = { logInfo("ensureFreeSpace(%d) called with curMem=%d, maxMem=%d".format( space, currentMemory, maxMemory)) + val droppedBlocks = new ArrayBuffer[(BlockId, BlockStatus)] + if (space > maxMemory) { logInfo("Will not store " + blockIdToAdd + " as it is larger than our memory limit") - return false + return ResultWithDroppedBlocks(success = false, droppedBlocks) } if (maxMemory - currentMemory < space) { @@ -256,17 +264,18 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } else { Right(entry.value.asInstanceOf[ByteBuffer].duplicate()) } - blockManager.dropFromMemory(blockId, data) + val droppedBlockStatus = blockManager.dropFromMemory(blockId, data) + droppedBlockStatus.foreach { status => droppedBlocks += ((blockId, status)) } } } - return true + return ResultWithDroppedBlocks(success = true, droppedBlocks) } else { logInfo(s"Will not store $blockIdToAdd as it would require dropping another block " + "from the same RDD") - return false + return ResultWithDroppedBlocks(success = false, droppedBlocks) } } - true + ResultWithDroppedBlocks(success = true, droppedBlocks) } override def contains(blockId: BlockId): Boolean = { @@ -274,3 +283,6 @@ private class MemoryStore(blockManager: BlockManager, maxMemory: Long) } } +private case class ResultWithDroppedBlocks( + success: Boolean, + droppedBlocks: Seq[(BlockId, BlockStatus)]) diff --git a/core/src/main/scala/org/apache/spark/storage/PutResult.scala b/core/src/main/scala/org/apache/spark/storage/PutResult.scala index 2eba2f06b5bfd..f0eac7594ecf6 100644 --- a/core/src/main/scala/org/apache/spark/storage/PutResult.scala +++ b/core/src/main/scala/org/apache/spark/storage/PutResult.scala @@ -20,7 +20,13 @@ package org.apache.spark.storage import java.nio.ByteBuffer /** - * Result of adding a block into a BlockStore. Contains its estimated size, and possibly the - * values put if the caller asked for them to be returned (e.g. for chaining replication) + * Result of adding a block into a BlockStore. This case class contains a few things: + * (1) The estimated size of the put, + * (2) The values put if the caller asked for them to be returned (e.g. for chaining + * replication), and + * (3) A list of blocks dropped as a result of this put. This is always empty for DiskStore. */ -private[spark] case class PutResult(size: Long, data: Either[Iterator[_], ByteBuffer]) +private[spark] case class PutResult( + size: Long, + data: Either[Iterator[_], ByteBuffer], + droppedBlocks: Seq[(BlockId, BlockStatus)] = Seq.empty) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala new file mode 100644 index 0000000000000..26565f56ad858 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -0,0 +1,94 @@ +/* + * 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.storage + +import scala.collection.mutable + +import org.apache.spark.scheduler._ + +/** + * A SparkListener that maintains executor storage status + */ +private[spark] class StorageStatusListener extends SparkListener { + private val executorIdToStorageStatus = mutable.Map[String, StorageStatus]() + + def storageStatusList = executorIdToStorageStatus.values.toSeq + + /** Update storage status list to reflect updated block statuses */ + def updateStorageStatus(execId: String, updatedBlocks: Seq[(BlockId, BlockStatus)]) { + val filteredStatus = storageStatusList.find(_.blockManagerId.executorId == execId) + filteredStatus.foreach { storageStatus => + updatedBlocks.foreach { case (blockId, updatedStatus) => + storageStatus.blocks(blockId) = updatedStatus + } + } + } + + /** Update storage status list to reflect the removal of an RDD from the cache */ + def updateStorageStatus(unpersistedRDDId: Int) { + storageStatusList.foreach { storageStatus => + val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId) + unpersistedBlocksIds.foreach { blockId => + storageStatus.blocks(blockId) = BlockStatus(StorageLevel.NONE, 0L, 0L) + } + } + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val info = taskEnd.taskInfo + val metrics = taskEnd.taskMetrics + if (info != null && metrics != null) { + val execId = formatExecutorId(info.executorId) + val updatedBlocks = metrics.updatedBlocks.getOrElse(Seq[(BlockId, BlockStatus)]()) + if (updatedBlocks.length > 0) { + updateStorageStatus(execId, updatedBlocks) + } + } + } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { + updateStorageStatus(unpersistRDD.rddId) + } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { + synchronized { + val blockManagerId = blockManagerAdded.blockManagerId + val executorId = blockManagerId.executorId + val maxMem = blockManagerAdded.maxMem + val storageStatus = new StorageStatus(blockManagerId, maxMem) + executorIdToStorageStatus(executorId) = storageStatus + } + } + + override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { + synchronized { + val executorId = blockManagerRemoved.blockManagerId.executorId + executorIdToStorageStatus.remove(executorId) + } + } + + /** + * In the local mode, there is a discrepancy between the executor ID according to the + * task ("localhost") and that according to SparkEnv (""). In the UI, this + * results in duplicate rows for the same executor. Thus, in this mode, we aggregate + * these two rows and use the executor ID of "" to be consistent. + */ + def formatExecutorId(execId: String): String = { + if (execId == "localhost") "" else execId + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 2d88a40fbb3f2..6153dfe0b7e13 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -17,13 +17,17 @@ package org.apache.spark.storage +import scala.collection.Map +import scala.collection.mutable + import org.apache.spark.SparkContext -import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus import org.apache.spark.util.Utils private[spark] -case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, - blocks: Map[BlockId, BlockStatus]) { +class StorageStatus( + val blockManagerId: BlockManagerId, + val maxMem: Long, + val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) @@ -43,14 +47,18 @@ case class StorageStatus(blockManagerId: BlockManagerId, maxMem: Long, } } -case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, - numCachedPartitions: Int, numPartitions: Int, memSize: Long, diskSize: Long) +private[spark] +class RDDInfo(val id: Int, val name: String, val numPartitions: Int, val storageLevel: StorageLevel) extends Ordered[RDDInfo] { + + var numCachedPartitions = 0 + var memSize = 0L + var diskSize = 0L + override def toString = { - import Utils.bytesToString ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, - numPartitions, bytesToString(memSize), bytesToString(diskSize)) + numPartitions, Utils.bytesToString(memSize), Utils.bytesToString(diskSize)) } override def compare(that: RDDInfo) = { @@ -62,55 +70,76 @@ case class RDDInfo(id: Int, name: String, storageLevel: StorageLevel, private[spark] object StorageUtils { - /* Returns RDD-level information, compiled from a list of StorageStatus objects */ - def rddInfoFromStorageStatus(storageStatusList: Seq[StorageStatus], - sc: SparkContext) : Array[RDDInfo] = { - rddInfoFromBlockStatusList( - storageStatusList.flatMap(_.rddBlocks).toMap[RDDBlockId, BlockStatus], sc) + /** + * Returns basic information of all RDDs persisted in the given SparkContext. This does not + * include storage information. + */ + def rddInfoFromSparkContext(sc: SparkContext): Array[RDDInfo] = { + sc.persistentRdds.values.map { rdd => + val rddName = Option(rdd.name).getOrElse(rdd.id.toString) + val rddNumPartitions = rdd.partitions.size + val rddStorageLevel = rdd.getStorageLevel + val rddInfo = new RDDInfo(rdd.id, rddName, rddNumPartitions, rddStorageLevel) + rddInfo + }.toArray } - /* Returns a map of blocks to their locations, compiled from a list of StorageStatus objects */ - def blockLocationsFromStorageStatus(storageStatusList: Seq[StorageStatus]) = { - val blockLocationPairs = storageStatusList - .flatMap(s => s.blocks.map(b => (b._1, s.blockManagerId.hostPort))) - blockLocationPairs.groupBy(_._1).map{case (k, v) => (k, v.unzip._2)}.toMap + /** Returns storage information of all RDDs persisted in the given SparkContext. */ + def rddInfoFromStorageStatus( + storageStatuses: Seq[StorageStatus], + sc: SparkContext): Array[RDDInfo] = { + rddInfoFromStorageStatus(storageStatuses, rddInfoFromSparkContext(sc)) } - /* Given a list of BlockStatus objets, returns information for each RDD */ - def rddInfoFromBlockStatusList(infos: Map[RDDBlockId, BlockStatus], - sc: SparkContext) : Array[RDDInfo] = { + /** Returns storage information of all RDDs in the given list. */ + def rddInfoFromStorageStatus( + storageStatuses: Seq[StorageStatus], + rddInfos: Seq[RDDInfo]): Array[RDDInfo] = { + + // Mapping from RDD ID -> an array of associated BlockStatuses + val blockStatusMap = storageStatuses.flatMap(_.rddBlocks).toMap + .groupBy { case (k, _) => k.rddId } + .mapValues(_.values.toArray) - // Group by rddId, ignore the partition name - val groupedRddBlocks = infos.groupBy { case(k, v) => k.rddId }.mapValues(_.values.toArray) + // Mapping from RDD ID -> the associated RDDInfo (with potentially outdated storage information) + val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap - // For each RDD, generate an RDDInfo object - val rddInfos = groupedRddBlocks.map { case (rddId, rddBlocks) => + val rddStorageInfos = blockStatusMap.flatMap { case (rddId, blocks) => // Add up memory and disk sizes - val memSize = rddBlocks.map(_.memSize).reduce(_ + _) - val diskSize = rddBlocks.map(_.diskSize).reduce(_ + _) - - // Get the friendly name and storage level for the RDD, if available - sc.persistentRdds.get(rddId).map { r => - val rddName = Option(r.name).getOrElse(rddId.toString) - val rddStorageLevel = r.getStorageLevel - RDDInfo(rddId, rddName, rddStorageLevel, rddBlocks.length, r.partitions.size, - memSize, diskSize) + val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize > 0 } + val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + rddInfoMap.get(rddId).map { rddInfo => + rddInfo.numCachedPartitions = persistedBlocks.length + rddInfo.memSize = memSize + rddInfo.diskSize = diskSize + rddInfo } - }.flatten.toArray + }.toArray - scala.util.Sorting.quickSort(rddInfos) - - rddInfos + scala.util.Sorting.quickSort(rddStorageInfos) + rddStorageInfos } - /* Filters storage status by a given RDD id. */ - def filterStorageStatusByRDD(storageStatusList: Array[StorageStatus], rddId: Int) - : Array[StorageStatus] = { - - storageStatusList.map { status => - val newBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toMap[BlockId, BlockStatus] - //val newRemainingMem = status.maxMem - newBlocks.values.map(_.memSize).reduce(_ + _) - StorageStatus(status.blockManagerId, status.maxMem, newBlocks) + /** Returns a mapping from BlockId to the locations of the associated block. */ + def blockLocationsFromStorageStatus( + storageStatuses: Seq[StorageStatus]): Map[BlockId, Seq[String]] = { + val blockLocationPairs = storageStatuses.flatMap { storageStatus => + storageStatus.blocks.map { case (bid, _) => (bid, storageStatus.blockManagerId.hostPort) } } + blockLocationPairs.toMap + .groupBy { case (blockId, _) => blockId } + .mapValues(_.values.toSeq) + } + + /** Filters the given list of StorageStatus by the given RDD ID. */ + def filterStorageStatusByRDD( + storageStatuses: Seq[StorageStatus], + rddId: Int): Array[StorageStatus] = { + storageStatuses.map { status => + val filteredBlocks = status.rddBlocks.filterKeys(_.rddId == rddId).toSeq + val filteredBlockMap = mutable.Map[BlockId, BlockStatus](filteredBlocks: _*) + new StorageStatus(status.blockManagerId, status.maxMem, filteredBlockMap) + }.toArray } } diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 36f2a0fd02724..226ed2a132b00 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,9 +22,9 @@ import java.util.concurrent.ArrayBlockingQueue import akka.actor._ import util.Random -import org.apache.spark.SparkConf -import org.apache.spark.serializer.KryoSerializer import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.scheduler.LiveListenerBus +import org.apache.spark.serializer.KryoSerializer /** * This class tests the BlockManager and MemoryStore for thread safety and @@ -97,7 +97,8 @@ private[spark] object ThreadingTest { val conf = new SparkConf() val serializer = new KryoSerializer(conf) val blockManagerMaster = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, new SecurityManager(conf)) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index e0555ca7ac02f..6e1736f6fbc23 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -17,30 +17,29 @@ package org.apache.spark.ui -import java.net.InetSocketAddress -import java.net.URL -import javax.servlet.http.{HttpServlet, HttpServletResponse, HttpServletRequest} +import java.net.{InetSocketAddress, URL} +import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.annotation.tailrec import scala.util.{Failure, Success, Try} import scala.xml.Node -import org.json4s.JValue -import org.json4s.jackson.JsonMethods.{pretty, render} - import org.eclipse.jetty.server.{DispatcherType, Server} -import org.eclipse.jetty.server.handler.HandlerList -import org.eclipse.jetty.servlet.{DefaultServlet, FilterHolder, ServletContextHandler, ServletHolder} +import org.eclipse.jetty.server.handler._ +import org.eclipse.jetty.servlet._ import org.eclipse.jetty.util.thread.QueuedThreadPool +import org.json4s.JValue +import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{Logging, SecurityManager, SparkConf} - -/** Utilities for launching a web server using Jetty's HTTP Server class */ +/** + * Utilities for launching a web server using Jetty's HTTP Server class + */ private[spark] object JettyUtils extends Logging { + // Base type for a function that returns something based on an HTTP request. Allows for // implicit conversion from many types of functions to jetty Handlers. - type Responder[T] = HttpServletRequest => T class ServletParams[T <% AnyRef](val responder: Responder[T], @@ -57,62 +56,73 @@ private[spark] object JettyUtils extends Logging { implicit def textResponderToServlet(responder: Responder[String]): ServletParams[String] = new ServletParams(responder, "text/plain") - def createServlet[T <% AnyRef](servletParams: ServletParams[T], + def createServlet[T <% AnyRef]( + servletParams: ServletParams[T], securityMgr: SecurityManager): HttpServlet = { new HttpServlet { - override def doGet(request: HttpServletRequest, - response: HttpServletResponse) { - if (securityMgr.checkUIViewPermissions(request.getRemoteUser())) { + override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + if (securityMgr.checkUIViewPermissions(request.getRemoteUser)) { response.setContentType("%s;charset=utf-8".format(servletParams.contentType)) response.setStatus(HttpServletResponse.SC_OK) val result = servletParams.responder(request) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") - response.getWriter().println(servletParams.extractFn(result)) + response.getWriter.println(servletParams.extractFn(result)) } else { response.setStatus(HttpServletResponse.SC_UNAUTHORIZED) response.setHeader("Cache-Control", "no-cache, no-store, must-revalidate") response.sendError(HttpServletResponse.SC_UNAUTHORIZED, - "User is not authorized to access this page."); + "User is not authorized to access this page.") } } } } - def createServletHandler(path: String, servlet: HttpServlet): ServletContextHandler = { - val contextHandler = new ServletContextHandler() + /** Create a context handler that responds to a request with the given path prefix */ + def createServletHandler[T <% AnyRef]( + path: String, + servletParams: ServletParams[T], + securityMgr: SecurityManager, + basePath: String = ""): ServletContextHandler = { + createServletHandler(path, createServlet(servletParams, securityMgr), basePath) + } + + /** Create a context handler that responds to a request with the given path prefix */ + def createServletHandler( + path: String, + servlet: HttpServlet, + basePath: String = ""): ServletContextHandler = { + val prefixedPath = attachPrefix(basePath, path) + val contextHandler = new ServletContextHandler val holder = new ServletHolder(servlet) - contextHandler.setContextPath(path) + contextHandler.setContextPath(prefixedPath) contextHandler.addServlet(holder, "/") contextHandler } - /** Creates a handler that always redirects the user to a given path */ - def createRedirectHandler(newPath: String, path: String): ServletContextHandler = { + /** Create a handler that always redirects the user to the given path */ + def createRedirectHandler( + srcPath: String, + destPath: String, + basePath: String = ""): ServletContextHandler = { + val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { - override def doGet(request: HttpServletRequest, - response: HttpServletResponse) { - // make sure we don't end up with // in the middle - val newUri = new URL(new URL(request.getRequestURL.toString), newPath).toURI - response.sendRedirect(newUri.toString) + override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + // Make sure we don't end up with "//" in the middle + val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString + response.sendRedirect(newUrl) } } - val contextHandler = new ServletContextHandler() - val holder = new ServletHolder(servlet) - contextHandler.setContextPath(path) - contextHandler.addServlet(holder, "/") - contextHandler + createServletHandler(srcPath, servlet, basePath) } - /** Creates a handler for serving files from a static directory */ + /** Create a handler for serving files from a static directory */ def createStaticHandler(resourceBase: String, path: String): ServletContextHandler = { - val contextHandler = new ServletContextHandler() + val contextHandler = new ServletContextHandler val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) Option(getClass.getClassLoader.getResource(resourceBase)) match { case Some(res) => holder.setInitParameter("resourceBase", res.toString) - holder.setInitParameter("welcomeServlets", "false") - holder.setInitParameter("pathInfoOnly", "false") case None => throw new Exception("Could not find resource path for Web UI: " + resourceBase) } @@ -121,6 +131,7 @@ private[spark] object JettyUtils extends Logging { contextHandler } + /** Add security filters, if any, do the given list of ServletContextHandlers */ private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) filters.foreach { @@ -129,7 +140,7 @@ private[spark] object JettyUtils extends Logging { logInfo("Adding filter: " + filter) val holder : FilterHolder = new FilterHolder() holder.setClassName(filter) - // get any parameters for each filter + // Get any parameters for each filter val paramName = "spark." + filter + ".params" val params = conf.get(paramName, "").split(',').map(_.trim()).toSet params.foreach { @@ -147,18 +158,21 @@ private[spark] object JettyUtils extends Logging { } /** - * Attempts to start a Jetty server at the supplied hostName:port which uses the supplied - * handlers. + * Attempt to start a Jetty server bound to the supplied hostName:port using the given + * context handlers. * - * If the desired port number is contented, continues incrementing ports until a free port is - * found. Returns the chosen port and the jetty Server object. + * If the desired port number is contended, continues incrementing ports until a free port is + * found. Return the jetty Server object, the chosen port, and a mutable collection of handlers. */ - def startJettyServer(hostName: String, port: Int, handlers: Seq[ServletContextHandler], - conf: SparkConf): (Server, Int) = { - + def startJettyServer( + hostName: String, + port: Int, + handlers: Seq[ServletContextHandler], + conf: SparkConf): ServerInfo = { + + val collection = new ContextHandlerCollection + collection.setHandlers(handlers.toArray) addFilters(handlers, conf) - val handlerList = new HandlerList - handlerList.setHandlers(handlers.toArray) @tailrec def connect(currentPort: Int): (Server, Int) = { @@ -166,7 +180,7 @@ private[spark] object JettyUtils extends Logging { val pool = new QueuedThreadPool pool.setDaemon(true) server.setThreadPool(pool) - server.setHandler(handlerList) + server.setHandler(collection) Try { server.start() @@ -181,6 +195,17 @@ private[spark] object JettyUtils extends Logging { } } - connect(port) + val (server, boundPort) = connect(port) + ServerInfo(server, boundPort, collection) + } + + /** Attach a prefix to the given path, but avoid returning an empty path */ + private def attachPrefix(basePath: String, relativePath: String): String = { + if (basePath == "") relativePath else (basePath + relativePath).stripSuffix("/") } } + +private[spark] case class ServerInfo( + server: Server, + boundPort: Int, + rootHandler: ContextHandlerCollection) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 5f0dee64fedb7..fd638c83aac6e 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,12 +17,11 @@ package org.apache.spark.ui -import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.{Logging, SparkContext, SparkEnv} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.scheduler._ +import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.env.EnvironmentUI import org.apache.spark.ui.exec.ExecutorsUI @@ -31,34 +30,57 @@ import org.apache.spark.ui.storage.BlockManagerUI import org.apache.spark.util.Utils /** Top level user interface for Spark */ -private[spark] class SparkUI(sc: SparkContext) extends Logging { - val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) - val port = sc.conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt - var boundPort: Option[Int] = None - var server: Option[Server] = None - - val handlers = Seq[ServletContextHandler] ( - createStaticHandler(SparkUI.STATIC_RESOURCE_DIR + "/static", "/static"), - createRedirectHandler("/stages", "/") - ) - val storage = new BlockManagerUI(sc) - val jobs = new JobProgressUI(sc) - val env = new EnvironmentUI(sc) - val exec = new ExecutorsUI(sc) - - // Add MetricsServlet handlers by default - val metricsServletHandlers = SparkEnv.get.metricsSystem.getServletHandlers - - val allHandlers = storage.getHandlers ++ jobs.getHandlers ++ env.getHandlers ++ - exec.getHandlers ++ metricsServletHandlers ++ handlers +private[spark] class SparkUI( + val sc: SparkContext, + conf: SparkConf, + val listenerBus: SparkListenerBus, + val appName: String, + val basePath: String = "") + extends Logging { + + def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) + def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, conf, listenerBus, appName, basePath) + + // If SparkContext is not provided, assume the associated application is not live + val live = sc != null + + val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) + + private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) + private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt + private var serverInfo: Option[ServerInfo] = None + + private val storage = new BlockManagerUI(this) + private val jobs = new JobProgressUI(this) + private val env = new EnvironmentUI(this) + private val exec = new ExecutorsUI(this) + + val handlers: Seq[ServletContextHandler] = { + val metricsServletHandlers = if (live) { + SparkEnv.get.metricsSystem.getServletHandlers + } else { + Array[ServletContextHandler]() + } + storage.getHandlers ++ + jobs.getHandlers ++ + env.getHandlers ++ + exec.getHandlers ++ + metricsServletHandlers ++ + Seq[ServletContextHandler] ( + createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"), + createRedirectHandler("/", "/stages", basePath) + ) + } + + // Maintain executor storage status through Spark events + val storageStatusListener = new StorageStatusListener /** Bind the HTTP server which backs this web interface */ def bind() { try { - val (srv, usedPort) = JettyUtils.startJettyServer(host, port, allHandlers, sc.conf) - logInfo("Started Spark Web UI at http://%s:%d".format(host, usedPort)) - server = Some(srv) - boundPort = Some(usedPort) + serverInfo = Some(startJettyServer(host, port, handlers, sc.conf)) + logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => logError("Failed to create Spark JettyUtils", e) @@ -66,25 +88,34 @@ private[spark] class SparkUI(sc: SparkContext) extends Logging { } } + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + /** Initialize all components of the server */ def start() { - // NOTE: This is decoupled from bind() because of the following dependency cycle: - // DAGScheduler() requires that the port of this server is known - // This server must register all handlers, including JobProgressUI, before binding - // JobProgressUI registers a listener with SparkContext, which requires sc to initialize + storage.start() jobs.start() + env.start() exec.start() + + // Storage status listener must receive events first, as other listeners depend on its state + listenerBus.addListener(storageStatusListener) + listenerBus.addListener(storage.listener) + listenerBus.addListener(jobs.listener) + listenerBus.addListener(env.listener) + listenerBus.addListener(exec.listener) } def stop() { - server.foreach(_.stop()) + assert(serverInfo.isDefined, "Attempted to stop a SparkUI that was not bound to a server!") + serverInfo.get.server.stop() + logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = host + ":" + boundPort.getOrElse("-1") + private[spark] def appUIAddress = "http://" + host + ":" + boundPort } private[spark] object SparkUI { val DEFAULT_PORT = "4040" - val STATIC_RESOURCE_DIR = "org/apache/spark/ui" + val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 547a194d58a5c..a487924effbff 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -19,38 +19,43 @@ package org.apache.spark.ui import scala.xml.Node -import org.apache.spark.SparkContext - /** Utility functions for generating XML pages with spark content. */ private[spark] object UIUtils { + import Page._ // Yarn has to go through a proxy so the base uri is provided and has to be on all links private[spark] val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")). getOrElse("") - def prependBaseUri(resource: String = "") = uiRoot + resource + def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource /** Returns a spark page with correctly formatted headers */ - def headerSparkPage(content: => Seq[Node], sc: SparkContext, title: String, page: Page.Value) - : Seq[Node] = { + def headerSparkPage( + content: => Seq[Node], + basePath: String, + appName: String, + title: String, + page: Page.Value) : Seq[Node] = { val jobs = page match { - case Stages =>
  • Stages
  • - case _ =>
  • Stages
  • + case Stages => +
  • Stages
  • + case _ =>
  • Stages
  • } val storage = page match { - case Storage =>
  • Storage
  • - case _ =>
  • Storage
  • + case Storage => +
  • Storage
  • + case _ =>
  • Storage
  • } val environment = page match { case Environment => -
  • Environment
  • - case _ =>
  • Environment
  • +
  • Environment
  • + case _ =>
  • Environment
  • } val executors = page match { case Executors => -
  • Executors
  • - case _ =>
  • Executors
  • +
  • Executors
  • + case _ =>
  • Executors
  • } @@ -58,14 +63,15 @@ private[spark] object UIUtils { - + - {sc.appName} - {title} + {appName} - {title} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 14333476c0e31..23e90c34d5b33 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -19,76 +19,74 @@ package org.apache.spark.ui.env import javax.servlet.http.HttpServletRequest -import scala.collection.JavaConversions._ -import scala.util.Properties import scala.xml.Node import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.SparkContext +import org.apache.spark.scheduler._ +import org.apache.spark.ui._ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Environment -import org.apache.spark.ui.UIUtils -private[spark] class EnvironmentUI(sc: SparkContext) { +private[ui] class EnvironmentUI(parent: SparkUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private var _listener: Option[EnvironmentListener] = None + + lazy val listener = _listener.get + + def start() { + _listener = Some(new EnvironmentListener) + } def getHandlers = Seq[ServletContextHandler]( createServletHandler("/environment", - createServlet((request: HttpServletRequest) => envDetails(request), sc.env.securityManager)) + (request: HttpServletRequest) => render(request), parent.securityManager, basePath) ) - def envDetails(request: HttpServletRequest): Seq[Node] = { - val jvmInformation = Seq( - ("Java Version", "%s (%s)".format(Properties.javaVersion, Properties.javaVendor)), - ("Java Home", Properties.javaHome), - ("Scala Version", Properties.versionString), - ("Scala Home", Properties.scalaHome) - ).sorted - def jvmRow(kv: (String, String)) = {kv._1}{kv._2} - def jvmTable = - UIUtils.listingTable(Seq("Name", "Value"), jvmRow, jvmInformation, fixedWidth = true) - - val sparkProperties = sc.conf.getAll.sorted - - val systemProperties = System.getProperties.iterator.toSeq - val classPathProperty = systemProperties.find { case (k, v) => - k == "java.class.path" - }.getOrElse(("", "")) - val otherProperties = systemProperties.filter { case (k, v) => - k != "java.class.path" && !k.startsWith("spark.") - }.sorted - - val propertyHeaders = Seq("Name", "Value") - def propertyRow(kv: (String, String)) = {kv._1}{kv._2} - val sparkPropertyTable = - UIUtils.listingTable(propertyHeaders, propertyRow, sparkProperties, fixedWidth = true) - val otherPropertyTable = - UIUtils.listingTable(propertyHeaders, propertyRow, otherProperties, fixedWidth = true) - - val classPathEntries = classPathProperty._2 - .split(sc.conf.get("path.separator", ":")) - .filterNot(e => e.isEmpty) - .map(e => (e, "System Classpath")) - val addedJars = sc.addedJars.iterator.toSeq.map{case (path, time) => (path, "Added By User")} - val addedFiles = sc.addedFiles.iterator.toSeq.map{case (path, time) => (path, "Added By User")} - val classPath = (addedJars ++ addedFiles ++ classPathEntries).sorted - - val classPathHeaders = Seq("Resource", "Source") - def classPathRow(data: (String, String)) = {data._1}{data._2} - val classPathTable = - UIUtils.listingTable(classPathHeaders, classPathRow, classPath, fixedWidth = true) - + def render(request: HttpServletRequest): Seq[Node] = { + val runtimeInformationTable = UIUtils.listingTable( + propertyHeader, jvmRow, listener.jvmInformation, fixedWidth = true) + val sparkPropertiesTable = UIUtils.listingTable( + propertyHeader, propertyRow, listener.sparkProperties, fixedWidth = true) + val systemPropertiesTable = UIUtils.listingTable( + propertyHeader, propertyRow, listener.systemProperties, fixedWidth = true) + val classpathEntriesTable = UIUtils.listingTable( + classPathHeaders, classPathRow, listener.classpathEntries, fixedWidth = true) val content = -

    Runtime Information

    {jvmTable} -

    Spark Properties

    - {sparkPropertyTable} -

    System Properties

    - {otherPropertyTable} -

    Classpath Entries

    - {classPathTable} +

    Runtime Information

    {runtimeInformationTable} +

    Spark Properties

    {sparkPropertiesTable} +

    System Properties

    {systemPropertiesTable} +

    Classpath Entries

    {classpathEntriesTable}
    - UIUtils.headerSparkPage(content, sc, "Environment", Environment) + UIUtils.headerSparkPage(content, basePath, appName, "Environment", Environment) + } + + private def propertyHeader = Seq("Name", "Value") + private def classPathHeaders = Seq("Resource", "Source") + private def jvmRow(kv: (String, String)) = {kv._1}{kv._2} + private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} + private def classPathRow(data: (String, String)) = {data._1}{data._2} +} + +/** + * A SparkListener that prepares information to be displayed on the EnvironmentUI + */ +private[ui] class EnvironmentListener extends SparkListener { + var jvmInformation = Seq[(String, String)]() + var sparkProperties = Seq[(String, String)]() + var systemProperties = Seq[(String, String)]() + var classpathEntries = Seq[(String, String)]() + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + jvmInformation = environmentDetails("JVM Information") + sparkProperties = environmentDetails("Spark Properties") + systemProperties = environmentDetails("System Properties") + classpathEntries = environmentDetails("Classpath Entries") + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 4235cfeff9fa2..031ed88a493a8 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -19,69 +19,43 @@ package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.{HashMap, HashSet} +import scala.collection.mutable.HashMap import scala.xml.Node import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.{ExceptionFailure, Logging, SparkContext} -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskEnd, SparkListenerTaskStart} -import org.apache.spark.scheduler.TaskInfo +import org.apache.spark.ExceptionFailure +import org.apache.spark.scheduler._ +import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{SparkUI, UIUtils} import org.apache.spark.util.Utils -private[spark] class ExecutorsUI(val sc: SparkContext) { - +private[ui] class ExecutorsUI(parent: SparkUI) { + private val appName = parent.appName + private val basePath = parent.basePath private var _listener: Option[ExecutorsListener] = None - def listener = _listener.get + + lazy val listener = _listener.get def start() { - _listener = Some(new ExecutorsListener) - sc.addSparkListener(listener) + _listener = Some(new ExecutorsListener(parent.storageStatusListener)) } def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/executors", createServlet((request: HttpServletRequest) => render - (request), sc.env.securityManager)) + createServletHandler("/executors", + (request: HttpServletRequest) => render(request), parent.securityManager, basePath) ) def render(request: HttpServletRequest): Seq[Node] = { - val storageStatusList = sc.getExecutorStorageStatus - + val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) - - val execHead = Seq("Executor ID", "Address", "RDD blocks", "Memory used", "Disk used", - "Active tasks", "Failed tasks", "Complete tasks", "Total tasks", "Task Time", "Shuffle Read", - "Shuffle Write") - - def execRow(kv: Seq[String]) = { - - {kv(0)} - {kv(1)} - {kv(2)} - - {Utils.bytesToString(kv(3).toLong)} / {Utils.bytesToString(kv(4).toLong)} - - - {Utils.bytesToString(kv(5).toLong)} - - {kv(6)} - {kv(7)} - {kv(8)} - {kv(9)} - {Utils.msDurationToString(kv(10).toLong)} - {Utils.bytesToString(kv(11).toLong)} - {Utils.bytesToString(kv(12).toLong)} - - } - val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) - val execTable = UIUtils.listingTable(execHead, execRow, execInfo) + val execInfoSorted = execInfo.sortBy(_.getOrElse("Executor ID", "")) + val execTable = UIUtils.listingTable(execHeader, execRow, execInfoSorted) val content =
    @@ -100,18 +74,61 @@ private[spark] class ExecutorsUI(val sc: SparkContext) {
    ; - UIUtils.headerSparkPage(content, sc, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage( + content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) + } + + /** Header fields for the executors table */ + private def execHeader = Seq( + "Executor ID", + "Address", + "RDD Blocks", + "Memory Used", + "Disk Used", + "Active Tasks", + "Failed Tasks", + "Complete Tasks", + "Total Tasks", + "Task Time", + "Shuffle Read", + "Shuffle Write") + + /** Render an HTML row representing an executor */ + private def execRow(values: Map[String, String]): Seq[Node] = { + val maximumMemory = values("Maximum Memory") + val memoryUsed = values("Memory Used") + val diskUsed = values("Disk Used") + + {values("Executor ID")} + {values("Address")} + {values("RDD Blocks")} + + {Utils.bytesToString(memoryUsed.toLong)} / + {Utils.bytesToString(maximumMemory.toLong)} + + + {Utils.bytesToString(diskUsed.toLong)} + + {values("Active Tasks")} + {values("Failed Tasks")} + {values("Complete Tasks")} + {values("Total Tasks")} + {Utils.msDurationToString(values("Task Time").toLong)} + {Utils.bytesToString(values("Shuffle Read").toLong)} + {Utils.bytesToString(values("Shuffle Write").toLong)} + } - def getExecInfo(statusId: Int): Seq[String] = { - val status = sc.getExecutorStorageStatus(statusId) + /** Represent an executor's info as a map given a storage status index */ + private def getExecInfo(statusId: Int): Map[String, String] = { + val status = listener.storageStatusList(statusId) val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort - val rddBlocks = status.blocks.size.toString - val memUsed = status.memUsed().toString - val maxMem = status.maxMem.toString - val diskUsed = status.diskUsed().toString - val activeTasks = listener.executorToTasksActive.getOrElse(execId, HashSet.empty[Long]).size + val rddBlocks = status.blocks.size + val memUsed = status.memUsed() + val maxMem = status.maxMem + val diskUsed = status.diskUsed() + val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0) val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) val totalTasks = activeTasks + failedTasks + completedTasks @@ -119,64 +136,77 @@ private[spark] class ExecutorsUI(val sc: SparkContext) { val totalShuffleRead = listener.executorToShuffleRead.getOrElse(execId, 0) val totalShuffleWrite = listener.executorToShuffleWrite.getOrElse(execId, 0) - Seq( + // Also include fields not in the header + val execFields = execHeader ++ Seq("Maximum Memory") + + val execValues = Seq( execId, hostPort, rddBlocks, memUsed, - maxMem, diskUsed, - activeTasks.toString, - failedTasks.toString, - completedTasks.toString, - totalTasks.toString, - totalDuration.toString, - totalShuffleRead.toString, - totalShuffleWrite.toString - ) + activeTasks, + failedTasks, + completedTasks, + totalTasks, + totalDuration, + totalShuffleRead, + totalShuffleWrite, + maxMem + ).map(_.toString) + + execFields.zip(execValues).toMap } +} - private[spark] class ExecutorsListener extends SparkListener with Logging { - val executorToTasksActive = HashMap[String, HashSet[TaskInfo]]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - - override def onTaskStart(taskStart: SparkListenerTaskStart) { - val eid = taskStart.taskInfo.executorId - val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) - activeTasks += taskStart.taskInfo - } +/** + * A SparkListener that prepares information to be displayed on the ExecutorsUI + */ +private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) + extends SparkListener { - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { - val eid = taskEnd.taskInfo.executorId - val activeTasks = executorToTasksActive.getOrElseUpdate(eid, new HashSet[TaskInfo]()) - val newDuration = executorToDuration.getOrElse(eid, 0L) + taskEnd.taskInfo.duration - executorToDuration.put(eid, newDuration) - - activeTasks -= taskEnd.taskInfo - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - (Some(e), e.metrics) - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - (None, Option(taskEnd.taskMetrics)) - } + val executorToTasksActive = HashMap[String, Int]() + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToDuration = HashMap[String, Long]() + val executorToShuffleRead = HashMap[String, Long]() + val executorToShuffleWrite = HashMap[String, Long]() + + def storageStatusList = storageStatusListener.storageStatusList + + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + val eid = formatExecutorId(taskStart.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + } - // update shuffle read/write - if (null != taskEnd.taskMetrics) { - taskEnd.taskMetrics.shuffleReadMetrics.foreach(shuffleRead => - executorToShuffleRead.put(eid, executorToShuffleRead.getOrElse(eid, 0L) + - shuffleRead.remoteBytesRead)) + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val info = taskEnd.taskInfo + if (info != null) { + val eid = formatExecutorId(info.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 + executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + } - taskEnd.taskMetrics.shuffleWriteMetrics.foreach(shuffleWrite => - executorToShuffleWrite.put(eid, executorToShuffleWrite.getOrElse(eid, 0L) + - shuffleWrite.shuffleBytesWritten)) + // Update shuffle read/write + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { shuffleRead => + executorToShuffleRead(eid) = + executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead + } + metrics.shuffleWriteMetrics.foreach { shuffleWrite => + executorToShuffleWrite(eid) = + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + } } } } + + // This addresses executor ID inconsistencies in the local mode + private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala index 64e22a30b48f9..1dfe1d4f1fa11 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorSummary.scala @@ -18,7 +18,7 @@ package org.apache.spark.ui.jobs /** class for reporting aggregated metrics for each executors in stageUI */ -private[spark] class ExecutorSummary { +private[ui] class ExecutorSummary { var taskTime : Long = 0 var failedTasks : Int = 0 var succeededTasks : Int = 0 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index d012ba4dbb3db..73861ae6746da 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -20,17 +20,13 @@ package org.apache.spark.ui.jobs import scala.collection.mutable import scala.xml.Node -import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.util.Utils /** Page showing executor summary */ -private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) { +private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { + private lazy val listener = parent.listener - val listener = parent.listener - val dateFmt = parent.dateFmt - val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR - - def toNodeSeq(): Seq[Node] = { + def toNodeSeq: Seq[Node] = { listener.synchronized { executorTable() } @@ -58,11 +54,9 @@ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) } private def createExecutorTable() : Seq[Node] = { - // make a executor-id -> address map + // Make an executor-id -> address map val executorIdToAddress = mutable.HashMap[String, String]() - val storageStatusList = parent.sc.getExecutorStorageStatus - for (statusId <- 0 until storageStatusList.size) { - val blockManagerId = parent.sc.getExecutorStorageStatus(statusId).blockManagerId + listener.blockManagerIds.foreach { blockManagerId => val address = blockManagerId.hostPort val executorId = blockManagerId.executorId executorIdToAddress.put(executorId, address) @@ -70,25 +64,23 @@ private[spark] class ExecutorTable(val parent: JobProgressUI, val stageId: Int) val executorIdToSummary = listener.stageIdToExecutorSummaries.get(stageId) executorIdToSummary match { - case Some(x) => { - x.toSeq.sortBy(_._1).map{ - case (k,v) => { - - {k} - {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} - {parent.formatDuration(v.taskTime)} - {v.failedTasks + v.succeededTasks} - {v.failedTasks} - {v.succeededTasks} - {Utils.bytesToString(v.shuffleRead)} - {Utils.bytesToString(v.shuffleWrite)} - {Utils.bytesToString(v.memoryBytesSpilled)} - {Utils.bytesToString(v.diskBytesSpilled)} - - } + case Some(x) => + x.toSeq.sortBy(_._1).map { case (k, v) => { + + {k} + {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} + {parent.formatDuration(v.taskTime)} + {v.failedTasks + v.succeededTasks} + {v.failedTasks} + {v.succeededTasks} + {Utils.bytesToString(v.shuffleRead)} + {Utils.bytesToString(v.shuffleWrite)} + {Utils.bytesToString(v.memoryBytesSpilled)} + {Utils.bytesToString(v.diskBytesSpilled)} + } } - case _ => { Seq[Node]() } + case _ => Seq[Node]() } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 81713edcf5db2..f3c93d4214ad0 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -19,72 +19,80 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import scala.xml.{NodeSeq, Node} +import scala.xml.{Node, NodeSeq} -import org.apache.spark.scheduler.SchedulingMode +import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools*/ -private[spark] class IndexPage(parent: JobProgressUI) { - def listener = parent.listener +private[ui] class IndexPage(parent: JobProgressUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private val live = parent.live + private val sc = parent.sc + private lazy val listener = parent.listener + private lazy val isFairScheduler = parent.isFairScheduler def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { - val activeStages = listener.activeStages.toSeq + val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq val now = System.currentTimeMillis() - var activeTime = 0L - for (tasks <- listener.stageIdToTasksActive.values; t <- tasks) { - activeTime += t.timeRunning(now) - } - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) - val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, - parent) + val completedStagesTable = + new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) - val pools = listener.sc.getAllPools - val poolTable = new PoolTable(pools, listener) + // For now, pool information is only accessible in live UIs + val pools = if (live) sc.getAllPools else Seq[Schedulable]() + val poolTable = new PoolTable(pools, parent) + val summary: NodeSeq = -
    -
      -
    • - Total Duration: - {parent.formatDuration(now - listener.sc.startTime)} -
    • -
    • Scheduling Mode: {parent.sc.getSchedulingMode}
    • -
    • - Active Stages: - {activeStages.size} -
    • -
    • - Completed Stages: - {completedStages.size} -
    • -
    • +
      +
        + {if (live) { + // Total duration is not meaningful unless the UI is live +
      • + Total Duration: + {parent.formatDuration(now - sc.startTime)} +
      • + }} +
      • + Scheduling Mode: + {listener.schedulingMode.map(_.toString).getOrElse("Unknown")} +
      • +
      • + Active Stages: + {activeStages.size} +
      • +
      • + Completed Stages: + {completedStages.size} +
      • +
      • Failed Stages: - {failedStages.size} -
      • -
      -
      + {failedStages.size} +
    • +
    +
    val content = summary ++ - {if (listener.sc.getSchedulingMode == SchedulingMode.FAIR) { -

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq + {if (live && isFairScheduler) { +

    {pools.size} Fair Scheduler Pools

    ++ poolTable.toNodeSeq } else { - Seq() + Seq[Node]() }} ++

    Active Stages ({activeStages.size})

    ++ - activeStagesTable.toNodeSeq++ + activeStagesTable.toNodeSeq ++

    Completed Stages ({completedStages.size})

    ++ - completedStagesTable.toNodeSeq++ + completedStagesTable.toNodeSeq ++

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - headerSparkPage(content, parent.sc, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 07a08f5277d19..d10aa12b9ebca 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -17,29 +17,29 @@ package org.apache.spark.ui.jobs -import scala.collection.mutable.{ListBuffer, HashMap, HashSet} +import scala.collection.mutable.{HashMap, ListBuffer} -import org.apache.spark.{ExceptionFailure, SparkContext, Success} +import org.apache.spark.{ExceptionFailure, SparkConf, SparkContext, Success} import org.apache.spark.executor.TaskMetrics import org.apache.spark.scheduler._ +import org.apache.spark.scheduler.SchedulingMode.SchedulingMode +import org.apache.spark.storage.BlockManagerId /** * Tracks task-level information to be displayed in the UI. * * All access to the data structures in this class must be synchronized on the - * class, since the UI thread and the DAGScheduler event loop may otherwise - * be reading/updating the internal data structures concurrently. + * class, since the UI thread and the EventBus loop may otherwise be reading and + * updating the internal data structures concurrently. */ -private[spark] class JobProgressListener(val sc: SparkContext) extends SparkListener { - // How many stages to remember - val RETAINED_STAGES = sc.conf.getInt("spark.ui.retainedStages", 1000) - val DEFAULT_POOL_NAME = "default" +private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { - val stageIdToPool = new HashMap[Int, String]() - val stageIdToDescription = new HashMap[Int, String]() - val poolToActiveStages = new HashMap[String, HashSet[StageInfo]]() + import JobProgressListener._ - val activeStages = HashSet[StageInfo]() + // How many stages to remember + val retainedStages = conf.getInt("spark.ui.retainedStages", DEFAULT_RETAINED_STAGES) + + val activeStages = HashMap[Int, StageInfo]() val completedStages = ListBuffer[StageInfo]() val failedStages = ListBuffer[StageInfo]() @@ -53,29 +53,37 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList val stageIdToShuffleWrite = HashMap[Int, Long]() val stageIdToMemoryBytesSpilled = HashMap[Int, Long]() val stageIdToDiskBytesSpilled = HashMap[Int, Long]() - val stageIdToTasksActive = HashMap[Int, HashSet[TaskInfo]]() + val stageIdToTasksActive = HashMap[Int, HashMap[Long, TaskInfo]]() val stageIdToTasksComplete = HashMap[Int, Int]() val stageIdToTasksFailed = HashMap[Int, Int]() - val stageIdToTaskInfos = - HashMap[Int, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]]() + val stageIdToTaskData = HashMap[Int, HashMap[Long, TaskUIData]]() val stageIdToExecutorSummaries = HashMap[Int, HashMap[String, ExecutorSummary]]() + val stageIdToPool = HashMap[Int, String]() + val stageIdToDescription = HashMap[Int, String]() + val poolToActiveStages = HashMap[String, HashMap[Int, StageInfo]]() + + val executorIdToBlockManagerId = HashMap[String, BlockManagerId]() - override def onJobStart(jobStart: SparkListenerJobStart) {} + var schedulingMode: Option[SchedulingMode] = None + + def blockManagerIds = executorIdToBlockManagerId.values.toSeq override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { - val stage = stageCompleted.stage - poolToActiveStages(stageIdToPool(stage.stageId)) -= stage - activeStages -= stage + val stage = stageCompleted.stageInfo + val stageId = stage.stageId + // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage + poolToActiveStages(stageIdToPool(stageId)).remove(stageId) + activeStages.remove(stageId) completedStages += stage trimIfNecessary(completedStages) } /** If stages is too large, remove and garbage collect old stages */ - def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { - if (stages.size > RETAINED_STAGES) { - val toRemove = RETAINED_STAGES / 10 + private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { + if (stages.size > retainedStages) { + val toRemove = retainedStages / 10 stages.takeRight(toRemove).foreach( s => { - stageIdToTaskInfos.remove(s.stageId) + stageIdToTaskData.remove(s.stageId) stageIdToTime.remove(s.stageId) stageIdToShuffleRead.remove(s.stageId) stageIdToShuffleWrite.remove(s.stageId) @@ -93,8 +101,8 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList /** For FIFO, all stages are contained by "default" pool but "default" pool here is meaningless */ override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { - val stage = stageSubmitted.stage - activeStages += stage + val stage = stageSubmitted.stageInfo + activeStages(stage.stageId) = stage val poolName = Option(stageSubmitted.properties).map { p => p.getProperty("spark.scheduler.pool", DEFAULT_POOL_NAME) @@ -106,121 +114,154 @@ private[spark] class JobProgressListener(val sc: SparkContext) extends SparkList } description.map(d => stageIdToDescription(stage.stageId) = d) - val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashSet[StageInfo]()) - stages += stage + val stages = poolToActiveStages.getOrElseUpdate(poolName, new HashMap[Int, StageInfo]()) + stages(stage.stageId) = stage } override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val sid = taskStart.task.stageId - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) - tasksActive += taskStart.taskInfo - val taskList = stageIdToTaskInfos.getOrElse( - sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) - taskList += ((taskStart.taskInfo, None, None)) - stageIdToTaskInfos(sid) = taskList + val sid = taskStart.stageId + val taskInfo = taskStart.taskInfo + if (taskInfo != null) { + val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) + tasksActive(taskInfo.taskId) = taskInfo + val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) + taskMap(taskInfo.taskId) = new TaskUIData(taskInfo) + stageIdToTaskData(sid) = taskMap + } } - override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) - = synchronized { + override def onTaskGettingResult(taskGettingResult: SparkListenerTaskGettingResult) { // Do nothing: because we don't do a deep copy of the TaskInfo, the TaskInfo in // stageToTaskInfos already has the updated status. } override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - val sid = taskEnd.task.stageId - - // create executor summary map if necessary - val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, - op = new HashMap[String, ExecutorSummary]()) - executorSummaryMap.getOrElseUpdate(key = taskEnd.taskInfo.executorId, - op = new ExecutorSummary()) - - val executorSummary = executorSummaryMap.get(taskEnd.taskInfo.executorId) - executorSummary match { - case Some(y) => { - // first update failed-task, succeed-task + val sid = taskEnd.stageId + val info = taskEnd.taskInfo + + if (info != null) { + // create executor summary map if necessary + val executorSummaryMap = stageIdToExecutorSummaries.getOrElseUpdate(key = sid, + op = new HashMap[String, ExecutorSummary]()) + executorSummaryMap.getOrElseUpdate(key = info.executorId, op = new ExecutorSummary) + + val executorSummary = executorSummaryMap.get(info.executorId) + executorSummary match { + case Some(y) => { + // first update failed-task, succeed-task + taskEnd.reason match { + case Success => + y.succeededTasks += 1 + case _ => + y.failedTasks += 1 + } + + // update duration + y.taskTime += info.duration + + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } + metrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } + y.memoryBytesSpilled += metrics.memoryBytesSpilled + y.diskBytesSpilled += metrics.diskBytesSpilled + } + } + case _ => {} + } + + val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashMap[Long, TaskInfo]()) + // Remove by taskId, rather than by TaskInfo, in case the TaskInfo is from storage + tasksActive.remove(info.taskId) + + val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = taskEnd.reason match { - case Success => - y.succeededTasks += 1 + case e: ExceptionFailure => + stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 + (Some(e), e.metrics) case _ => - y.failedTasks += 1 + stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 + (None, Option(taskEnd.taskMetrics)) } - // update duration - y.taskTime += taskEnd.taskInfo.duration + stageIdToTime.getOrElseUpdate(sid, 0L) + val time = metrics.map(_.executorRunTime).getOrElse(0L) + stageIdToTime(sid) += time + totalTime += time - Option(taskEnd.taskMetrics).foreach { taskMetrics => - taskMetrics.shuffleReadMetrics.foreach { y.shuffleRead += _.remoteBytesRead } - taskMetrics.shuffleWriteMetrics.foreach { y.shuffleWrite += _.shuffleBytesWritten } - y.memoryBytesSpilled += taskMetrics.memoryBytesSpilled - y.diskBytesSpilled += taskMetrics.diskBytesSpilled - } - } - case _ => {} - } + stageIdToShuffleRead.getOrElseUpdate(sid, 0L) + val shuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead).getOrElse(0L) + stageIdToShuffleRead(sid) += shuffleRead + totalShuffleRead += shuffleRead - val tasksActive = stageIdToTasksActive.getOrElseUpdate(sid, new HashSet[TaskInfo]()) - tasksActive -= taskEnd.taskInfo - - val (failureInfo, metrics): (Option[ExceptionFailure], Option[TaskMetrics]) = - taskEnd.reason match { - case e: ExceptionFailure => - stageIdToTasksFailed(sid) = stageIdToTasksFailed.getOrElse(sid, 0) + 1 - (Some(e), e.metrics) - case _ => - stageIdToTasksComplete(sid) = stageIdToTasksComplete.getOrElse(sid, 0) + 1 - (None, Option(taskEnd.taskMetrics)) - } + stageIdToShuffleWrite.getOrElseUpdate(sid, 0L) + val shuffleWrite = + metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten).getOrElse(0L) + stageIdToShuffleWrite(sid) += shuffleWrite + totalShuffleWrite += shuffleWrite - stageIdToTime.getOrElseUpdate(sid, 0L) - val time = metrics.map(m => m.executorRunTime).getOrElse(0) - stageIdToTime(sid) += time - totalTime += time - - stageIdToShuffleRead.getOrElseUpdate(sid, 0L) - val shuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => - s.remoteBytesRead).getOrElse(0L) - stageIdToShuffleRead(sid) += shuffleRead - totalShuffleRead += shuffleRead - - stageIdToShuffleWrite.getOrElseUpdate(sid, 0L) - val shuffleWrite = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => - s.shuffleBytesWritten).getOrElse(0L) - stageIdToShuffleWrite(sid) += shuffleWrite - totalShuffleWrite += shuffleWrite - - stageIdToMemoryBytesSpilled.getOrElseUpdate(sid, 0L) - val memoryBytesSpilled = metrics.map(m => m.memoryBytesSpilled).getOrElse(0L) - stageIdToMemoryBytesSpilled(sid) += memoryBytesSpilled - - stageIdToDiskBytesSpilled.getOrElseUpdate(sid, 0L) - val diskBytesSpilled = metrics.map(m => m.diskBytesSpilled).getOrElse(0L) - stageIdToDiskBytesSpilled(sid) += diskBytesSpilled - - val taskList = stageIdToTaskInfos.getOrElse( - sid, HashSet[(TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])]()) - taskList -= ((taskEnd.taskInfo, None, None)) - taskList += ((taskEnd.taskInfo, metrics, failureInfo)) - stageIdToTaskInfos(sid) = taskList + stageIdToMemoryBytesSpilled.getOrElseUpdate(sid, 0L) + val memoryBytesSpilled = metrics.map(_.memoryBytesSpilled).getOrElse(0L) + stageIdToMemoryBytesSpilled(sid) += memoryBytesSpilled + + stageIdToDiskBytesSpilled.getOrElseUpdate(sid, 0L) + val diskBytesSpilled = metrics.map(_.diskBytesSpilled).getOrElse(0L) + stageIdToDiskBytesSpilled(sid) += diskBytesSpilled + + val taskMap = stageIdToTaskData.getOrElse(sid, HashMap[Long, TaskUIData]()) + taskMap(info.taskId) = new TaskUIData(info, metrics, failureInfo) + stageIdToTaskData(sid) = taskMap + } } override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { - jobEnd match { - case end: SparkListenerJobEnd => - end.jobResult match { - case JobFailed(ex, Some(stage)) => - /* If two jobs share a stage we could get this failure message twice. So we first - * check whether we've already retired this stage. */ - val stageInfo = activeStages.filter(s => s.stageId == stage.id).headOption - stageInfo.foreach {s => - activeStages -= s - poolToActiveStages(stageIdToPool(stage.id)) -= s - failedStages += s - trimIfNecessary(failedStages) - } - case _ => + jobEnd.jobResult match { + case JobFailed(_, stageId) => + activeStages.get(stageId).foreach { s => + // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage + activeStages.remove(s.stageId) + poolToActiveStages(stageIdToPool(stageId)).remove(s.stageId) + failedStages += s + trimIfNecessary(failedStages) } case _ => } } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val schedulingModeName = + environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") + schedulingMode = schedulingModeName match { + case Some(name) => Some(SchedulingMode.withName(name)) + case None => None + } + } + } + + override def onBlockManagerAdded(blockManagerAdded: SparkListenerBlockManagerAdded) { + synchronized { + val blockManagerId = blockManagerAdded.blockManagerId + val executorId = blockManagerId.executorId + executorIdToBlockManagerId(executorId) = blockManagerId + } + } + + override def onBlockManagerRemoved(blockManagerRemoved: SparkListenerBlockManagerRemoved) { + synchronized { + val executorId = blockManagerRemoved.blockManagerId.executorId + executorIdToBlockManagerId.remove(executorId) + } + } + +} + +private[ui] case class TaskUIData( + taskInfo: TaskInfo, + taskMetrics: Option[TaskMetrics] = None, + exception: Option[ExceptionFailure] = None) + +private object JobProgressListener { + val DEFAULT_POOL_NAME = "default" + val DEFAULT_RETAINED_STAGES = 1000 } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 2d95d47e154cd..ee4e9c69c1bd1 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -20,41 +20,43 @@ package org.apache.spark.ui.jobs import java.text.SimpleDateFormat import javax.servlet.http.HttpServletRequest -import scala.Seq - -import org.eclipse.jetty.server.Handler import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.SparkContext +import org.apache.spark.SparkConf +import org.apache.spark.scheduler.SchedulingMode import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.ui.SparkUI import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[spark] class JobProgressUI(val sc: SparkContext) { - private var _listener: Option[JobProgressListener] = None - def listener = _listener.get +private[ui] class JobProgressUI(parent: SparkUI) { + val appName = parent.appName + val basePath = parent.basePath val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + val live = parent.live + val sc = parent.sc + + lazy val listener = _listener.get + lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) private val indexPage = new IndexPage(this) private val stagePage = new StagePage(this) private val poolPage = new PoolPage(this) + private var _listener: Option[JobProgressListener] = None def start() { - _listener = Some(new JobProgressListener(sc)) - sc.addSparkListener(listener) + val conf = if (live) sc.conf else new SparkConf + _listener = Some(new JobProgressListener(conf)) } def formatDuration(ms: Long) = Utils.msDurationToString(ms) def getHandlers = Seq[ServletContextHandler]( createServletHandler("/stages/stage", - createServlet((request: HttpServletRequest) => stagePage.render(request), - sc.env.securityManager)), + (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), createServletHandler("/stages/pool", - createServlet((request: HttpServletRequest) => poolPage.render(request), - sc.env.securityManager)), + (request: HttpServletRequest) => poolPage.render(request), parent.securityManager, basePath), createServletHandler("/stages", - createServlet((request: HttpServletRequest) => indexPage.render(request), - sc.env.securityManager)) + (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) ) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index eb7518a020840..bd33182b70059 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -21,27 +21,38 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node +import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ -private[spark] class PoolPage(parent: JobProgressUI) { - def listener = parent.listener +private[ui] class PoolPage(parent: JobProgressUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private val live = parent.live + private val sc = parent.sc + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val poolName = request.getParameter("poolname") val poolToActiveStages = listener.poolToActiveStages - val activeStages = poolToActiveStages.get(poolName).toSeq.flatten + val activeStages = poolToActiveStages.get(poolName) match { + case Some(s) => s.values.toSeq + case None => Seq[StageInfo]() + } val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) - val pool = listener.sc.getPoolForName(poolName).get - val poolTable = new PoolTable(Seq(pool), listener) + // For now, pool information is only accessible in live UIs + val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq[Schedulable]() + val poolTable = new PoolTable(pools, parent) - val content =

    Summary

    ++ poolTable.toNodeSeq() ++ -

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq() + val content = +

    Summary

    ++ poolTable.toNodeSeq ++ +

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - headerSparkPage(content, parent.sc, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage( + content, basePath, appName, "Fair Scheduler Pool: " + poolName, Stages) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index 22bc97ada18be..c5c8d8668740b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -18,26 +18,26 @@ package org.apache.spark.ui.jobs import scala.collection.mutable.HashMap -import scala.collection.mutable.HashSet import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressListener) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { + private val basePath = parent.basePath + private val poolToActiveStages = listener.poolToActiveStages + private lazy val listener = parent.listener - var poolToActiveStages: HashMap[String, HashSet[StageInfo]] = listener.poolToActiveStages - - def toNodeSeq(): Seq[Node] = { + def toNodeSeq: Seq[Node] = { listener.synchronized { poolTable(poolRow, pools) } } - private def poolTable(makeRow: (Schedulable, HashMap[String, HashSet[StageInfo]]) => Seq[Node], - rows: Seq[Schedulable] - ): Seq[Node] = { + private def poolTable( + makeRow: (Schedulable, HashMap[String, HashMap[Int, StageInfo]]) => Seq[Node], + rows: Seq[Schedulable]): Seq[Node] = { @@ -53,15 +53,18 @@ private[spark] class PoolTable(pools: Seq[Schedulable], listener: JobProgressLis
    Pool Name
    } - private def poolRow(p: Schedulable, poolToActiveStages: HashMap[String, HashSet[StageInfo]]) - : Seq[Node] = { + private def poolRow( + p: Schedulable, + poolToActiveStages: HashMap[String, HashMap[Int, StageInfo]]): Seq[Node] = { val activeStages = poolToActiveStages.get(p.name) match { case Some(stages) => stages.size case None => 0 } - {p.name} + + {p.name} + {p.minShare} {p.weight} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index ddc687a45a095..da7f20233063e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,47 +22,47 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ExceptionFailure -import org.apache.spark.executor.TaskMetrics -import org.apache.spark.scheduler.TaskInfo -import org.apache.spark.ui.UIUtils._ import org.apache.spark.ui.Page._ +import org.apache.spark.ui.UIUtils import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[spark] class StagePage(parent: JobProgressUI) { - def listener = parent.listener - val dateFmt = parent.dateFmt +private[ui] class StagePage(parent: JobProgressUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private val dateFmt = parent.dateFmt + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt - val now = System.currentTimeMillis() - if (!listener.stageIdToTaskInfos.contains(stageId)) { + if (!listener.stageIdToTaskData.contains(stageId)) { val content =

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet
    - return headerSparkPage(content, parent.sc, "Details for Stage %s".format(stageId), Stages) + return UIUtils.headerSparkPage( + content, basePath, appName, "Details for Stage %s".format(stageId), Stages) } - val tasks = listener.stageIdToTaskInfos(stageId).toSeq.sortBy(_._1.launchTime) + val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) - val numCompleted = tasks.count(_._1.finished) + val numCompleted = tasks.count(_.taskInfo.finished) val shuffleReadBytes = listener.stageIdToShuffleRead.getOrElse(stageId, 0L) val hasShuffleRead = shuffleReadBytes > 0 val shuffleWriteBytes = listener.stageIdToShuffleWrite.getOrElse(stageId, 0L) val hasShuffleWrite = shuffleWriteBytes > 0 val memoryBytesSpilled = listener.stageIdToMemoryBytesSpilled.getOrElse(stageId, 0L) val diskBytesSpilled = listener.stageIdToDiskBytesSpilled.getOrElse(stageId, 0L) - val hasBytesSpilled = (memoryBytesSpilled > 0 && diskBytesSpilled > 0) + val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 var activeTime = 0L - listener.stageIdToTasksActive(stageId).foreach(activeTime += _.timeRunning(now)) + val now = System.currentTimeMillis() + val tasksActive = listener.stageIdToTasksActive(stageId).values + tasksActive.foreach(activeTime += _.timeRunning(now)) - val finishedTasks = listener.stageIdToTaskInfos(stageId).filter(_._1.finished) // scalastyle:off val summary =
    @@ -104,42 +104,45 @@ private[spark] class StagePage(parent: JobProgressUI) { {if (hasBytesSpilled) Seq("Shuffle Spill (Memory)", "Shuffle Spill (Disk)") else Nil} ++ Seq("Errors") - val taskTable = listingTable( + val taskTable = UIUtils.listingTable( taskHeaders, taskRow(hasShuffleRead, hasShuffleWrite, hasBytesSpilled), tasks) // Excludes tasks which failed and have incomplete metrics - val validTasks = tasks.filter(t => t._1.status == "SUCCESS" && (t._2.isDefined)) + val validTasks = tasks.filter(t => t.taskInfo.status == "SUCCESS" && t.taskMetrics.isDefined) val summaryTable: Option[Seq[Node]] = if (validTasks.size == 0) { None } else { - val serializationTimes = validTasks.map{case (info, metrics, exception) => - metrics.get.resultSerializationTime.toDouble} + val serializationTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.resultSerializationTime.toDouble + } val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes). get.getQuantiles().map(ms => parent.formatDuration(ms.toLong)) - val serviceTimes = validTasks.map{case (info, metrics, exception) => - metrics.get.executorRunTime.toDouble} - val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles().map( - ms => parent.formatDuration(ms.toLong)) + val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.executorRunTime.toDouble + } + val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles() + .map(ms => parent.formatDuration(ms.toLong)) - val gettingResultTimes = validTasks.map{case (info, metrics, exception) => + val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { (info.finishTime - info.gettingResultTime).toDouble } else { 0.0 } } - val gettingResultQuantiles = ("Time spent fetching task results" +: - Distribution(gettingResultTimes).get.getQuantiles().map( - millis => parent.formatDuration(millis.toLong))) + val gettingResultQuantiles = "Time spent fetching task results" +: + Distribution(gettingResultTimes).get.getQuantiles().map { millis => + parent.formatDuration(millis.toLong) + } // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, // if it needed to be fetched from the block manager on the worker). - val schedulerDelays = validTasks.map{case (info, metrics, exception) => + val schedulerDelays = validTasks.map { case TaskUIData(info, metrics, _) => val totalExecutionTime = { if (info.gettingResultTime > 0) { (info.gettingResultTime - info.launchTime).toDouble @@ -149,35 +152,32 @@ private[spark] class StagePage(parent: JobProgressUI) { } totalExecutionTime - metrics.get.executorRunTime } - val schedulerDelayQuantiles = ("Scheduler delay" +: - Distribution(schedulerDelays).get.getQuantiles().map( - millis => parent.formatDuration(millis.toLong))) + val schedulerDelayQuantiles = "Scheduler delay" +: + Distribution(schedulerDelays).get.getQuantiles().map { millis => + parent.formatDuration(millis.toLong) + } def getQuantileCols(data: Seq[Double]) = Distribution(data).get.getQuantiles().map(d => Utils.bytesToString(d.toLong)) - val shuffleReadSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble + val shuffleReadSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleReadMetrics.map(_.remoteBytesRead).getOrElse(0L).toDouble } val shuffleReadQuantiles = "Shuffle Read (Remote)" +: getQuantileCols(shuffleReadSizes) - val shuffleWriteSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble + val shuffleWriteSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.shuffleWriteMetrics.map(_.shuffleBytesWritten).getOrElse(0L).toDouble } val shuffleWriteQuantiles = "Shuffle Write" +: getQuantileCols(shuffleWriteSizes) - val memoryBytesSpilledSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.memoryBytesSpilled.toDouble + val memoryBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.memoryBytesSpilled.toDouble } val memoryBytesSpilledQuantiles = "Shuffle spill (memory)" +: getQuantileCols(memoryBytesSpilledSizes) - val diskBytesSpilledSizes = validTasks.map { - case(info, metrics, exception) => - metrics.get.diskBytesSpilled.toDouble + val diskBytesSpilledSizes = validTasks.map { case TaskUIData(_, metrics, _) => + metrics.get.diskBytesSpilled.toDouble } val diskBytesSpilledQuantiles = "Shuffle spill (disk)" +: getQuantileCols(diskBytesSpilledSizes) @@ -195,98 +195,104 @@ private[spark] class StagePage(parent: JobProgressUI) { val quantileHeaders = Seq("Metric", "Min", "25th percentile", "Median", "75th percentile", "Max") def quantileRow(data: Seq[String]): Seq[Node] = {data.map(d => {d})} - Some(listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) + Some(UIUtils.listingTable(quantileHeaders, quantileRow, listings, fixedWidth = true)) } - val executorTable = new ExecutorTable(parent, stageId) + val executorTable = new ExecutorTable(stageId, parent) val content = summary ++

    Summary Metrics for {numCompleted} Completed Tasks

    ++
    {summaryTable.getOrElse("No tasks have reported metrics yet.")}
    ++ -

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq() ++ +

    Aggregated Metrics by Executor

    ++ executorTable.toNodeSeq ++

    Tasks

    ++ taskTable - headerSparkPage(content, parent.sc, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage( + content, basePath, appName, "Details for Stage %d".format(stageId), Stages) } } def taskRow(shuffleRead: Boolean, shuffleWrite: Boolean, bytesSpilled: Boolean) - (taskData: (TaskInfo, Option[TaskMetrics], Option[ExceptionFailure])): Seq[Node] = { + (taskData: TaskUIData): Seq[Node] = { def fmtStackTrace(trace: Seq[StackTraceElement]): Seq[Node] = trace.map(e => {e.toString}) - val (info, metrics, exception) = taskData - - val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) - else metrics.map(m => m.executorRunTime).getOrElse(1) - val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) - else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") - val gcTime = metrics.map(m => m.jvmGCTime).getOrElse(0L) - val serializationTime = metrics.map(m => m.resultSerializationTime).getOrElse(0L) - - val maybeShuffleRead = metrics.flatMap(m => m.shuffleReadMetrics).map(s => s.remoteBytesRead) - val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") - val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") - - val maybeShuffleWrite = - metrics.flatMap{m => m.shuffleWriteMetrics}.map(s => s.shuffleBytesWritten) - val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") - val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") - - val maybeWriteTime = metrics.flatMap(m => m.shuffleWriteMetrics).map(s => s.shuffleWriteTime) - val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") - val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map{ ms => - if (ms == 0) "" else parent.formatDuration(ms)}.getOrElse("") - - val maybeMemoryBytesSpilled = metrics.map(m => m.memoryBytesSpilled) - val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.map(_.toString).getOrElse("") - val memoryBytesSpilledReadable = maybeMemoryBytesSpilled.map(Utils.bytesToString).getOrElse("") - - val maybeDiskBytesSpilled = metrics.map{m => m.diskBytesSpilled} - val diskBytesSpilledSortable = maybeDiskBytesSpilled.map(_.toString).getOrElse("") - val diskBytesSpilledReadable = maybeDiskBytesSpilled.map(Utils.bytesToString).getOrElse("") - - - {info.index} - {info.taskId} - {info.status} - {info.taskLocality} - {info.host} - {dateFmt.format(new Date(info.launchTime))} - - {formatDuration} - - - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} - - - {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} - - {if (shuffleRead) { - - {shuffleReadReadable} - - }} - {if (shuffleWrite) { - - {writeTimeReadable} - - - {shuffleWriteReadable} - - }} - {if (bytesSpilled) { - - {memoryBytesSpilledReadable} + + taskData match { case TaskUIData(info, metrics, exception) => + val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) + else metrics.map(_.executorRunTime).getOrElse(1L) + val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) + else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) + val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) + + val maybeShuffleRead = metrics.flatMap(_.shuffleReadMetrics).map(_.remoteBytesRead) + val shuffleReadSortable = maybeShuffleRead.map(_.toString).getOrElse("") + val shuffleReadReadable = maybeShuffleRead.map(Utils.bytesToString).getOrElse("") + + val maybeShuffleWrite = + metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleBytesWritten) + val shuffleWriteSortable = maybeShuffleWrite.map(_.toString).getOrElse("") + val shuffleWriteReadable = maybeShuffleWrite.map(Utils.bytesToString).getOrElse("") + + val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) + val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") + val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => + if (ms == 0) "" else parent.formatDuration(ms) + }.getOrElse("") + + val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) + val memoryBytesSpilledSortable = maybeMemoryBytesSpilled.map(_.toString).getOrElse("") + val memoryBytesSpilledReadable = + maybeMemoryBytesSpilled.map(Utils.bytesToString).getOrElse("") + + val maybeDiskBytesSpilled = metrics.map(_.diskBytesSpilled) + val diskBytesSpilledSortable = maybeDiskBytesSpilled.map(_.toString).getOrElse("") + val diskBytesSpilledReadable = maybeDiskBytesSpilled.map(Utils.bytesToString).getOrElse("") + + + {info.index} + {info.taskId} + {info.status} + {info.taskLocality} + {info.host} + {dateFmt.format(new Date(info.launchTime))} + + {formatDuration} + + + {if (gcTime > 0) parent.formatDuration(gcTime) else ""} - - {diskBytesSpilledReadable} + + {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} - }} - {exception.map(e => - - {e.className} ({e.description})
    - {fmtStackTrace(e.stackTrace)} -
    ).getOrElse("")} - - + {if (shuffleRead) { + + {shuffleReadReadable} + + }} + {if (shuffleWrite) { + + {writeTimeReadable} + + + {shuffleWriteReadable} + + }} + {if (bytesSpilled) { + + {memoryBytesSpilledReadable} + + + {diskBytesSpilledReadable} + + }} + + {exception.map { e => + + {e.className} ({e.description})
    + {fmtStackTrace(e.stackTrace)} +
    + }.getOrElse("")} + + + } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 99828487f1902..68fef5234c9ab 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -19,27 +19,27 @@ package org.apache.spark.ui.jobs import java.util.Date -import scala.collection.mutable.HashSet +import scala.collection.mutable.HashMap import scala.xml.Node -import org.apache.spark.scheduler.{SchedulingMode, StageInfo, TaskInfo} +import org.apache.spark.scheduler.{StageInfo, TaskInfo} import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgressUI) { +private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { + private val basePath = parent.basePath + private val dateFmt = parent.dateFmt + private lazy val listener = parent.listener + private lazy val isFairScheduler = parent.isFairScheduler - val listener = parent.listener - val dateFmt = parent.dateFmt - val isFairScheduler = listener.sc.getSchedulingMode == SchedulingMode.FAIR - - def toNodeSeq(): Seq[Node] = { + def toNodeSeq: Seq[Node] = { listener.synchronized { stageTable(stageRow, stages) } } - /** Special table which merges two header cells. */ + /** Special table that merges two header cells. */ private def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { @@ -72,55 +72,56 @@ private[spark] class StageTable(val stages: Seq[StageInfo], val parent: JobProgr } - + /** Render an HTML row that represents a stage */ private def stageRow(s: StageInfo): Seq[Node] = { + val poolName = listener.stageIdToPool.get(s.stageId) + val nameLink = + + {s.name} + + val description = listener.stageIdToDescription.get(s.stageId) + .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) val submissionTime = s.submissionTime match { case Some(t) => dateFmt.format(new Date(t)) case None => "Unknown" } - + val finishTime = s.completionTime.getOrElse(System.currentTimeMillis) + val duration = s.submissionTime.map { t => + if (finishTime > t) finishTime - t else System.currentTimeMillis - t + } + val formattedDuration = duration.map(d => parent.formatDuration(d)).getOrElse("Unknown") + val startedTasks = + listener.stageIdToTasksActive.getOrElse(s.stageId, HashMap[Long, TaskInfo]()).size + val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) + val failedTasks = listener.stageIdToTasksFailed.getOrElse(s.stageId, 0) match { + case f if f > 0 => "(%s failed)".format(f) + case _ => "" + } + val totalTasks = s.numTasks val shuffleReadSortable = listener.stageIdToShuffleRead.getOrElse(s.stageId, 0L) val shuffleRead = shuffleReadSortable match { case 0 => "" case b => Utils.bytesToString(b) } - val shuffleWriteSortable = listener.stageIdToShuffleWrite.getOrElse(s.stageId, 0L) val shuffleWrite = shuffleWriteSortable match { case 0 => "" case b => Utils.bytesToString(b) } - val startedTasks = listener.stageIdToTasksActive.getOrElse(s.stageId, HashSet[TaskInfo]()).size - val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) - val failedTasks = listener.stageIdToTasksFailed.getOrElse(s.stageId, 0) match { - case f if f > 0 => "(%s failed)".format(f) - case _ => "" - } - val totalTasks = s.numTasks - - val poolName = listener.stageIdToPool.get(s.stageId) - - val nameLink = - {s.name} - val description = listener.stageIdToDescription.get(s.stageId) - .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) - val finishTime = s.completionTime.getOrElse(System.currentTimeMillis()) - val duration = s.submissionTime.map { t => - if (finishTime > t) finishTime - t else System.currentTimeMillis - t - } - {if (isFairScheduler) { - } - } + + }} - + diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index cb2083eb019bf..4d8b01dbe6e1b 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -19,22 +19,80 @@ package org.apache.spark.ui.storage import javax.servlet.http.HttpServletRequest +import scala.collection.mutable + import org.eclipse.jetty.servlet.ServletContextHandler -import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.ui._ import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.scheduler._ +import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[spark] class BlockManagerUI(val sc: SparkContext) extends Logging { - val indexPage = new IndexPage(this) - val rddPage = new RDDPage(this) +private[ui] class BlockManagerUI(parent: SparkUI) { + val appName = parent.appName + val basePath = parent.basePath + + private val indexPage = new IndexPage(this) + private val rddPage = new RDDPage(this) + private var _listener: Option[BlockManagerListener] = None + + lazy val listener = _listener.get + + def start() { + _listener = Some(new BlockManagerListener(parent.storageStatusListener)) + } def getHandlers = Seq[ServletContextHandler]( createServletHandler("/storage/rdd", - createServlet((request: HttpServletRequest) => rddPage.render(request), - sc.env.securityManager)), + (request: HttpServletRequest) => rddPage.render(request), parent.securityManager, basePath), createServletHandler("/storage", - createServlet((request: HttpServletRequest) => indexPage.render(request), - sc.env.securityManager)) + (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) ) } + +/** + * A SparkListener that prepares information to be displayed on the BlockManagerUI + */ +private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListener) + extends SparkListener { + + private val _rddInfoMap = mutable.Map[Int, RDDInfo]() + + def storageStatusList = storageStatusListener.storageStatusList + + /** Filter RDD info to include only those with cached partitions */ + def rddInfoList = _rddInfoMap.values.filter(_.numCachedPartitions > 0).toSeq + + /** Update each RDD's info to reflect any updates to the RDD's storage status */ + private def updateRDDInfo() { + val rddInfos = _rddInfoMap.values.toSeq + val updatedRddInfos = StorageUtils.rddInfoFromStorageStatus(storageStatusList, rddInfos) + updatedRddInfos.foreach { info => _rddInfoMap(info.id) = info } + } + + /** + * Assumes the storage status list is fully up-to-date. This implies the corresponding + * StorageStatusSparkListener must process the SparkListenerTaskEnd event before this listener. + */ + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val metrics = taskEnd.taskMetrics + if (metrics != null && metrics.updatedBlocks.isDefined) { + updateRDDInfo() + } + } + + override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { + val rddInfo = stageSubmitted.stageInfo.rddInfo + _rddInfoMap(rddInfo.id) = rddInfo + } + + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { + // Remove all partitions that are no longer cached + _rddInfoMap.retain { case (_, info) => info.numCachedPartitions > 0 } + } + + override def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) = synchronized { + updateRDDInfo() + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 6a3c41fb1155d..b2732de51058a 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -21,36 +21,37 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.storage.{RDDInfo, StorageUtils} +import org.apache.spark.storage.RDDInfo import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[spark] class IndexPage(parent: BlockManagerUI) { - val sc = parent.sc +private[ui] class IndexPage(parent: BlockManagerUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val storageStatusList = sc.getExecutorStorageStatus - // Calculate macro-level statistics - - val rddHeaders = Seq( - "RDD Name", - "Storage Level", - "Cached Partitions", - "Fraction Cached", - "Size in Memory", - "Size on Disk") - val rdds = StorageUtils.rddInfoFromStorageStatus(storageStatusList, sc) - val content = listingTable(rddHeaders, rddRow, rdds) - - headerSparkPage(content, parent.sc, "Storage ", Storage) + val rdds = listener.rddInfoList + val content = UIUtils.listingTable(rddHeader, rddRow, rdds) + UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) } - def rddRow(rdd: RDDInfo): Seq[Node] = { + /** Header fields for the RDD table */ + private def rddHeader = Seq( + "RDD Name", + "Storage Level", + "Cached Partitions", + "Fraction Cached", + "Size in Memory", + "Size on Disk") + + /** Render an HTML row representing an RDD */ + private def rddRow(rdd: RDDInfo): Seq[Node] = { diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 78b149b14b1d4..3f42eba4ece00 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -21,36 +21,37 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.storage.{BlockId, StorageStatus, StorageUtils} -import org.apache.spark.storage.BlockManagerMasterActor.BlockStatus +import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils._ +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[spark] class RDDPage(parent: BlockManagerUI) { - val sc = parent.sc +private[ui] class RDDPage(parent: BlockManagerUI) { + private val appName = parent.appName + private val basePath = parent.basePath + private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val id = request.getParameter("id").toInt - val storageStatusList = sc.getExecutorStorageStatus - val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, id) - val rddInfo = StorageUtils.rddInfoFromStorageStatus(filteredStorageStatusList, sc).head - - val workerHeaders = Seq("Host", "Memory Usage", "Disk Usage") - val workers = filteredStorageStatusList.map((id, _)) - val workerTable = listingTable(workerHeaders, workerRow, workers) + val rddId = request.getParameter("id").toInt + val storageStatusList = listener.storageStatusList + val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { + // Rather than crashing, render an "RDD Not Found" page + return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", Storage) + } - val blockHeaders = Seq("Block Name", "Storage Level", "Size in Memory", "Size on Disk", - "Executors") + // Worker table + val workers = storageStatusList.map((rddId, _)) + val workerTable = UIUtils.listingTable(workerHeader, workerRow, workers) - val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).toArray. - sortWith(_._1.name < _._1.name) + // Block table + val filteredStorageStatusList = StorageUtils.filterStorageStatusByRDD(storageStatusList, rddId) + val blockStatuses = filteredStorageStatusList.flatMap(_.blocks).sortWith(_._1.name < _._1.name) val blockLocations = StorageUtils.blockLocationsFromStorageStatus(filteredStorageStatusList) - val blocks = blockStatuses.map { - case(id, status) => (id, status, blockLocations.get(id).getOrElse(Seq("UNKNOWN"))) + val blocks = blockStatuses.map { case (blockId, status) => + (blockId, status, blockLocations.get(blockId).getOrElse(Seq[String]("Unknown"))) } - val blockTable = listingTable(blockHeaders, blockRow, blocks) + val blockTable = UIUtils.listingTable(blockHeader, blockRow, blocks) val content =
    @@ -94,10 +95,39 @@ private[spark] class RDDPage(parent: BlockManagerUI) {
    ; - headerSparkPage(content, parent.sc, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage( + content, basePath, appName, "RDD Storage Info for " + rddInfo.name, Storage) + } + + /** Header fields for the worker table */ + private def workerHeader = Seq( + "Host", + "Memory Usage", + "Disk Usage") + + /** Header fields for the block table */ + private def blockHeader = Seq( + "Block Name", + "Storage Level", + "Size in Memory", + "Size on Disk", + "Executors") + + /** Render an HTML row representing a worker */ + private def workerRow(worker: (Int, StorageStatus)): Seq[Node] = { + val (rddId, status) = worker + + + + + } - def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = { + /** Render an HTML row representing a block */ + private def blockRow(row: (BlockId, BlockStatus, Seq[String])): Seq[Node] = { val (id, block, locations) = row @@ -115,16 +145,4 @@ private[spark] class RDDPage(parent: BlockManagerUI) { } - - def workerRow(worker: (Int, StorageStatus)): Seq[Node] = { - val (rddId, status) = worker - - - - - - } } diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala new file mode 100644 index 0000000000000..f07962096a32c --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -0,0 +1,165 @@ +/* + * 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.util + +import java.io._ +import java.net.URI +import java.text.SimpleDateFormat +import java.util.Date + +import it.unimi.dsi.fastutil.io.FastBufferedOutputStream +import org.apache.hadoop.fs.{FSDataOutputStream, Path} + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.io.CompressionCodec + +/** + * A generic class for logging information to file. + * + * @param logDir Path to the directory in which files are logged + * @param outputBufferSize The buffer size to use when writing to an output stream in bytes + * @param compress Whether to compress output + * @param overwrite Whether to overwrite existing files + */ +class FileLogger( + logDir: String, + conf: SparkConf = new SparkConf, + outputBufferSize: Int = 8 * 1024, // 8 KB + compress: Boolean = false, + overwrite: Boolean = true) + extends Logging { + + private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + private var fileIndex = 0 + + // Only used if compression is enabled + private lazy val compressionCodec = CompressionCodec.createCodec(conf) + + // Only defined if the file system scheme is not local + private var hadoopDataStream: Option[FSDataOutputStream] = None + + private var writer: Option[PrintWriter] = { + createLogDir() + Some(createWriter()) + } + + /** + * Create a logging directory with the given path. + */ + private def createLogDir() { + val path = new Path(logDir) + if (fileSystem.exists(path)) { + if (overwrite) { + logWarning("Log directory %s already exists. Overwriting...".format(logDir)) + // Second parameter is whether to delete recursively + fileSystem.delete(path, true) + } else { + throw new IOException("Log directory %s already exists!".format(logDir)) + } + } + if (!fileSystem.mkdirs(path)) { + throw new IOException("Error in creating log directory: %s".format(logDir)) + } + } + + /** + * Create a new writer for the file identified by the given path. + */ + private def createWriter(): PrintWriter = { + val logPath = logDir + "/" + fileIndex + val uri = new URI(logPath) + + /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). + * Therefore, for local files, use FileOutputStream instead. */ + val dstream = uri.getScheme match { + case "file" | null => + // Second parameter is whether to append + new FileOutputStream(logPath, !overwrite) + + case _ => + val path = new Path(logPath) + hadoopDataStream = Some(fileSystem.create(path, overwrite)) + hadoopDataStream.get + } + + val bstream = new FastBufferedOutputStream(dstream, outputBufferSize) + val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream + new PrintWriter(cstream) + } + + /** + * Log the message to the given writer. + * @param msg The message to be logged + * @param withTime Whether to prepend message with a timestamp + */ + def log(msg: String, withTime: Boolean = false) { + val writeInfo = if (!withTime) msg else { + val date = new Date(System.currentTimeMillis()) + DATE_FORMAT.format(date) + ": " + msg + } + writer.foreach(_.print(writeInfo)) + } + + /** + * Log the message to the given writer as a new line. + * @param msg The message to be logged + * @param withTime Whether to prepend message with a timestamp + */ + def logLine(msg: String, withTime: Boolean = false) = log(msg + "\n", withTime) + + /** + * Flush the writer to disk manually. + * + * If the Hadoop FileSystem is used, the underlying FSDataOutputStream (r1.0.4) must be + * sync()'ed manually as it does not support flush(), which is invoked by when higher + * level streams are flushed. + */ + def flush() { + writer.foreach(_.flush()) + hadoopDataStream.foreach(_.sync()) + } + + /** + * Close the writer. Any subsequent calls to log or flush will have no effect. + */ + def close() { + writer.foreach(_.close()) + writer = None + } + + /** + * Start a writer for a new file if one does not already exit. + */ + def start() { + writer.getOrElse { + fileIndex += 1 + writer = Some(createWriter()) + } + } + + /** + * Close all open writers, streams, and file systems. Any subsequent uses of this FileLogger + * instance will throw exceptions. + */ + def stop() { + hadoopDataStream.foreach(_.close()) + writer.foreach(_.close()) + fileSystem.close() + } +} diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala new file mode 100644 index 0000000000000..346f2b7856791 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -0,0 +1,710 @@ +/* + * 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.util + +import java.util.{Properties, UUID} + +import scala.collection.JavaConverters._ +import scala.collection.Map + +import org.json4s.DefaultFormats +import org.json4s.JsonDSL._ +import org.json4s.JsonAST._ + +import org.apache.spark.executor.{ShuffleReadMetrics, ShuffleWriteMetrics, TaskMetrics} +import org.apache.spark.scheduler._ +import org.apache.spark.storage._ +import org.apache.spark._ + +private[spark] object JsonProtocol { + private implicit val format = DefaultFormats + + /** ------------------------------------------------- * + * JSON serialization methods for SparkListenerEvents | + * -------------------------------------------------- */ + + def sparkEventToJson(event: SparkListenerEvent): JValue = { + event match { + case stageSubmitted: SparkListenerStageSubmitted => + stageSubmittedToJson(stageSubmitted) + case stageCompleted: SparkListenerStageCompleted => + stageCompletedToJson(stageCompleted) + case taskStart: SparkListenerTaskStart => + taskStartToJson(taskStart) + case taskGettingResult: SparkListenerTaskGettingResult => + taskGettingResultToJson(taskGettingResult) + case taskEnd: SparkListenerTaskEnd => + taskEndToJson(taskEnd) + case jobStart: SparkListenerJobStart => + jobStartToJson(jobStart) + case jobEnd: SparkListenerJobEnd => + jobEndToJson(jobEnd) + case environmentUpdate: SparkListenerEnvironmentUpdate => + environmentUpdateToJson(environmentUpdate) + case blockManagerAdded: SparkListenerBlockManagerAdded => + blockManagerAddedToJson(blockManagerAdded) + case blockManagerRemoved: SparkListenerBlockManagerRemoved => + blockManagerRemovedToJson(blockManagerRemoved) + case unpersistRDD: SparkListenerUnpersistRDD => + unpersistRDDToJson(unpersistRDD) + + // Not used, but keeps compiler happy + case SparkListenerShutdown => JNothing + } + } + + def stageSubmittedToJson(stageSubmitted: SparkListenerStageSubmitted): JValue = { + val stageInfo = stageInfoToJson(stageSubmitted.stageInfo) + val properties = propertiesToJson(stageSubmitted.properties) + ("Event" -> Utils.getFormattedClassName(stageSubmitted)) ~ + ("Stage Info" -> stageInfo) ~ + ("Properties" -> properties) + } + + def stageCompletedToJson(stageCompleted: SparkListenerStageCompleted): JValue = { + val stageInfo = stageInfoToJson(stageCompleted.stageInfo) + ("Event" -> Utils.getFormattedClassName(stageCompleted)) ~ + ("Stage Info" -> stageInfo) + } + + def taskStartToJson(taskStart: SparkListenerTaskStart): JValue = { + val taskInfo = taskStart.taskInfo + val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing + ("Event" -> Utils.getFormattedClassName(taskStart)) ~ + ("Stage ID" -> taskStart.stageId) ~ + ("Task Info" -> taskInfoJson) + } + + def taskGettingResultToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { + val taskInfo = taskGettingResult.taskInfo + val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing + ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ + ("Task Info" -> taskInfoJson) + } + + def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { + val taskEndReason = taskEndReasonToJson(taskEnd.reason) + val taskInfo = taskEnd.taskInfo + val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing + val taskMetrics = taskEnd.taskMetrics + val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing + ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ + ("Stage ID" -> taskEnd.stageId) ~ + ("Task Type" -> taskEnd.taskType) ~ + ("Task End Reason" -> taskEndReason) ~ + ("Task Info" -> taskInfoJson) ~ + ("Task Metrics" -> taskMetricsJson) + } + + def jobStartToJson(jobStart: SparkListenerJobStart): JValue = { + val properties = propertiesToJson(jobStart.properties) + ("Event" -> Utils.getFormattedClassName(jobStart)) ~ + ("Job ID" -> jobStart.jobId) ~ + ("Stage IDs" -> jobStart.stageIds) ~ + ("Properties" -> properties) + } + + def jobEndToJson(jobEnd: SparkListenerJobEnd): JValue = { + val jobResult = jobResultToJson(jobEnd.jobResult) + ("Event" -> Utils.getFormattedClassName(jobEnd)) ~ + ("Job ID" -> jobEnd.jobId) ~ + ("Job Result" -> jobResult) + } + + def environmentUpdateToJson(environmentUpdate: SparkListenerEnvironmentUpdate): JValue = { + val environmentDetails = environmentUpdate.environmentDetails + val jvmInformation = mapToJson(environmentDetails("JVM Information").toMap) + val sparkProperties = mapToJson(environmentDetails("Spark Properties").toMap) + val systemProperties = mapToJson(environmentDetails("System Properties").toMap) + val classpathEntries = mapToJson(environmentDetails("Classpath Entries").toMap) + ("Event" -> Utils.getFormattedClassName(environmentUpdate)) ~ + ("JVM Information" -> jvmInformation) ~ + ("Spark Properties" -> sparkProperties) ~ + ("System Properties" -> systemProperties) ~ + ("Classpath Entries" -> classpathEntries) + } + + def blockManagerAddedToJson(blockManagerAdded: SparkListenerBlockManagerAdded): JValue = { + val blockManagerId = blockManagerIdToJson(blockManagerAdded.blockManagerId) + ("Event" -> Utils.getFormattedClassName(blockManagerAdded)) ~ + ("Block Manager ID" -> blockManagerId) ~ + ("Maximum Memory" -> blockManagerAdded.maxMem) + } + + def blockManagerRemovedToJson(blockManagerRemoved: SparkListenerBlockManagerRemoved): JValue = { + val blockManagerId = blockManagerIdToJson(blockManagerRemoved.blockManagerId) + ("Event" -> Utils.getFormattedClassName(blockManagerRemoved)) ~ + ("Block Manager ID" -> blockManagerId) + } + + def unpersistRDDToJson(unpersistRDD: SparkListenerUnpersistRDD): JValue = { + ("Event" -> Utils.getFormattedClassName(unpersistRDD)) ~ + ("RDD ID" -> unpersistRDD.rddId) + } + + + /** ------------------------------------------------------------------- * + * JSON serialization methods for classes SparkListenerEvents depend on | + * -------------------------------------------------------------------- */ + + def stageInfoToJson(stageInfo: StageInfo): JValue = { + val rddInfo = rddInfoToJson(stageInfo.rddInfo) + val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) + val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) + ("Stage ID" -> stageInfo.stageId) ~ + ("Stage Name" -> stageInfo.name) ~ + ("Number of Tasks" -> stageInfo.numTasks) ~ + ("RDD Info" -> rddInfo) ~ + ("Submission Time" -> submissionTime) ~ + ("Completion Time" -> completionTime) ~ + ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) + } + + def taskInfoToJson(taskInfo: TaskInfo): JValue = { + ("Task ID" -> taskInfo.taskId) ~ + ("Index" -> taskInfo.index) ~ + ("Launch Time" -> taskInfo.launchTime) ~ + ("Executor ID" -> taskInfo.executorId) ~ + ("Host" -> taskInfo.host) ~ + ("Locality" -> taskInfo.taskLocality.toString) ~ + ("Getting Result Time" -> taskInfo.gettingResultTime) ~ + ("Finish Time" -> taskInfo.finishTime) ~ + ("Failed" -> taskInfo.failed) ~ + ("Serialized Size" -> taskInfo.serializedSize) + } + + def taskMetricsToJson(taskMetrics: TaskMetrics): JValue = { + val shuffleReadMetrics = + taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) + val shuffleWriteMetrics = + taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) + val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => + JArray(blocks.toList.map { case (id, status) => + ("Block ID" -> blockIdToJson(id)) ~ + ("Status" -> blockStatusToJson(status)) + }) + }.getOrElse(JNothing) + ("Host Name" -> taskMetrics.hostname) ~ + ("Executor Deserialize Time" -> taskMetrics.executorDeserializeTime) ~ + ("Executor Run Time" -> taskMetrics.executorRunTime) ~ + ("Result Size" -> taskMetrics.resultSize) ~ + ("JVM GC Time" -> taskMetrics.jvmGCTime) ~ + ("Result Serialization Time" -> taskMetrics.resultSerializationTime) ~ + ("Memory Bytes Spilled" -> taskMetrics.memoryBytesSpilled) ~ + ("Disk Bytes Spilled" -> taskMetrics.diskBytesSpilled) ~ + ("Shuffle Read Metrics" -> shuffleReadMetrics) ~ + ("Shuffle Write Metrics" -> shuffleWriteMetrics) ~ + ("Updated Blocks" -> updatedBlocks) + } + + def shuffleReadMetricsToJson(shuffleReadMetrics: ShuffleReadMetrics): JValue = { + ("Shuffle Finish Time" -> shuffleReadMetrics.shuffleFinishTime) ~ + ("Total Blocks Fetched" -> shuffleReadMetrics.totalBlocksFetched) ~ + ("Remote Blocks Fetched" -> shuffleReadMetrics.remoteBlocksFetched) ~ + ("Local Blocks Fetched" -> shuffleReadMetrics.localBlocksFetched) ~ + ("Fetch Wait Time" -> shuffleReadMetrics.fetchWaitTime) ~ + ("Remote Bytes Read" -> shuffleReadMetrics.remoteBytesRead) + } + + def shuffleWriteMetricsToJson(shuffleWriteMetrics: ShuffleWriteMetrics): JValue = { + ("Shuffle Bytes Written" -> shuffleWriteMetrics.shuffleBytesWritten) ~ + ("Shuffle Write Time" -> shuffleWriteMetrics.shuffleWriteTime) + } + + def taskEndReasonToJson(taskEndReason: TaskEndReason): JValue = { + val reason = Utils.getFormattedClassName(taskEndReason) + val json = taskEndReason match { + case fetchFailed: FetchFailed => + val blockManagerAddress = blockManagerIdToJson(fetchFailed.bmAddress) + ("Block Manager Address" -> blockManagerAddress) ~ + ("Shuffle ID" -> fetchFailed.shuffleId) ~ + ("Map ID" -> fetchFailed.mapId) ~ + ("Reduce ID" -> fetchFailed.reduceId) + case exceptionFailure: ExceptionFailure => + val stackTrace = stackTraceToJson(exceptionFailure.stackTrace) + val metrics = exceptionFailure.metrics.map(taskMetricsToJson).getOrElse(JNothing) + ("Class Name" -> exceptionFailure.className) ~ + ("Description" -> exceptionFailure.description) ~ + ("Stack Trace" -> stackTrace) ~ + ("Metrics" -> metrics) + case _ => Utils.emptyJson + } + ("Reason" -> reason) ~ json + } + + def blockManagerIdToJson(blockManagerId: BlockManagerId): JValue = { + ("Executor ID" -> blockManagerId.executorId) ~ + ("Host" -> blockManagerId.host) ~ + ("Port" -> blockManagerId.port) ~ + ("Netty Port" -> blockManagerId.nettyPort) + } + + def jobResultToJson(jobResult: JobResult): JValue = { + val result = Utils.getFormattedClassName(jobResult) + val json = jobResult match { + case JobSucceeded => Utils.emptyJson + case jobFailed: JobFailed => + val exception = exceptionToJson(jobFailed.exception) + ("Exception" -> exception) ~ + ("Failed Stage ID" -> jobFailed.failedStageId) + } + ("Result" -> result) ~ json + } + + def rddInfoToJson(rddInfo: RDDInfo): JValue = { + val storageLevel = storageLevelToJson(rddInfo.storageLevel) + ("RDD ID" -> rddInfo.id) ~ + ("Name" -> rddInfo.name) ~ + ("Storage Level" -> storageLevel) ~ + ("Number of Partitions" -> rddInfo.numPartitions) ~ + ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ + ("Memory Size" -> rddInfo.memSize) ~ + ("Disk Size" -> rddInfo.diskSize) + } + + def storageLevelToJson(storageLevel: StorageLevel): JValue = { + ("Use Disk" -> storageLevel.useDisk) ~ + ("Use Memory" -> storageLevel.useMemory) ~ + ("Deserialized" -> storageLevel.deserialized) ~ + ("Replication" -> storageLevel.replication) + } + + def blockIdToJson(blockId: BlockId): JValue = { + val blockType = Utils.getFormattedClassName(blockId) + val json: JObject = blockId match { + case rddBlockId: RDDBlockId => + ("RDD ID" -> rddBlockId.rddId) ~ + ("Split Index" -> rddBlockId.splitIndex) + case shuffleBlockId: ShuffleBlockId => + ("Shuffle ID" -> shuffleBlockId.shuffleId) ~ + ("Map ID" -> shuffleBlockId.mapId) ~ + ("Reduce ID" -> shuffleBlockId.reduceId) + case broadcastBlockId: BroadcastBlockId => + "Broadcast ID" -> broadcastBlockId.broadcastId + case broadcastHelperBlockId: BroadcastHelperBlockId => + ("Broadcast Block ID" -> blockIdToJson(broadcastHelperBlockId.broadcastId)) ~ + ("Helper Type" -> broadcastHelperBlockId.hType) + case taskResultBlockId: TaskResultBlockId => + "Task ID" -> taskResultBlockId.taskId + case streamBlockId: StreamBlockId => + ("Stream ID" -> streamBlockId.streamId) ~ + ("Unique ID" -> streamBlockId.uniqueId) + case tempBlockId: TempBlockId => + val uuid = UUIDToJson(tempBlockId.id) + "Temp ID" -> uuid + case testBlockId: TestBlockId => + "Test ID" -> testBlockId.id + } + ("Type" -> blockType) ~ json + } + + def blockStatusToJson(blockStatus: BlockStatus): JValue = { + val storageLevel = storageLevelToJson(blockStatus.storageLevel) + ("Storage Level" -> storageLevel) ~ + ("Memory Size" -> blockStatus.memSize) ~ + ("Disk Size" -> blockStatus.diskSize) + } + + + /** ------------------------------ * + * Util JSON serialization methods | + * ------------------------------- */ + + def mapToJson(m: Map[String, String]): JValue = { + val jsonFields = m.map { case (k, v) => JField(k, JString(v)) } + JObject(jsonFields.toList) + } + + def propertiesToJson(properties: Properties): JValue = { + Option(properties).map { p => + mapToJson(p.asScala) + }.getOrElse(JNothing) + } + + def UUIDToJson(id: UUID): JValue = { + ("Least Significant Bits" -> id.getLeastSignificantBits) ~ + ("Most Significant Bits" -> id.getMostSignificantBits) + } + + def stackTraceToJson(stackTrace: Array[StackTraceElement]): JValue = { + JArray(stackTrace.map { case line => + ("Declaring Class" -> line.getClassName) ~ + ("Method Name" -> line.getMethodName) ~ + ("File Name" -> line.getFileName) ~ + ("Line Number" -> line.getLineNumber) + }.toList) + } + + def exceptionToJson(exception: Exception): JValue = { + ("Message" -> exception.getMessage) ~ + ("Stack Trace" -> stackTraceToJson(exception.getStackTrace)) + } + + + /** --------------------------------------------------- * + * JSON deserialization methods for SparkListenerEvents | + * ---------------------------------------------------- */ + + def sparkEventFromJson(json: JValue): SparkListenerEvent = { + val stageSubmitted = Utils.getFormattedClassName(SparkListenerStageSubmitted) + val stageCompleted = Utils.getFormattedClassName(SparkListenerStageCompleted) + val taskStart = Utils.getFormattedClassName(SparkListenerTaskStart) + val taskGettingResult = Utils.getFormattedClassName(SparkListenerTaskGettingResult) + val taskEnd = Utils.getFormattedClassName(SparkListenerTaskEnd) + val jobStart = Utils.getFormattedClassName(SparkListenerJobStart) + val jobEnd = Utils.getFormattedClassName(SparkListenerJobEnd) + val environmentUpdate = Utils.getFormattedClassName(SparkListenerEnvironmentUpdate) + val blockManagerAdded = Utils.getFormattedClassName(SparkListenerBlockManagerAdded) + val blockManagerRemoved = Utils.getFormattedClassName(SparkListenerBlockManagerRemoved) + val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) + + (json \ "Event").extract[String] match { + case `stageSubmitted` => stageSubmittedFromJson(json) + case `stageCompleted` => stageCompletedFromJson(json) + case `taskStart` => taskStartFromJson(json) + case `taskGettingResult` => taskGettingResultFromJson(json) + case `taskEnd` => taskEndFromJson(json) + case `jobStart` => jobStartFromJson(json) + case `jobEnd` => jobEndFromJson(json) + case `environmentUpdate` => environmentUpdateFromJson(json) + case `blockManagerAdded` => blockManagerAddedFromJson(json) + case `blockManagerRemoved` => blockManagerRemovedFromJson(json) + case `unpersistRDD` => unpersistRDDFromJson(json) + } + } + + def stageSubmittedFromJson(json: JValue): SparkListenerStageSubmitted = { + val stageInfo = stageInfoFromJson(json \ "Stage Info") + val properties = propertiesFromJson(json \ "Properties") + SparkListenerStageSubmitted(stageInfo, properties) + } + + def stageCompletedFromJson(json: JValue): SparkListenerStageCompleted = { + val stageInfo = stageInfoFromJson(json \ "Stage Info") + SparkListenerStageCompleted(stageInfo) + } + + def taskStartFromJson(json: JValue): SparkListenerTaskStart = { + val stageId = (json \ "Stage ID").extract[Int] + val taskInfo = taskInfoFromJson(json \ "Task Info") + SparkListenerTaskStart(stageId, taskInfo) + } + + def taskGettingResultFromJson(json: JValue): SparkListenerTaskGettingResult = { + val taskInfo = taskInfoFromJson(json \ "Task Info") + SparkListenerTaskGettingResult(taskInfo) + } + + def taskEndFromJson(json: JValue): SparkListenerTaskEnd = { + val stageId = (json \ "Stage ID").extract[Int] + val taskType = (json \ "Task Type").extract[String] + val taskEndReason = taskEndReasonFromJson(json \ "Task End Reason") + val taskInfo = taskInfoFromJson(json \ "Task Info") + val taskMetrics = taskMetricsFromJson(json \ "Task Metrics") + SparkListenerTaskEnd(stageId, taskType, taskEndReason, taskInfo, taskMetrics) + } + + def jobStartFromJson(json: JValue): SparkListenerJobStart = { + val jobId = (json \ "Job ID").extract[Int] + val stageIds = (json \ "Stage IDs").extract[List[JValue]].map(_.extract[Int]) + val properties = propertiesFromJson(json \ "Properties") + SparkListenerJobStart(jobId, stageIds, properties) + } + + def jobEndFromJson(json: JValue): SparkListenerJobEnd = { + val jobId = (json \ "Job ID").extract[Int] + val jobResult = jobResultFromJson(json \ "Job Result") + SparkListenerJobEnd(jobId, jobResult) + } + + def environmentUpdateFromJson(json: JValue): SparkListenerEnvironmentUpdate = { + val environmentDetails = Map[String, Seq[(String, String)]]( + "JVM Information" -> mapFromJson(json \ "JVM Information").toSeq, + "Spark Properties" -> mapFromJson(json \ "Spark Properties").toSeq, + "System Properties" -> mapFromJson(json \ "System Properties").toSeq, + "Classpath Entries" -> mapFromJson(json \ "Classpath Entries").toSeq) + SparkListenerEnvironmentUpdate(environmentDetails) + } + + def blockManagerAddedFromJson(json: JValue): SparkListenerBlockManagerAdded = { + val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") + val maxMem = (json \ "Maximum Memory").extract[Long] + SparkListenerBlockManagerAdded(blockManagerId, maxMem) + } + + def blockManagerRemovedFromJson(json: JValue): SparkListenerBlockManagerRemoved = { + val blockManagerId = blockManagerIdFromJson(json \ "Block Manager ID") + SparkListenerBlockManagerRemoved(blockManagerId) + } + + def unpersistRDDFromJson(json: JValue): SparkListenerUnpersistRDD = { + SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) + } + + + /** --------------------------------------------------------------------- * + * JSON deserialization methods for classes SparkListenerEvents depend on | + * ---------------------------------------------------------------------- */ + + def stageInfoFromJson(json: JValue): StageInfo = { + val stageId = (json \ "Stage ID").extract[Int] + val stageName = (json \ "Stage Name").extract[String] + val numTasks = (json \ "Number of Tasks").extract[Int] + val rddInfo = rddInfoFromJson(json \ "RDD Info") + val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) + val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) + val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] + + val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) + stageInfo.submissionTime = submissionTime + stageInfo.completionTime = completionTime + stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning + stageInfo + } + + def taskInfoFromJson(json: JValue): TaskInfo = { + val taskId = (json \ "Task ID").extract[Long] + val index = (json \ "Index").extract[Int] + val launchTime = (json \ "Launch Time").extract[Long] + val executorId = (json \ "Executor ID").extract[String] + val host = (json \ "Host").extract[String] + val taskLocality = TaskLocality.withName((json \ "Locality").extract[String]) + val gettingResultTime = (json \ "Getting Result Time").extract[Long] + val finishTime = (json \ "Finish Time").extract[Long] + val failed = (json \ "Failed").extract[Boolean] + val serializedSize = (json \ "Serialized Size").extract[Int] + + val taskInfo = new TaskInfo(taskId, index, launchTime, executorId, host, taskLocality) + taskInfo.gettingResultTime = gettingResultTime + taskInfo.finishTime = finishTime + taskInfo.failed = failed + taskInfo.serializedSize = serializedSize + taskInfo + } + + def taskMetricsFromJson(json: JValue): TaskMetrics = { + val metrics = new TaskMetrics + metrics.hostname = (json \ "Host Name").extract[String] + metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] + metrics.executorRunTime = (json \ "Executor Run Time").extract[Long] + metrics.resultSize = (json \ "Result Size").extract[Long] + metrics.jvmGCTime = (json \ "JVM GC Time").extract[Long] + metrics.resultSerializationTime = (json \ "Result Serialization Time").extract[Long] + metrics.memoryBytesSpilled = (json \ "Memory Bytes Spilled").extract[Long] + metrics.diskBytesSpilled = (json \ "Disk Bytes Spilled").extract[Long] + metrics.shuffleReadMetrics = + Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) + metrics.shuffleWriteMetrics = + Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) + metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => + value.extract[List[JValue]].map { block => + val id = blockIdFromJson(block \ "Block ID") + val status = blockStatusFromJson(block \ "Status") + (id, status) + } + } + metrics + } + + def shuffleReadMetricsFromJson(json: JValue): ShuffleReadMetrics = { + val metrics = new ShuffleReadMetrics + metrics.shuffleFinishTime = (json \ "Shuffle Finish Time").extract[Long] + metrics.totalBlocksFetched = (json \ "Total Blocks Fetched").extract[Int] + metrics.remoteBlocksFetched = (json \ "Remote Blocks Fetched").extract[Int] + metrics.localBlocksFetched = (json \ "Local Blocks Fetched").extract[Int] + metrics.fetchWaitTime = (json \ "Fetch Wait Time").extract[Long] + metrics.remoteBytesRead = (json \ "Remote Bytes Read").extract[Long] + metrics + } + + def shuffleWriteMetricsFromJson(json: JValue): ShuffleWriteMetrics = { + val metrics = new ShuffleWriteMetrics + metrics.shuffleBytesWritten = (json \ "Shuffle Bytes Written").extract[Long] + metrics.shuffleWriteTime = (json \ "Shuffle Write Time").extract[Long] + metrics + } + + def taskEndReasonFromJson(json: JValue): TaskEndReason = { + val success = Utils.getFormattedClassName(Success) + val resubmitted = Utils.getFormattedClassName(Resubmitted) + val fetchFailed = Utils.getFormattedClassName(FetchFailed) + val exceptionFailure = Utils.getFormattedClassName(ExceptionFailure) + val taskResultLost = Utils.getFormattedClassName(TaskResultLost) + val taskKilled = Utils.getFormattedClassName(TaskKilled) + val executorLostFailure = Utils.getFormattedClassName(ExecutorLostFailure) + val unknownReason = Utils.getFormattedClassName(UnknownReason) + + (json \ "Reason").extract[String] match { + case `success` => Success + case `resubmitted` => Resubmitted + case `fetchFailed` => + val blockManagerAddress = blockManagerIdFromJson(json \ "Block Manager Address") + val shuffleId = (json \ "Shuffle ID").extract[Int] + val mapId = (json \ "Map ID").extract[Int] + val reduceId = (json \ "Reduce ID").extract[Int] + new FetchFailed(blockManagerAddress, shuffleId, mapId, reduceId) + case `exceptionFailure` => + val className = (json \ "Class Name").extract[String] + val description = (json \ "Description").extract[String] + val stackTrace = stackTraceFromJson(json \ "Stack Trace") + val metrics = Utils.jsonOption(json \ "Metrics").map(taskMetricsFromJson) + new ExceptionFailure(className, description, stackTrace, metrics) + case `taskResultLost` => TaskResultLost + case `taskKilled` => TaskKilled + case `executorLostFailure` => ExecutorLostFailure + case `unknownReason` => UnknownReason + } + } + + def blockManagerIdFromJson(json: JValue): BlockManagerId = { + val executorId = (json \ "Executor ID").extract[String] + val host = (json \ "Host").extract[String] + val port = (json \ "Port").extract[Int] + val nettyPort = (json \ "Netty Port").extract[Int] + BlockManagerId(executorId, host, port, nettyPort) + } + + def jobResultFromJson(json: JValue): JobResult = { + val jobSucceeded = Utils.getFormattedClassName(JobSucceeded) + val jobFailed = Utils.getFormattedClassName(JobFailed) + + (json \ "Result").extract[String] match { + case `jobSucceeded` => JobSucceeded + case `jobFailed` => + val exception = exceptionFromJson(json \ "Exception") + val failedStageId = (json \ "Failed Stage ID").extract[Int] + new JobFailed(exception, failedStageId) + } + } + + def rddInfoFromJson(json: JValue): RDDInfo = { + val rddId = (json \ "RDD ID").extract[Int] + val name = (json \ "Name").extract[String] + val storageLevel = storageLevelFromJson(json \ "Storage Level") + val numPartitions = (json \ "Number of Partitions").extract[Int] + val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] + val memSize = (json \ "Memory Size").extract[Long] + val diskSize = (json \ "Disk Size").extract[Long] + + val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) + rddInfo.numCachedPartitions = numCachedPartitions + rddInfo.memSize = memSize + rddInfo.diskSize = diskSize + rddInfo + } + + def storageLevelFromJson(json: JValue): StorageLevel = { + val useDisk = (json \ "Use Disk").extract[Boolean] + val useMemory = (json \ "Use Memory").extract[Boolean] + val deserialized = (json \ "Deserialized").extract[Boolean] + val replication = (json \ "Replication").extract[Int] + StorageLevel(useDisk, useMemory, deserialized, replication) + } + + def blockIdFromJson(json: JValue): BlockId = { + val rddBlockId = Utils.getFormattedClassName(RDDBlockId) + val shuffleBlockId = Utils.getFormattedClassName(ShuffleBlockId) + val broadcastBlockId = Utils.getFormattedClassName(BroadcastBlockId) + val broadcastHelperBlockId = Utils.getFormattedClassName(BroadcastHelperBlockId) + val taskResultBlockId = Utils.getFormattedClassName(TaskResultBlockId) + val streamBlockId = Utils.getFormattedClassName(StreamBlockId) + val tempBlockId = Utils.getFormattedClassName(TempBlockId) + val testBlockId = Utils.getFormattedClassName(TestBlockId) + + (json \ "Type").extract[String] match { + case `rddBlockId` => + val rddId = (json \ "RDD ID").extract[Int] + val splitIndex = (json \ "Split Index").extract[Int] + new RDDBlockId(rddId, splitIndex) + case `shuffleBlockId` => + val shuffleId = (json \ "Shuffle ID").extract[Int] + val mapId = (json \ "Map ID").extract[Int] + val reduceId = (json \ "Reduce ID").extract[Int] + new ShuffleBlockId(shuffleId, mapId, reduceId) + case `broadcastBlockId` => + val broadcastId = (json \ "Broadcast ID").extract[Long] + new BroadcastBlockId(broadcastId) + case `broadcastHelperBlockId` => + val broadcastBlockId = + blockIdFromJson(json \ "Broadcast Block ID").asInstanceOf[BroadcastBlockId] + val hType = (json \ "Helper Type").extract[String] + new BroadcastHelperBlockId(broadcastBlockId, hType) + case `taskResultBlockId` => + val taskId = (json \ "Task ID").extract[Long] + new TaskResultBlockId(taskId) + case `streamBlockId` => + val streamId = (json \ "Stream ID").extract[Int] + val uniqueId = (json \ "Unique ID").extract[Long] + new StreamBlockId(streamId, uniqueId) + case `tempBlockId` => + val tempId = UUIDFromJson(json \ "Temp ID") + new TempBlockId(tempId) + case `testBlockId` => + val testId = (json \ "Test ID").extract[String] + new TestBlockId(testId) + } + } + + def blockStatusFromJson(json: JValue): BlockStatus = { + val storageLevel = storageLevelFromJson(json \ "Storage Level") + val memorySize = (json \ "Memory Size").extract[Long] + val diskSize = (json \ "Disk Size").extract[Long] + BlockStatus(storageLevel, memorySize, diskSize) + } + + + /** -------------------------------- * + * Util JSON deserialization methods | + * --------------------------------- */ + + def mapFromJson(json: JValue): Map[String, String] = { + val jsonFields = json.asInstanceOf[JObject].obj + jsonFields.map { case JField(k, JString(v)) => (k, v) }.toMap + } + + def propertiesFromJson(json: JValue): Properties = { + val properties = new Properties() + if (json != JNothing) { + mapFromJson(json).map { case (k, v) => properties.setProperty(k, v) } + } + properties + } + + def UUIDFromJson(json: JValue): UUID = { + val leastSignificantBits = (json \ "Least Significant Bits").extract[Long] + val mostSignificantBits = (json \ "Most Significant Bits").extract[Long] + new UUID(leastSignificantBits, mostSignificantBits) + } + + def stackTraceFromJson(json: JValue): Array[StackTraceElement] = { + json.extract[List[JValue]].map { line => + val declaringClass = (line \ "Declaring Class").extract[String] + val methodName = (line \ "Method Name").extract[String] + val fileName = (line \ "File Name").extract[String] + val lineNumber = (line \ "Line Number").extract[Int] + new StackTraceElement(declaringClass, methodName, fileName, lineNumber) + }.toArray + } + + def exceptionFromJson(json: JValue): Exception = { + val e = new Exception((json \ "Message").extract[String]) + e.setStackTrace(stackTraceFromJson(json \ "Stack Trace")) + e + } + +} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 38a275d438959..13d9dbdd9af2d 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -32,11 +32,11 @@ import scala.reflect.ClassTag import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} +import org.json4s._ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} -import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} import org.apache.spark.deploy.SparkHadoopUtil - +import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} /** * Various utility methods used by Spark. @@ -245,7 +245,7 @@ private[spark] object Utils extends Logging { val userCred = securityMgr.getSecretKey() if (userCred == null) throw new Exception("Secret key is null with authentication on") val userInfo = securityMgr.getHttpUser() + ":" + userCred - new URI(uri.getScheme(), userInfo, uri.getHost(), uri.getPort(), uri.getPath(), + new URI(uri.getScheme(), userInfo, uri.getHost(), uri.getPort(), uri.getPath(), uri.getQuery(), uri.getFragment()) } @@ -282,7 +282,7 @@ private[spark] object Utils extends Logging { uc.setConnectTimeout(timeout) uc.setReadTimeout(timeout) uc.connect() - val in = uc.getInputStream(); + val in = uc.getInputStream() val out = new FileOutputStream(tempFile) Utils.copyStream(in, out, true) if (targetFile.exists && !Files.equal(tempFile, targetFile)) { @@ -328,8 +328,7 @@ private[spark] object Utils extends Logging { } case _ => // Use the Hadoop filesystem library, which supports file://, hdfs://, s3://, and others - val conf = SparkHadoopUtil.get.newConfiguration() - val fs = FileSystem.get(uri, conf) + val fs = getHadoopFileSystem(uri) val in = fs.open(new Path(uri)) val out = new FileOutputStream(tempFile) Utils.copyStream(in, out, true) @@ -500,7 +499,7 @@ private[spark] object Utils extends Logging { * millisecond. */ def getUsedTimeMs(startTimeMs: Long): String = { - return " " + (System.currentTimeMillis - startTimeMs) + " ms" + " " + (System.currentTimeMillis - startTimeMs) + " ms" } /** @@ -789,7 +788,7 @@ private[spark] object Utils extends Logging { } var i = 0 while (i < s.length) { - var nextChar = s.charAt(i) + val nextChar = s.charAt(i) if (inDoubleQuote) { if (nextChar == '"') { inDoubleQuote = false @@ -895,4 +894,27 @@ private[spark] object Utils extends Logging { } count } + + /** Return the class name of the given object, removing all dollar signs */ + def getFormattedClassName(obj: AnyRef) = { + obj.getClass.getSimpleName.replace("$", "") + } + + /** Return an option that translates JNothing to None */ + def jsonOption(json: JValue): Option[JValue] = { + json match { + case JNothing => None + case value: JValue => Some(value) + } + } + + /** Return an empty JSON object */ + def emptyJson = JObject(List[JField]()) + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + */ + def getHadoopFileSystem(path: URI): FileSystem = { + FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) + } } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 40e853c39ca99..c6b65c7348ae0 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -43,6 +43,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; +import org.apache.spark.executor.TaskMetrics; import org.apache.spark.partial.BoundedDouble; import org.apache.spark.partial.PartialResult; import org.apache.spark.storage.StorageLevel; @@ -402,16 +403,16 @@ public Boolean call(Double x) { @Test public void javaDoubleRDDHistoGram() { - JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); - // Test using generated buckets - Tuple2 results = rdd.histogram(2); - double[] expected_buckets = {1.0, 2.5, 4.0}; - long[] expected_counts = {2, 2}; - Assert.assertArrayEquals(expected_buckets, results._1, 0.1); - Assert.assertArrayEquals(expected_counts, results._2); - // Test with provided buckets - long[] histogram = rdd.histogram(expected_buckets); - Assert.assertArrayEquals(expected_counts, histogram); + JavaDoubleRDD rdd = sc.parallelizeDoubles(Arrays.asList(1.0, 2.0, 3.0, 4.0)); + // Test using generated buckets + Tuple2 results = rdd.histogram(2); + double[] expected_buckets = {1.0, 2.5, 4.0}; + long[] expected_counts = {2, 2}; + Assert.assertArrayEquals(expected_buckets, results._1, 0.1); + Assert.assertArrayEquals(expected_counts, results._2); + // Test with provided buckets + long[] histogram = rdd.histogram(expected_buckets); + Assert.assertArrayEquals(expected_counts, histogram); } @Test @@ -570,7 +571,7 @@ public void persist() { @Test public void iterator() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 2, 3, 4, 5), 2); - TaskContext context = new TaskContext(0, 0, 0, false, false, null); + TaskContext context = new TaskContext(0, 0, 0, false, false, new TaskMetrics()); Assert.assertEquals(1, rdd.iterator(rdd.splits().get(0), context).next().intValue()); } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index ea936e815b24a..b86923f07f793 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -23,7 +23,8 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.mock.EasyMockSugar import org.apache.spark.rdd.RDD -import org.apache.spark.storage.{BlockManager, RDDBlockId, StorageLevel} +import org.apache.spark.executor.TaskMetrics +import org.apache.spark.storage._ // TODO: Test the CacheManager's thread-safety aspects class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar { @@ -54,12 +55,12 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar expecting { blockManager.get(RDDBlockId(0, 0)).andReturn(None) blockManager.put(RDDBlockId(0, 0), ArrayBuffer[Any](1, 2, 3, 4), StorageLevel.MEMORY_ONLY, - true).andReturn(0) + true).andStubReturn(Seq[(BlockId, BlockStatus)]()) } whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = null) + taskMetrics = TaskMetrics.empty()) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -72,7 +73,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = null) + taskMetrics = TaskMetrics.empty()) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -86,7 +87,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false, - taskMetrics = null) + taskMetrics = TaskMetrics.empty()) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 20c503d30c801..7a39d1af9e2d5 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.matchers.ShouldMatchers import org.apache.spark.SparkContext._ -import org.apache.spark.scheduler.{SparkListenerTaskStart, SparkListener} +import org.apache.spark.scheduler.{SparkListener, SparkListenerTaskStart} /** * Test suite for cancelling running jobs. We run the cancellation tasks for single job action @@ -89,7 +89,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem.release() } @@ -161,7 +161,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf { // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem.release() } @@ -191,7 +191,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf { // Add a listener to release the semaphore once any tasks are launched. val sem = new Semaphore(0) - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem.release() } diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 0bac78d8a6bdf..6e7fd55fa4bb1 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -27,8 +27,11 @@ import org.apache.hadoop.fs.Path import scala.collection.Map import scala.sys.process._ import scala.util.Try + import org.apache.hadoop.io.{Text, LongWritable} +import org.apache.spark.executor.TaskMetrics + class PipedRDDSuite extends FunSuite with SharedSparkContext { test("basic pipe") { @@ -151,7 +154,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = null) + taskMetrics = TaskMetrics.empty()) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 3bb936790d506..b543471a5d35b 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark import org.scalatest.{FunSuite, PrivateMethodTester} -import org.apache.spark.scheduler.{TaskSchedulerImpl, TaskScheduler} +import org.apache.spark.scheduler.{TaskScheduler, TaskSchedulerImpl} import org.apache.spark.scheduler.cluster.{SimrSchedulerBackend, SparkDeploySchedulerBackend} import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, MesosSchedulerBackend} import org.apache.spark.scheduler.local.LocalBackend @@ -32,7 +32,7 @@ class SparkContextSchedulerCreationSuite // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. sc = new SparkContext("local", "test") val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) - val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master, "test") + val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] } diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index bae3b37e267d5..9f2924c23b73c 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -20,12 +20,9 @@ package org.apache.spark.deploy import java.io.File import java.util.Date +import com.fasterxml.jackson.core.JsonParseException import org.json4s._ - -import org.json4s.JValue import org.json4s.jackson.JsonMethods -import com.fasterxml.jackson.core.JsonParseException - import org.scalatest.FunSuite import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, WorkerStateResponse} @@ -96,7 +93,7 @@ class JsonProtocolSuite extends FunSuite { def createAppInfo() : ApplicationInfo = { val appInfo = new ApplicationInfo(JsonConstants.appInfoStartTime, - "id", createAppDesc(), JsonConstants.submitDate, null, "appUriStr", Int.MaxValue) + "id", createAppDesc(), JsonConstants.submitDate, null, Int.MaxValue) appInfo.endTime = JsonConstants.currTimeInMillis appInfo } @@ -148,12 +145,12 @@ object JsonConstants { val submitDate = new Date(123456789) val appInfoJsonStr = """ - |{"starttime":3,"id":"id","name":"name","appuiurl":"appUriStr", + |{"starttime":3,"id":"id","name":"name", |"cores":4,"user":"%s", |"memoryperslave":1234,"submitdate":"%s", |"state":"WAITING","duration":%d} """.format(System.getProperty("user.name", ""), - submitDate.toString, (currTimeInMillis - appInfoStartTime)).stripMargin + submitDate.toString, currTimeInMillis - appInfoStartTime).stripMargin val workerInfoJsonStr = """ diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ad890b4e4d08e..c97543f57d8f3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -94,7 +94,12 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) - scheduler = new DAGScheduler(taskScheduler, mapOutputTracker, blockManagerMaster, sc.env) { + scheduler = new DAGScheduler( + taskScheduler, + sc.listenerBus, + mapOutputTracker, + blockManagerMaster, + sc.env) { override def runLocally(job: ActiveJob) { // don't bother with the thread while unit testing runLocallyWithinThread(job) @@ -422,15 +427,15 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont private def assertDataStructuresEmpty = { assert(scheduler.pendingTasks.isEmpty) assert(scheduler.activeJobs.isEmpty) - assert(scheduler.failed.isEmpty) - assert(scheduler.idToActiveJob.isEmpty) + assert(scheduler.failedStages.isEmpty) + assert(scheduler.stageIdToActiveJob.isEmpty) assert(scheduler.jobIdToStageIds.isEmpty) assert(scheduler.stageIdToJobIds.isEmpty) assert(scheduler.stageIdToStage.isEmpty) assert(scheduler.stageToInfos.isEmpty) assert(scheduler.resultStageToJob.isEmpty) - assert(scheduler.running.isEmpty) + assert(scheduler.runningStages.isEmpty) assert(scheduler.shuffleToMapStage.isEmpty) - assert(scheduler.waiting.isEmpty) + assert(scheduler.waitingStages.isEmpty) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala deleted file mode 100644 index 25fe63c2655a8..0000000000000 --- a/core/src/test/scala/org/apache/spark/scheduler/JobLoggerSuite.scala +++ /dev/null @@ -1,126 +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.scheduler - -import org.scalatest.FunSuite -import org.scalatest.matchers.ShouldMatchers - -import org.apache.spark._ -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -class JobLoggerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { - val WAIT_TIMEOUT_MILLIS = 10000 - - test("inner method") { - sc = new SparkContext("local", "joblogger") - val joblogger = new JobLogger { - def createLogWriterTest(jobID: Int) = createLogWriter(jobID) - def closeLogWriterTest(jobID: Int) = closeLogWriter(jobID) - def getRddNameTest(rdd: RDD[_]) = getRddName(rdd) - def buildJobDepTest(jobID: Int, stage: Stage) = buildJobDep(jobID, stage) - } - type MyRDD = RDD[(Int, Int)] - def makeRdd(numPartitions: Int, dependencies: List[Dependency[_]]): MyRDD = { - val maxPartition = numPartitions - 1 - new MyRDD(sc, dependencies) { - override def compute(split: Partition, context: TaskContext): Iterator[(Int, Int)] = - throw new RuntimeException("should not be reached") - override def getPartitions = (0 to maxPartition).map(i => new Partition { - override def index = i - }).toArray - } - } - val jobID = 5 - val parentRdd = makeRdd(4, Nil) - val shuffleDep = new ShuffleDependency(parentRdd, null) - val rootRdd = makeRdd(4, List(shuffleDep)) - val shuffleMapStage = - new Stage(1, parentRdd, parentRdd.partitions.size, Some(shuffleDep), Nil, jobID, None) - val rootStage = - new Stage(0, rootRdd, rootRdd.partitions.size, None, List(shuffleMapStage), jobID, None) - val rootStageInfo = new StageInfo(rootStage) - - joblogger.onStageSubmitted(SparkListenerStageSubmitted(rootStageInfo, null)) - joblogger.getRddNameTest(parentRdd) should be (parentRdd.getClass.getSimpleName) - parentRdd.setName("MyRDD") - joblogger.getRddNameTest(parentRdd) should be ("MyRDD") - joblogger.createLogWriterTest(jobID) - joblogger.getJobIDtoPrintWriter.size should be (1) - joblogger.buildJobDepTest(jobID, rootStage) - joblogger.getJobIDToStages.get(jobID).get.size should be (2) - joblogger.getStageIDToJobID.get(0) should be (Some(jobID)) - joblogger.getStageIDToJobID.get(1) should be (Some(jobID)) - joblogger.closeLogWriterTest(jobID) - joblogger.getStageIDToJobID.size should be (0) - joblogger.getJobIDToStages.size should be (0) - joblogger.getJobIDtoPrintWriter.size should be (0) - } - - test("inner variables") { - sc = new SparkContext("local[4]", "joblogger") - val joblogger = new JobLogger { - override protected def closeLogWriter(jobID: Int) = - getJobIDtoPrintWriter.get(jobID).foreach { fileWriter => - fileWriter.close() - } - } - sc.addSparkListener(joblogger) - val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } - rdd.reduceByKey(_+_).collect() - - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - - val user = System.getProperty("user.name", SparkContext.SPARK_UNKNOWN_USER) - - joblogger.getLogDir should be ("/tmp/spark-%s".format(user)) - joblogger.getJobIDtoPrintWriter.size should be (1) - joblogger.getStageIDToJobID.size should be (2) - joblogger.getStageIDToJobID.get(0) should be (Some(0)) - joblogger.getStageIDToJobID.get(1) should be (Some(0)) - joblogger.getJobIDToStages.size should be (1) - } - - - test("interface functions") { - sc = new SparkContext("local[4]", "joblogger") - val joblogger = new JobLogger { - var onTaskEndCount = 0 - var onJobEndCount = 0 - var onJobStartCount = 0 - var onStageCompletedCount = 0 - var onStageSubmittedCount = 0 - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = onTaskEndCount += 1 - override def onJobEnd(jobEnd: SparkListenerJobEnd) = onJobEndCount += 1 - override def onJobStart(jobStart: SparkListenerJobStart) = onJobStartCount += 1 - override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = onStageCompletedCount += 1 - override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = onStageSubmittedCount += 1 - } - sc.addSparkListener(joblogger) - val rdd = sc.parallelize(1 to 1e2.toInt, 4).map{ i => (i % 12, 2 * i) } - rdd.reduceByKey(_+_).collect() - - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) - - joblogger.onJobStartCount should be (1) - joblogger.onJobEndCount should be (1) - joblogger.onTaskEndCount should be (8) - joblogger.onStageSubmittedCount should be (2) - joblogger.onStageCompletedCount should be (2) - } -} diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 7c4f2b4361892..a25ce35736146 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -17,13 +17,14 @@ package org.apache.spark.scheduler -import scala.collection.mutable.{Buffer, HashSet} +import scala.collection.mutable import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} import org.scalatest.matchers.ShouldMatchers import org.apache.spark.{LocalSparkContext, SparkContext} import org.apache.spark.SparkContext._ +import org.apache.spark.executor.TaskMetrics class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers with BeforeAndAfter with BeforeAndAfterAll { @@ -38,43 +39,76 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc System.clearProperty("spark.akka.frameSize") } + test("basic creation and shutdown of LiveListenerBus") { + val counter = new BasicJobCounter + val bus = new LiveListenerBus + bus.addListener(counter) + + // Listener bus hasn't started yet, so posting events should not increment counter + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + assert(counter.count === 0) + + // Starting listener bus should flush all buffered events (asynchronously, hence the sleep) + bus.start() + Thread.sleep(1000) + assert(counter.count === 5) + + // After listener bus has stopped, posting events should not increment counter + bus.stop() + (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } + assert(counter.count === 5) + + // Listener bus must not be started twice + intercept[IllegalStateException] { + val bus = new LiveListenerBus + bus.start() + bus.start() + } + + // ... or stopped before starting + intercept[IllegalStateException] { + val bus = new LiveListenerBus + bus.stop() + } + } + test("basic creation of StageInfo") { - val listener = new SaveStageInfo + val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) - val rdd2 = rdd1.map(x => x.toString) + val rdd2 = rdd1.map(_.toString) rdd2.setName("Target RDD") rdd2.count - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} - val first = listener.stageInfos.head - first.rddName should be {"Target RDD"} - first.numTasks should be {4} - first.numPartitions should be {4} - first.submissionTime should be ('defined) - first.completionTime should be ('defined) - first.taskInfos.length should be {4} + val (stageInfo, taskInfoMetrics) = listener.stageInfos.head + stageInfo.rddInfo.name should be {"Target RDD"} + stageInfo.numTasks should be {4} + stageInfo.rddInfo.numPartitions should be {4} + stageInfo.submissionTime should be ('defined) + stageInfo.completionTime should be ('defined) + taskInfoMetrics.length should be {4} } test("StageInfo with fewer tasks than partitions") { - val listener = new SaveStageInfo + val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) val rdd1 = sc.parallelize(1 to 100, 4) - val rdd2 = rdd1.map(x => x.toString) + val rdd2 = rdd1.map(_.toString) sc.runJob(rdd2, (items: Iterator[String]) => items.size, Seq(0, 1), true) - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} - val first = listener.stageInfos.head - first.numTasks should be {2} - first.numPartitions should be {4} + val (stageInfo, _) = listener.stageInfos.head + stageInfo.numTasks should be {2} + stageInfo.rddInfo.numPartitions should be {4} } test("local metrics") { - val listener = new SaveStageInfo + val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) //just to make sure some of the tasks take a noticeable amount of time @@ -84,45 +118,45 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc i } - val d = sc.parallelize(0 to 1e4.toInt, 64).map{i => w(i)} + val d = sc.parallelize(0 to 1e4.toInt, 64).map(w) d.count() - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be (1) - val d2 = d.map{i => w(i) -> i * 2}.setName("shuffle input 1") - - val d3 = d.map{i => w(i) -> (0 to (i % 5))}.setName("shuffle input 2") - - val d4 = d2.cogroup(d3, 64).map{case(k,(v1,v2)) => w(k) -> (v1.size, v2.size)} + val d2 = d.map { i => w(i) -> i * 2 }.setName("shuffle input 1") + val d3 = d.map { i => w(i) -> (0 to (i % 5)) }.setName("shuffle input 2") + val d4 = d2.cogroup(d3, 64).map { case (k, (v1, v2)) => + w(k) -> (v1.size, v2.size) + } d4.setName("A Cogroup") - d4.collectAsMap() - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be (4) - listener.stageInfos.foreach { stageInfo => - /* small test, so some tasks might take less than 1 millisecond, but average should be greater - * than 0 ms. */ - checkNonZeroAvg(stageInfo.taskInfos.map{_._1.duration}, stageInfo + " duration") + listener.stageInfos.foreach { case (stageInfo, taskInfoMetrics) => + /** + * Small test, so some tasks might take less than 1 millisecond, but average should be greater + * than 0 ms. + */ checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.executorRunTime.toLong}, + taskInfoMetrics.map(_._2.executorRunTime), stageInfo + " executorRunTime") checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.executorDeserializeTime.toLong}, + taskInfoMetrics.map(_._2.executorDeserializeTime), stageInfo + " executorDeserializeTime") - if (stageInfo.rddName == d4.name) { + if (stageInfo.rddInfo.name == d4.name) { checkNonZeroAvg( - stageInfo.taskInfos.map{_._2.shuffleReadMetrics.get.fetchWaitTime}, + taskInfoMetrics.map(_._2.shuffleReadMetrics.get.fetchWaitTime), stageInfo + " fetchWaitTime") } - stageInfo.taskInfos.foreach { case (taskInfo, taskMetrics) => + taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) - if (stageInfo.rddName == d2.name || stageInfo.rddName == d3.name) { + if (stageInfo.rddInfo.name == d2.name || stageInfo.rddInfo.name == d3.name) { taskMetrics.shuffleWriteMetrics should be ('defined) taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) } - if (stageInfo.rddName == d4.name) { + if (stageInfo.rddInfo.name == d4.name) { taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get sm.totalBlocksFetched should be > (0) @@ -142,10 +176,12 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc System.setProperty("spark.akka.frameSize", "1") val akkaFrameSize = sc.env.actorSystem.settings.config.getBytes("akka.remote.netty.tcp.maximum-frame-size").toInt - val result = sc.parallelize(Seq(1), 1).map(x => 1.to(akkaFrameSize).toArray).reduce((x,y) => x) + val result = sc.parallelize(Seq(1), 1) + .map { x => 1.to(akkaFrameSize).toArray } + .reduce { case (x, y) => x } assert(result === 1.to(akkaFrameSize).toArray) - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) assert(listener.startedGettingResultTasks.contains(TASK_INDEX)) @@ -157,13 +193,13 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc.addSparkListener(listener) // Make a task whose result is larger than the akka frame size - val result = sc.parallelize(Seq(1), 1).map(x => 2 * x).reduce((x, y) => x) + val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) - assert(sc.dagScheduler.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) val TASK_INDEX = 0 assert(listener.startedTasks.contains(TASK_INDEX)) - assert(listener.startedGettingResultTasks.isEmpty == true) + assert(listener.startedGettingResultTasks.isEmpty) assert(listener.endedTasks.contains(TASK_INDEX)) } @@ -204,17 +240,33 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc assert(m.sum / m.size.toDouble > 0.0, msg) } - class SaveStageInfo extends SparkListener { - val stageInfos = Buffer[StageInfo]() + class BasicJobCounter extends SparkListener { + var count = 0 + override def onJobEnd(job: SparkListenerJobEnd) = count += 1 + } + + class SaveStageAndTaskInfo extends SparkListener { + val stageInfos = mutable.Map[StageInfo, Seq[(TaskInfo, TaskMetrics)]]() + var taskInfoMetrics = mutable.Buffer[(TaskInfo, TaskMetrics)]() + + override def onTaskEnd(task: SparkListenerTaskEnd) { + val info = task.taskInfo + val metrics = task.taskMetrics + if (info != null && metrics != null) { + taskInfoMetrics += ((info, metrics)) + } + } + override def onStageCompleted(stage: SparkListenerStageCompleted) { - stageInfos += stage.stage + stageInfos(stage.stageInfo) = taskInfoMetrics + taskInfoMetrics = mutable.Buffer.empty } } class SaveTaskEvents extends SparkListener { - val startedTasks = new HashSet[Int]() - val startedGettingResultTasks = new HashSet[Int]() - val endedTasks = new HashSet[Int]() + val startedTasks = new mutable.HashSet[Int]() + val startedGettingResultTasks = new mutable.HashSet[Int]() + val endedTasks = new mutable.HashSet[Int]() override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { startedTasks += taskStart.taskInfo.index diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 6b0800af9c6d0..9274e01632d58 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark.scheduler import java.util.Properties -import scala.collection.mutable.ArrayBuffer - import org.scalatest.FunSuite import org.apache.spark._ @@ -270,9 +268,9 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin val taskScheduler = new TaskSchedulerImpl(sc) taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. - var dagScheduler = new DAGScheduler(taskScheduler) { + val dagScheduler = new DAGScheduler(sc, taskScheduler) { override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} - override def executorGained(execId: String, host: String) {} + override def executorAdded(execId: String, host: String) {} } val numFreeCores = 1 @@ -291,7 +289,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin assert(1 === taskDescriptions.length) taskDescriptions(0).executorId } - var count = selectedExecutorIds.count(_ == workerOffers(0).executorId) + val count = selectedExecutorIds.count(_ == workerOffers(0).executorId) assert(count > 0) assert(count < numTrials) } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 73153d23c4698..9af5d3a303ae2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -26,7 +26,9 @@ import org.apache.spark._ import org.apache.spark.executor.TaskMetrics import org.apache.spark.util.FakeClock -class FakeDAGScheduler(taskScheduler: FakeTaskScheduler) extends DAGScheduler(taskScheduler) { +class FakeDAGScheduler(sc: SparkContext, taskScheduler: FakeTaskScheduler) + extends DAGScheduler(sc) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) { taskScheduler.startedTasks += taskInfo.index } @@ -41,7 +43,7 @@ class FakeDAGScheduler(taskScheduler: FakeTaskScheduler) extends DAGScheduler(ta taskScheduler.endedTasks(taskInfo.index) = reason } - override def executorGained(execId: String, host: String) {} + override def executorAdded(execId: String, host: String) {} override def executorLost(execId: String) {} @@ -66,7 +68,7 @@ class FakeTaskScheduler(sc: SparkContext, liveExecutors: (String, String)* /* ex val executors = new mutable.HashMap[String, String] ++ liveExecutors - dagScheduler = new FakeDAGScheduler(this) + dagScheduler = new FakeDAGScheduler(sc, this) def removeExecutor(execId: String): Unit = executors -= execId diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index 1036b9f34e9dd..e83cd55e73691 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -28,7 +28,8 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SecurityManager, SparkConf, SparkContext} +import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} @@ -57,7 +58,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT conf.set("spark.driver.port", boundPort.toString) master = new BlockManagerMaster( - actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf))), conf) + actorSystem.actorOf(Props(new BlockManagerMasterActor(true, conf, new LiveListenerBus))), + conf) // Set the arch to 64-bit and compressedOops to true to get a deterministic test-case oldArch = System.setProperty("os.arch", "amd64") @@ -492,12 +494,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store.putSingle("a2", a2, StorageLevel.MEMORY_ONLY_SER) store.putSingle("a3", a3, StorageLevel.DISK_ONLY) // At this point LRU should not kick in because a3 is only on disk - assert(store.getSingle("a1").isDefined, "a2 was not in store") - assert(store.getSingle("a2").isDefined, "a3 was not in store") - assert(store.getSingle("a3").isDefined, "a1 was not in store") - assert(store.getSingle("a1").isDefined, "a2 was not in store") - assert(store.getSingle("a2").isDefined, "a3 was not in store") - assert(store.getSingle("a3").isDefined, "a1 was not in store") + assert(store.getSingle("a1").isDefined, "a1 was not in store") + assert(store.getSingle("a2").isDefined, "a2 was not in store") + assert(store.getSingle("a3").isDefined, "a3 was not in store") // Now let's add in a4, which uses both disk and memory; a1 should drop out store.putSingle("a4", a4, StorageLevel.MEMORY_AND_DISK_SER) assert(store.getSingle("a1") == None, "a1 was in store") @@ -663,6 +662,60 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } } + test("updated block statuses") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + val list = List.fill(2)(new Array[Byte](200)) + val bigList = List.fill(8)(new Array[Byte](200)) + + // 1 updated block (i.e. list1) + val updatedBlocks1 = + store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks1.size === 1) + assert(updatedBlocks1.head._1 === TestBlockId("list1")) + assert(updatedBlocks1.head._2.storageLevel === StorageLevel.MEMORY_ONLY) + + // 1 updated block (i.e. list2) + val updatedBlocks2 = + store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + assert(updatedBlocks2.size === 1) + assert(updatedBlocks2.head._1 === TestBlockId("list2")) + assert(updatedBlocks2.head._2.storageLevel === StorageLevel.MEMORY_ONLY) + + // 2 updated blocks - list1 is kicked out of memory while list3 is added + val updatedBlocks3 = + store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks3.size === 2) + updatedBlocks3.foreach { case (id, status) => + id match { + case TestBlockId("list1") => assert(status.storageLevel === StorageLevel.NONE) + case TestBlockId("list3") => assert(status.storageLevel === StorageLevel.MEMORY_ONLY) + case _ => fail("Updated block is neither list1 nor list3") + } + } + assert(store.get("list3").isDefined, "list3 was not in store") + + // 2 updated blocks - list2 is kicked out of memory (but put on disk) while list4 is added + val updatedBlocks4 = + store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks4.size === 2) + updatedBlocks4.foreach { case (id, status) => + id match { + case TestBlockId("list2") => assert(status.storageLevel === StorageLevel.DISK_ONLY) + case TestBlockId("list4") => assert(status.storageLevel === StorageLevel.MEMORY_ONLY) + case _ => fail("Updated block is neither list2 nor list4") + } + } + assert(store.get("list4").isDefined, "list4 was not in store") + + // No updated blocks - nothing is kicked out of memory because list5 is too big to be added + val updatedBlocks5 = + store.put("list5", bigList.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + assert(updatedBlocks5.size === 0) + assert(store.get("list2").isDefined, "list2 was not in store") + assert(store.get("list4").isDefined, "list4 was not in store") + assert(!store.get("list5").isDefined, "list5 was in store") + } + test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 30415814adbba..45c322427930d 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -22,6 +22,7 @@ import java.net.ServerSocket import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server +import org.eclipse.jetty.servlet.ServletContextHandler import org.scalatest.FunSuite import org.apache.spark.SparkConf @@ -36,22 +37,27 @@ class UISuite extends FunSuite { case Failure(e) => // Either case server port is busy hence setup for test complete } - val (jettyServer1, boundPort1) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq(), - new SparkConf) - val (jettyServer2, boundPort2) = JettyUtils.startJettyServer("0.0.0.0", startPort, Seq(), - new SparkConf) + val serverInfo1 = JettyUtils.startJettyServer( + "0.0.0.0", startPort, Seq[ServletContextHandler](), new SparkConf) + val serverInfo2 = JettyUtils.startJettyServer( + "0.0.0.0", startPort, Seq[ServletContextHandler](), new SparkConf) // Allow some wiggle room in case ports on the machine are under contention + val boundPort1 = serverInfo1.boundPort + val boundPort2 = serverInfo2.boundPort assert(boundPort1 > startPort && boundPort1 < startPort + 10) assert(boundPort2 > boundPort1 && boundPort2 < boundPort1 + 10) } test("jetty binds to port 0 correctly") { - val (jettyServer, boundPort) = JettyUtils.startJettyServer("0.0.0.0", 0, Seq(), new SparkConf) - assert(jettyServer.getState === "STARTED") + val serverInfo = JettyUtils.startJettyServer( + "0.0.0.0", 0, Seq[ServletContextHandler](), new SparkConf) + val server = serverInfo.server + val boundPort = serverInfo.boundPort + assert(server.getState === "STARTED") assert(boundPort != 0) - Try {new ServerSocket(boundPort)} match { + Try { new ServerSocket(boundPort) } match { case Success(s) => fail("Port %s doesn't seem used by jetty server".format(boundPort)) - case Failure (e) => + case Failure(e) => } } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 8ca863e8b3667..d8a3e859f85cd 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -22,11 +22,12 @@ import org.scalatest.FunSuite import org.apache.spark.{LocalSparkContext, SparkContext, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ +import org.apache.spark.util.Utils class JobProgressListenerSuite extends FunSuite with LocalSparkContext { test("test executor id to summary") { val sc = new SparkContext("local", "test") - val listener = new JobProgressListener(sc) + val listener = new JobProgressListener(sc.conf) val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() @@ -38,16 +39,17 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) var taskInfo = new TaskInfo(1234L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + var task = new ShuffleMapTask(0, null, null, 0, null) + val taskType = Utils.getFormattedClassName(task) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) .shuffleRead == 1000) // finish a task with unknown executor-id, nothing should happen taskInfo = new TaskInfo(1234L, 0, 1000L, "exe-unknown", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + task = new ShuffleMapTask(0, null, null, 0, null) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.size == 1) // finish this task, should get updated duration @@ -55,8 +57,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1235L, 0, 0L, "exe-1", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + task = new ShuffleMapTask(0, null, null, 0, null) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-1", fail()) .shuffleRead == 2000) @@ -65,8 +67,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext { taskMetrics.shuffleReadMetrics = Some(shuffleReadMetrics) taskInfo = new TaskInfo(1236L, 0, 0L, "exe-2", "host1", TaskLocality.NODE_LOCAL) taskInfo.finishTime = 1 - listener.onTaskEnd(new SparkListenerTaskEnd( - new ShuffleMapTask(0, null, null, 0, null), Success, taskInfo, taskMetrics)) + task = new ShuffleMapTask(0, null, null, 0, null) + listener.onTaskEnd(SparkListenerTaskEnd(task.stageId, taskType, Success, taskInfo, taskMetrics)) assert(listener.stageIdToExecutorSummaries.getOrElse(0, fail()).getOrElse("exe-2", fail()) .shuffleRead == 1000) } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala new file mode 100644 index 0000000000000..67c0a434c9b52 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -0,0 +1,559 @@ +/* + * 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.util + +import java.util.{Properties, UUID} + +import scala.collection.Map + +import org.json4s.jackson.JsonMethods._ +import org.scalatest.FunSuite + +import org.apache.spark._ +import org.apache.spark.executor._ +import org.apache.spark.scheduler._ +import org.apache.spark.storage._ + +class JsonProtocolSuite extends FunSuite { + + test("SparkListenerEvent") { + val stageSubmitted = + SparkListenerStageSubmitted(makeStageInfo(100, 200, 300, 400L, 500L), properties) + val stageCompleted = SparkListenerStageCompleted(makeStageInfo(101, 201, 301, 401L, 501L)) + val taskStart = SparkListenerTaskStart(111, makeTaskInfo(222L, 333, 444L)) + val taskGettingResult = SparkListenerTaskGettingResult(makeTaskInfo(1000L, 2000, 3000L)) + val taskEnd = SparkListenerTaskEnd(1, "ShuffleMapTask", Success, + makeTaskInfo(123L, 234, 345L), makeTaskMetrics(300L, 400L, 500L, 600L, 700, 800)) + val jobStart = SparkListenerJobStart(10, Seq[Int](1, 2, 3, 4), properties) + val jobEnd = SparkListenerJobEnd(20, JobSucceeded) + val environmentUpdate = SparkListenerEnvironmentUpdate(Map[String, Seq[(String, String)]]( + "JVM Information" -> Seq(("GC speed", "9999 objects/s"), ("Java home", "Land of coffee")), + "Spark Properties" -> Seq(("Job throughput", "80000 jobs/s, regardless of job type")), + "System Properties" -> Seq(("Username", "guest"), ("Password", "guest")), + "Classpath Entries" -> Seq(("Super library", "/tmp/super_library")) + )) + val blockManagerAdded = SparkListenerBlockManagerAdded( + BlockManagerId("Stars", "In your multitude...", 300, 400), 500) + val blockManagerRemoved = SparkListenerBlockManagerRemoved( + BlockManagerId("Scarce", "to be counted...", 100, 200)) + val unpersistRdd = SparkListenerUnpersistRDD(12345) + + testEvent(stageSubmitted, stageSubmittedJsonString) + testEvent(stageCompleted, stageCompletedJsonString) + testEvent(taskStart, taskStartJsonString) + testEvent(taskGettingResult, taskGettingResultJsonString) + testEvent(taskEnd, taskEndJsonString) + testEvent(jobStart, jobStartJsonString) + testEvent(jobEnd, jobEndJsonString) + testEvent(environmentUpdate, environmentUpdateJsonString) + testEvent(blockManagerAdded, blockManagerAddedJsonString) + testEvent(blockManagerRemoved, blockManagerRemovedJsonString) + testEvent(unpersistRdd, unpersistRDDJsonString) + } + + test("Dependent Classes") { + testRDDInfo(makeRddInfo(2, 3, 4, 5L, 6L)) + testStageInfo(makeStageInfo(10, 20, 30, 40L, 50L)) + testTaskInfo(makeTaskInfo(999L, 888, 777L)) + testTaskMetrics(makeTaskMetrics(33333L, 44444L, 55555L, 66666L, 7, 8)) + testBlockManagerId(BlockManagerId("Hong", "Kong", 500, 1000)) + + // StorageLevel + testStorageLevel(StorageLevel.NONE) + testStorageLevel(StorageLevel.DISK_ONLY) + testStorageLevel(StorageLevel.DISK_ONLY_2) + testStorageLevel(StorageLevel.MEMORY_ONLY) + testStorageLevel(StorageLevel.MEMORY_ONLY_2) + testStorageLevel(StorageLevel.MEMORY_ONLY_SER) + testStorageLevel(StorageLevel.MEMORY_ONLY_SER_2) + testStorageLevel(StorageLevel.MEMORY_AND_DISK) + testStorageLevel(StorageLevel.MEMORY_AND_DISK_2) + testStorageLevel(StorageLevel.MEMORY_AND_DISK_SER) + testStorageLevel(StorageLevel.MEMORY_AND_DISK_SER_2) + + // JobResult + val exception = new Exception("Out of Memory! Please restock film.") + exception.setStackTrace(stackTrace) + val jobFailed = JobFailed(exception, 2) + testJobResult(JobSucceeded) + testJobResult(jobFailed) + + // TaskEndReason + val fetchFailed = FetchFailed(BlockManagerId("With or", "without you", 15, 16), 17, 18, 19) + val exceptionFailure = ExceptionFailure("To be", "or not to be", stackTrace, None) + testTaskEndReason(Success) + testTaskEndReason(Resubmitted) + testTaskEndReason(fetchFailed) + testTaskEndReason(exceptionFailure) + testTaskEndReason(TaskResultLost) + testTaskEndReason(TaskKilled) + testTaskEndReason(ExecutorLostFailure) + testTaskEndReason(UnknownReason) + + // BlockId + testBlockId(RDDBlockId(1, 2)) + testBlockId(ShuffleBlockId(1, 2, 3)) + testBlockId(BroadcastBlockId(1L)) + testBlockId(BroadcastHelperBlockId(BroadcastBlockId(2L), "Spark")) + testBlockId(TaskResultBlockId(1L)) + testBlockId(StreamBlockId(1, 2L)) + testBlockId(TempBlockId(UUID.randomUUID())) + } + + + /** -------------------------- * + | Helper test running methods | + * --------------------------- */ + + private def testEvent(event: SparkListenerEvent, jsonString: String) { + val actualJsonString = compact(render(JsonProtocol.sparkEventToJson(event))) + val newEvent = JsonProtocol.sparkEventFromJson(parse(actualJsonString)) + assertJsonStringEquals(jsonString, actualJsonString) + assertEquals(event, newEvent) + } + + private def testRDDInfo(info: RDDInfo) { + val newInfo = JsonProtocol.rddInfoFromJson(JsonProtocol.rddInfoToJson(info)) + assertEquals(info, newInfo) + } + + private def testStageInfo(info: StageInfo) { + val newInfo = JsonProtocol.stageInfoFromJson(JsonProtocol.stageInfoToJson(info)) + assertEquals(info, newInfo) + } + + private def testStorageLevel(level: StorageLevel) { + val newLevel = JsonProtocol.storageLevelFromJson(JsonProtocol.storageLevelToJson(level)) + assertEquals(level, newLevel) + } + + private def testTaskMetrics(metrics: TaskMetrics) { + val newMetrics = JsonProtocol.taskMetricsFromJson(JsonProtocol.taskMetricsToJson(metrics)) + assertEquals(metrics, newMetrics) + } + + private def testBlockManagerId(id: BlockManagerId) { + val newId = JsonProtocol.blockManagerIdFromJson(JsonProtocol.blockManagerIdToJson(id)) + assertEquals(id, newId) + } + + private def testTaskInfo(info: TaskInfo) { + val newInfo = JsonProtocol.taskInfoFromJson(JsonProtocol.taskInfoToJson(info)) + assertEquals(info, newInfo) + } + + private def testJobResult(result: JobResult) { + val newResult = JsonProtocol.jobResultFromJson(JsonProtocol.jobResultToJson(result)) + assertEquals(result, newResult) + } + + private def testTaskEndReason(reason: TaskEndReason) { + val newReason = JsonProtocol.taskEndReasonFromJson(JsonProtocol.taskEndReasonToJson(reason)) + assertEquals(reason, newReason) + } + + private def testBlockId(blockId: BlockId) { + val newBlockId = JsonProtocol.blockIdFromJson(JsonProtocol.blockIdToJson(blockId)) + blockId == newBlockId + } + + + /** -------------------------------- * + | Util methods for comparing events | + * --------------------------------- */ + + private def assertEquals(event1: SparkListenerEvent, event2: SparkListenerEvent) { + (event1, event2) match { + case (e1: SparkListenerStageSubmitted, e2: SparkListenerStageSubmitted) => + assert(e1.properties == e2.properties) + assertEquals(e1.stageInfo, e2.stageInfo) + case (e1: SparkListenerStageCompleted, e2: SparkListenerStageCompleted) => + assertEquals(e1.stageInfo, e2.stageInfo) + case (e1: SparkListenerTaskStart, e2: SparkListenerTaskStart) => + assert(e1.stageId == e2.stageId) + assertEquals(e1.taskInfo, e2.taskInfo) + case (e1: SparkListenerTaskGettingResult, e2: SparkListenerTaskGettingResult) => + assertEquals(e1.taskInfo, e2.taskInfo) + case (e1: SparkListenerTaskEnd, e2: SparkListenerTaskEnd) => + assert(e1.stageId == e2.stageId) + assert(e1.taskType == e2.taskType) + assertEquals(e1.reason, e2.reason) + assertEquals(e1.taskInfo, e2.taskInfo) + assertEquals(e1.taskMetrics, e2.taskMetrics) + case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) => + assert(e1.jobId == e2.jobId) + assert(e1.properties == e2.properties) + assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 == i2)) + case (e1: SparkListenerJobEnd, e2: SparkListenerJobEnd) => + assert(e1.jobId == e2.jobId) + assertEquals(e1.jobResult, e2.jobResult) + case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) => + assertEquals(e1.environmentDetails, e2.environmentDetails) + case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) => + assert(e1.maxMem == e2.maxMem) + assertEquals(e1.blockManagerId, e2.blockManagerId) + case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => + assertEquals(e1.blockManagerId, e2.blockManagerId) + case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => + assert(e1.rddId == e2.rddId) + case (SparkListenerShutdown, SparkListenerShutdown) => + case _ => fail("Events don't match in types!") + } + } + + private def assertEquals(info1: StageInfo, info2: StageInfo) { + assert(info1.stageId == info2.stageId) + assert(info1.name == info2.name) + assert(info1.numTasks == info2.numTasks) + assert(info1.submissionTime == info2.submissionTime) + assert(info1.completionTime == info2.completionTime) + assert(info1.emittedTaskSizeWarning == info2.emittedTaskSizeWarning) + assertEquals(info1.rddInfo, info2.rddInfo) + } + + private def assertEquals(info1: RDDInfo, info2: RDDInfo) { + assert(info1.id == info2.id) + assert(info1.name == info2.name) + assert(info1.numPartitions == info2.numPartitions) + assert(info1.numCachedPartitions == info2.numCachedPartitions) + assert(info1.memSize == info2.memSize) + assert(info1.diskSize == info2.diskSize) + assertEquals(info1.storageLevel, info2.storageLevel) + } + + private def assertEquals(level1: StorageLevel, level2: StorageLevel) { + assert(level1.useDisk == level2.useDisk) + assert(level1.useMemory == level2.useMemory) + assert(level1.deserialized == level2.deserialized) + assert(level1.replication == level2.replication) + } + + private def assertEquals(info1: TaskInfo, info2: TaskInfo) { + assert(info1.taskId == info2.taskId) + assert(info1.index == info2.index) + assert(info1.launchTime == info2.launchTime) + assert(info1.executorId == info2.executorId) + assert(info1.host == info2.host) + assert(info1.taskLocality == info2.taskLocality) + assert(info1.gettingResultTime == info2.gettingResultTime) + assert(info1.finishTime == info2.finishTime) + assert(info1.failed == info2.failed) + assert(info1.serializedSize == info2.serializedSize) + } + + private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) { + assert(metrics1.hostname == metrics2.hostname) + assert(metrics1.executorDeserializeTime == metrics2.executorDeserializeTime) + assert(metrics1.resultSize == metrics2.resultSize) + assert(metrics1.jvmGCTime == metrics2.jvmGCTime) + assert(metrics1.resultSerializationTime == metrics2.resultSerializationTime) + assert(metrics1.memoryBytesSpilled == metrics2.memoryBytesSpilled) + assert(metrics1.diskBytesSpilled == metrics2.diskBytesSpilled) + assertOptionEquals( + metrics1.shuffleReadMetrics, metrics2.shuffleReadMetrics, assertShuffleReadEquals) + assertOptionEquals( + metrics1.shuffleWriteMetrics, metrics2.shuffleWriteMetrics, assertShuffleWriteEquals) + assertOptionEquals(metrics1.updatedBlocks, metrics2.updatedBlocks, assertBlocksEquals) + } + + private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { + assert(metrics1.shuffleFinishTime == metrics2.shuffleFinishTime) + assert(metrics1.totalBlocksFetched == metrics2.totalBlocksFetched) + assert(metrics1.remoteBlocksFetched == metrics2.remoteBlocksFetched) + assert(metrics1.localBlocksFetched == metrics2.localBlocksFetched) + assert(metrics1.fetchWaitTime == metrics2.fetchWaitTime) + assert(metrics1.remoteBytesRead == metrics2.remoteBytesRead) + } + + private def assertEquals(metrics1: ShuffleWriteMetrics, metrics2: ShuffleWriteMetrics) { + assert(metrics1.shuffleBytesWritten == metrics2.shuffleBytesWritten) + assert(metrics1.shuffleWriteTime == metrics2.shuffleWriteTime) + } + + private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { + assert(bm1.executorId == bm2.executorId) + assert(bm1.host == bm2.host) + assert(bm1.port == bm2.port) + assert(bm1.nettyPort == bm2.nettyPort) + } + + private def assertEquals(result1: JobResult, result2: JobResult) { + (result1, result2) match { + case (JobSucceeded, JobSucceeded) => + case (r1: JobFailed, r2: JobFailed) => + assert(r1.failedStageId == r2.failedStageId) + assertEquals(r1.exception, r2.exception) + case _ => fail("Job results don't match in types!") + } + } + + private def assertEquals(reason1: TaskEndReason, reason2: TaskEndReason) { + (reason1, reason2) match { + case (Success, Success) => + case (Resubmitted, Resubmitted) => + case (r1: FetchFailed, r2: FetchFailed) => + assert(r1.shuffleId == r2.shuffleId) + assert(r1.mapId == r2.mapId) + assert(r1.reduceId == r2.reduceId) + assertEquals(r1.bmAddress, r2.bmAddress) + case (r1: ExceptionFailure, r2: ExceptionFailure) => + assert(r1.className == r2.className) + assert(r1.description == r2.description) + assertSeqEquals(r1.stackTrace, r2.stackTrace, assertStackTraceElementEquals) + assertOptionEquals(r1.metrics, r2.metrics, assertTaskMetricsEquals) + case (TaskResultLost, TaskResultLost) => + case (TaskKilled, TaskKilled) => + case (ExecutorLostFailure, ExecutorLostFailure) => + case (UnknownReason, UnknownReason) => + case _ => fail("Task end reasons don't match in types!") + } + } + + private def assertEquals( + details1: Map[String, Seq[(String, String)]], + details2: Map[String, Seq[(String, String)]]) { + details1.zip(details2).foreach { + case ((key1, values1: Seq[(String, String)]), (key2, values2: Seq[(String, String)])) => + assert(key1 == key2) + values1.zip(values2).foreach { case (v1, v2) => assert(v1 == v2) } + } + } + + private def assertEquals(exception1: Exception, exception2: Exception) { + assert(exception1.getMessage == exception2.getMessage) + assertSeqEquals( + exception1.getStackTrace, + exception2.getStackTrace, + assertStackTraceElementEquals) + } + + private def assertJsonStringEquals(json1: String, json2: String) { + val formatJsonString = (json: String) => json.replaceAll("[\\s|]", "") + formatJsonString(json1) == formatJsonString(json2) + } + + private def assertSeqEquals[T](seq1: Seq[T], seq2: Seq[T], assertEquals: (T, T) => Unit) { + assert(seq1.length == seq2.length) + seq1.zip(seq2).foreach { case (t1, t2) => + assertEquals(t1, t2) + } + } + + private def assertOptionEquals[T]( + opt1: Option[T], + opt2: Option[T], + assertEquals: (T, T) => Unit) { + if (opt1.isDefined) { + assert(opt2.isDefined) + assertEquals(opt1.get, opt2.get) + } else { + assert(!opt2.isDefined) + } + } + + /** + * Use different names for methods we pass in to assertSeqEquals or assertOptionEquals + */ + + private def assertShuffleReadEquals(r1: ShuffleReadMetrics, r2: ShuffleReadMetrics) { + assertEquals(r1, r2) + } + + private def assertShuffleWriteEquals(w1: ShuffleWriteMetrics, w2: ShuffleWriteMetrics) { + assertEquals(w1, w2) + } + + private def assertTaskMetricsEquals(t1: TaskMetrics, t2: TaskMetrics) { + assertEquals(t1, t2) + } + + private def assertBlocksEquals( + blocks1: Seq[(BlockId, BlockStatus)], + blocks2: Seq[(BlockId, BlockStatus)]) = { + assertSeqEquals(blocks1, blocks2, assertBlockEquals) + } + + private def assertBlockEquals(b1: (BlockId, BlockStatus), b2: (BlockId, BlockStatus)) { + assert(b1 == b2) + } + + private def assertStackTraceElementEquals(ste1: StackTraceElement, ste2: StackTraceElement) { + assert(ste1 == ste2) + } + + + /** ----------------------------------- * + | Util methods for constructing events | + * ------------------------------------ */ + + private val properties = { + val p = new Properties + p.setProperty("Ukraine", "Kiev") + p.setProperty("Russia", "Moscow") + p.setProperty("France", "Paris") + p.setProperty("Germany", "Berlin") + p + } + + private val stackTrace = { + Array[StackTraceElement]( + new StackTraceElement("Apollo", "Venus", "Mercury", 42), + new StackTraceElement("Afollo", "Vemus", "Mercurry", 420), + new StackTraceElement("Ayollo", "Vesus", "Blackberry", 4200) + ) + } + + private def makeRddInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { + val r = new RDDInfo(a, "mayor", b, StorageLevel.MEMORY_AND_DISK) + r.numCachedPartitions = c + r.memSize = d + r.diskSize = e + r + } + + private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { + new StageInfo(a, "greetings", b, makeRddInfo(a, b, c, d, e)) + } + + private def makeTaskInfo(a: Long, b: Int, c: Long) = { + new TaskInfo(a, b, c, "executor", "your kind sir", TaskLocality.NODE_LOCAL) + } + + private def makeTaskMetrics(a: Long, b: Long, c: Long, d: Long, e: Int, f: Int) = { + val t = new TaskMetrics + val sr = new ShuffleReadMetrics + val sw = new ShuffleWriteMetrics + t.hostname = "localhost" + t.executorDeserializeTime = a + t.executorRunTime = b + t.resultSize = c + t.jvmGCTime = d + t.resultSerializationTime = a + b + t.memoryBytesSpilled = a + c + sr.shuffleFinishTime = b + c + sr.totalBlocksFetched = e + f + sr.remoteBytesRead = b + d + sr.localBlocksFetched = e + sr.fetchWaitTime = a + d + sr.remoteBlocksFetched = f + sw.shuffleBytesWritten = a + b + c + sw.shuffleWriteTime = b + c + d + t.shuffleReadMetrics = Some(sr) + t.shuffleWriteMetrics = Some(sw) + // Make at most 6 blocks + t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i => + (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i)) + }.toSeq) + t + } + + + /** --------------------------------------- * + | JSON string representation of each event | + * ---------------------------------------- */ + + private val stageSubmittedJsonString = + """ + {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": + "greetings","Number of Tasks":200,"RDD Info":{"RDD ID":100,"Name":"mayor","Storage + Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, + "Number of Partitions":200,"Number of Cached Partitions":300,"Memory Size":400, + "Disk Size":500},"Emitted Task Size Warning":false},"Properties":{"France":"Paris", + "Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + """ + + private val stageCompletedJsonString = + """ + {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name": + "greetings","Number of Tasks":201,"RDD Info":{"RDD ID":101,"Name":"mayor","Storage + Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, + "Number of Partitions":201,"Number of Cached Partitions":301,"Memory Size":401, + "Disk Size":501},"Emitted Task Size Warning":false}} + """ + + private val taskStartJsonString = + """ + {"Event":"SparkListenerTaskStart","Stage ID":111,"Task Info":{"Task ID":222, + "Index":333,"Launch Time":444,"Executor ID":"executor","Host":"your kind sir", + "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed":false, + "Serialized Size":0}} + """ + + private val taskGettingResultJsonString = + """ + {"Event":"SparkListenerTaskGettingResult","Task Info":{"Task ID":1000,"Index": + 2000,"Launch Time":3000,"Executor ID":"executor","Host":"your kind sir", + "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed":false, + "Serialized Size":0}} + """ + + private val taskEndJsonString = + """ + {"Event":"SparkListenerTaskEnd","Stage ID":1,"Task Type":"ShuffleMapTask", + "Task End Reason":{"Reason":"Success"},"Task Info":{"Task ID":123,"Index": + 234,"Launch Time":345,"Executor ID":"executor","Host":"your kind sir", + "Locality":"NODE_LOCAL","Getting Result Time":0,"Finish Time":0,"Failed": + false,"Serialized Size":0},"Task Metrics":{"Host Name":"localhost", + "Executor Deserialize Time":300,"Executor Run Time":400,"Result Size":500, + "JVM GC Time":600,"Result Serialization Time":700,"Memory Bytes Spilled": + 800,"Disk Bytes Spilled":0,"Shuffle Read Metrics":{"Shuffle Finish Time": + 900,"Total Blocks Fetched":1500,"Remote Blocks Fetched":800,"Local Blocks Fetched": + 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": + {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": + [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":false, + "Replication":2},"Memory Size":0,"Disk Size":0}}]}} + """ + + private val jobStartJsonString = + """ + {"Event":"SparkListenerJobStart","Job ID":10,"Stage IDs":[1,2,3,4],"Properties": + {"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + """ + + private val jobEndJsonString = + """ + {"Event":"SparkListenerJobEnd","Job ID":20,"Job Result":{"Result":"JobSucceeded"}} + """ + + private val environmentUpdateJsonString = + """ + {"Event":"SparkListenerEnvironmentUpdate","JVM Information":{"GC speed":"9999 objects/s", + "Java home":"Land of coffee"},"Spark Properties":{"Job throughput":"80000 jobs/s, + regardless of job type"},"System Properties":{"Username":"guest","Password":"guest"}, + "Classpath Entries":{"Super library":"/tmp/super_library"}} + """ + + private val blockManagerAddedJsonString = + """ + {"Event":"SparkListenerBlockManagerAdded","Block Manager ID":{"Executor ID":"Stars", + "Host":"In your multitude...","Port":300,"Netty Port":400},"Maximum Memory":500} + """ + + private val blockManagerRemovedJsonString = + """ + {"Event":"SparkListenerBlockManagerRemoved","Block Manager ID":{"Executor ID":"Scarce", + "Host":"to be counted...","Port":100,"Netty Port":200}} + """ + + private val unpersistRDDJsonString = + """ + {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} + """ + + } diff --git a/docs/configuration.md b/docs/configuration.md index a006224d5080c..16ee5ec0f230f 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -490,7 +490,30 @@ Apart from these, the following properties are also available, and may be useful + + + + + + + + + + + + + + + From 67fa71cba2cc07a65478899592e6ebad000e24c5 Mon Sep 17 00:00:00 2001 From: Jyotiska NK Date: Wed, 19 Mar 2014 14:04:45 -0700 Subject: [PATCH 110/397] Added doctest for map function in rdd.py Doctest added for map in rdd.py Author: Jyotiska NK Closes #177 from jyotiska/pyspark_rdd_map_doctest and squashes the following commits: a38527f [Jyotiska NK] Added doctest for map function in rdd.py --- python/pyspark/rdd.py | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index ae09dbff02a36..ca2dc119a461a 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -180,6 +180,10 @@ def getCheckpointFile(self): def map(self, f, preservesPartitioning=False): """ Return a new RDD by applying a function to each element of this RDD. + + >>> rdd = sc.parallelize(["b", "a", "c"]) + >>> sorted(rdd.map(lambda x: (x, 1)).collect()) + [('a', 1), ('b', 1), ('c', 1)] """ def func(split, iterator): return imap(f, iterator) return PipelinedRDD(self, func, preservesPartitioning) From 16789317a34c1974f7b35960f06a7b51d8e0f29f Mon Sep 17 00:00:00 2001 From: qqsun8819 Date: Wed, 19 Mar 2014 16:33:54 -0700 Subject: [PATCH 111/397] SPARK-1099:Spark's local mode should probably respect spark.cores.max by default This is for JIRA:https://spark-project.atlassian.net/browse/SPARK-1099 And this is what I do in this patch (also commented in the JIRA) @aarondav This is really a behavioral change, so I do this with great caution, and welcome any review advice: 1 I change the "MASTER=local" pattern of create LocalBackEnd . In the past, we passed 1 core to it . now it use a default cores The reason here is that when someone use spark-shell to start local mode , Repl will use this "MASTER=local" pattern as default. So if one also specify cores in the spark-shell command line, it will all go in here. So here pass 1 core is not suitalbe reponding to our change here. 2 In the LocalBackEnd , the "totalCores" variable are fetched following a different rule(in the past it just take in a userd passed cores, like 1 in "MASTER=local" pattern, 2 in "MASTER=local[2]" pattern" rules: a The second argument of LocalBackEnd 's constructor indicating cores have a default value which is Int.MaxValue. If user didn't pass it , its first default value is Int.MaxValue b In getMaxCores, we first compare the former value to Int.MaxValue. if it's not equal, we think that user has passed their desired value, so just use it c. If b is not satified, we then get cores from spark.cores.max, and we get real logical cores from Runtime. And if cores specified by spark.cores.max is bigger than logical cores, we use logical cores, otherwise we use spark.cores.max 3 In SparkContextSchedulerCreationSuite 's test("local") case, assertion is modified from 1 to logical cores, because "MASTER=local" pattern use default vaules. Author: qqsun8819 Closes #110 from qqsun8819/local-cores and squashes the following commits: 731aefa [qqsun8819] 1 LocalBackend not change 2 In SparkContext do some process to the cores and pass it to original LocalBackend constructor 78b9c60 [qqsun8819] 1 SparkContext MASTER=local pattern use default cores instead of 1 to construct LocalBackEnd , for use of spark-shell and cores specified in cmd line 2 some test case change from local to local[1]. 3 SparkContextSchedulerCreationSuite test spark.cores.max config in local pattern 6ae1ee8 [qqsun8819] Add a static function in LocalBackEnd to let it use spark.cores.max specified cores when no cores are passed to it --- .../scala/org/apache/spark/SparkContext.scala | 5 ++++- .../scala/org/apache/spark/FileSuite.scala | 4 ++-- .../SparkContextSchedulerCreationSuite.scala | 19 ++++++++++++++++--- 3 files changed, 22 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a1003b7925715..8f746072782bc 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1262,7 +1262,10 @@ object SparkContext extends Logging { master match { case "local" => val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, 1) + // Use user specified in config, up to all available cores + val realCores = Runtime.getRuntime.availableProcessors() + val toUseCores = math.min(sc.conf.getInt("spark.cores.max", realCores), realCores) + val backend = new LocalBackend(scheduler, toUseCores) scheduler.initialize(backend) scheduler diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 01af94077144a..b4a5881cd97ad 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.SparkContext._ class FileSuite extends FunSuite with LocalSparkContext { test("text files") { - sc = new SparkContext("local", "test") + sc = new SparkContext("local[1]", "test") val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) @@ -176,7 +176,7 @@ class FileSuite extends FunSuite with LocalSparkContext { test("write SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat - sc = new SparkContext("local", "test") + sc = new SparkContext("local[1]", "test") val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index b543471a5d35b..9dd42be1d7c38 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -27,10 +27,10 @@ import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite extends FunSuite with PrivateMethodTester with LocalSparkContext with Logging { - def createTaskScheduler(master: String): TaskSchedulerImpl = { + def createTaskScheduler(master: String, conf: SparkConf = new SparkConf()): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. - sc = new SparkContext("local", "test") + sc = new SparkContext("local", "test", conf) val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] @@ -44,13 +44,26 @@ class SparkContextSchedulerCreationSuite } test("local") { - val sched = createTaskScheduler("local") + var conf = new SparkConf() + conf.set("spark.cores.max", "1") + val sched = createTaskScheduler("local", conf) sched.backend match { case s: LocalBackend => assert(s.totalCores === 1) case _ => fail() } } + test("local-cores-exceed") { + val cores = Runtime.getRuntime.availableProcessors() + 1 + var conf = new SparkConf() + conf.set("spark.cores.max", cores.toString) + val sched = createTaskScheduler("local", conf) + sched.backend match { + case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } + } + test("local-n") { val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) From ffe272d97c22955fe7744b1c0132cd9877b6df96 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 19 Mar 2014 17:56:48 -0700 Subject: [PATCH 112/397] Revert "SPARK-1099:Spark's local mode should probably respect spark.cores.max by default" This reverts commit 16789317a34c1974f7b35960f06a7b51d8e0f29f. Jenkins was not run for this PR. --- .../scala/org/apache/spark/SparkContext.scala | 5 +---- .../scala/org/apache/spark/FileSuite.scala | 4 ++-- .../SparkContextSchedulerCreationSuite.scala | 19 +++---------------- 3 files changed, 6 insertions(+), 22 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8f746072782bc..a1003b7925715 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1262,10 +1262,7 @@ object SparkContext extends Logging { master match { case "local" => val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - // Use user specified in config, up to all available cores - val realCores = Runtime.getRuntime.availableProcessors() - val toUseCores = math.min(sc.conf.getInt("spark.cores.max", realCores), realCores) - val backend = new LocalBackend(scheduler, toUseCores) + val backend = new LocalBackend(scheduler, 1) scheduler.initialize(backend) scheduler diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index b4a5881cd97ad..01af94077144a 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -34,7 +34,7 @@ import org.apache.spark.SparkContext._ class FileSuite extends FunSuite with LocalSparkContext { test("text files") { - sc = new SparkContext("local[1]", "test") + sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 4) @@ -176,7 +176,7 @@ class FileSuite extends FunSuite with LocalSparkContext { test("write SequenceFile using new Hadoop API") { import org.apache.hadoop.mapreduce.lib.output.SequenceFileOutputFormat - sc = new SparkContext("local[1]", "test") + sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), new Text("a" * x))) diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index 9dd42be1d7c38..b543471a5d35b 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -27,10 +27,10 @@ import org.apache.spark.scheduler.local.LocalBackend class SparkContextSchedulerCreationSuite extends FunSuite with PrivateMethodTester with LocalSparkContext with Logging { - def createTaskScheduler(master: String, conf: SparkConf = new SparkConf()): TaskSchedulerImpl = { + def createTaskScheduler(master: String): TaskSchedulerImpl = { // Create local SparkContext to setup a SparkEnv. We don't actually want to start() the // real schedulers, so we don't want to create a full SparkContext with the desired scheduler. - sc = new SparkContext("local", "test", conf) + sc = new SparkContext("local", "test") val createTaskSchedulerMethod = PrivateMethod[TaskScheduler]('createTaskScheduler) val sched = SparkContext invokePrivate createTaskSchedulerMethod(sc, master) sched.asInstanceOf[TaskSchedulerImpl] @@ -44,26 +44,13 @@ class SparkContextSchedulerCreationSuite } test("local") { - var conf = new SparkConf() - conf.set("spark.cores.max", "1") - val sched = createTaskScheduler("local", conf) + val sched = createTaskScheduler("local") sched.backend match { case s: LocalBackend => assert(s.totalCores === 1) case _ => fail() } } - test("local-cores-exceed") { - val cores = Runtime.getRuntime.availableProcessors() + 1 - var conf = new SparkConf() - conf.set("spark.cores.max", cores.toString) - val sched = createTaskScheduler("local", conf) - sched.backend match { - case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) - case _ => fail() - } - } - test("local-n") { val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) From 66a03e5fe0167f590d150e099b15902e826a188f Mon Sep 17 00:00:00 2001 From: Reza Zadeh Date: Thu, 20 Mar 2014 10:39:20 -0700 Subject: [PATCH 113/397] Principal Component Analysis # Principal Component Analysis Computes the top k principal component coefficients for the m-by-n data matrix X. Rows of X correspond to observations and columns correspond to variables. The coefficient matrix is n-by-k. Each column of the coefficients return matrix contains coefficients for one principal component, and the columns are in descending order of component variance. This function centers the data and uses the singular value decomposition (SVD) algorithm. ## Testing Tests included: * All principal components * Only top k principal components * Dense SVD tests * Dense/sparse matrix tests The results are tested against MATLAB's pca: http://www.mathworks.com/help/stats/pca.html ## Documentation Added to mllib-guide.md ## Example Usage Added to examples directory under SparkPCA.scala Author: Reza Zadeh Closes #88 from rezazadeh/sparkpca and squashes the following commits: e298700 [Reza Zadeh] reformat using IDE 3f23271 [Reza Zadeh] documentation and cleanup b025ab2 [Reza Zadeh] documentation e2667d4 [Reza Zadeh] assertMatrixApproximatelyEquals 3787bb4 [Reza Zadeh] stylin c6ecc1f [Reza Zadeh] docs aa2bbcb [Reza Zadeh] rename sparseToTallSkinnyDense 56975b0 [Reza Zadeh] docs 2df9bde [Reza Zadeh] docs update 8fb0015 [Reza Zadeh] rcond documentation dbf7797 [Reza Zadeh] correct argument number a9f1f62 [Reza Zadeh] documentation 4ce6caa [Reza Zadeh] style changes 9a56a02 [Reza Zadeh] use rcond relative to larget svalue 120f796 [Reza Zadeh] housekeeping 156ff78 [Reza Zadeh] string comprehension 2e1cf43 [Reza Zadeh] rename rcond ea223a6 [Reza Zadeh] many style changes f4002d7 [Reza Zadeh] more docs bd53c7a [Reza Zadeh] proper accumulator a8b5ecf [Reza Zadeh] Don't use for loops 0dc7980 [Reza Zadeh] filter zeros in sparse 6115610 [Reza Zadeh] More documentation 36d51e8 [Reza Zadeh] use JBLAS for UVS^-1 computation bc4599f [Reza Zadeh] configurable rcond 86f7515 [Reza Zadeh] compute per parition, use while 09726b3 [Reza Zadeh] more style changes 4195e69 [Reza Zadeh] private, accumulator 17002be [Reza Zadeh] style changes 4ba7471 [Reza Zadeh] style change f4982e6 [Reza Zadeh] Use dense matrix in example 2828d28 [Reza Zadeh] optimizations: normalize once, use inplace ops 72c9fa1 [Reza Zadeh] rename DenseMatrix to TallSkinnyDenseMatrix, lean f807be9 [Reza Zadeh] fix typo 2d7ccde [Reza Zadeh] Array interface for dense svd and pca cd290fa [Reza Zadeh] provide RDD[Array[Double]] support 398d123 [Reza Zadeh] style change 55abbfa [Reza Zadeh] docs fix ef29644 [Reza Zadeh] bad chnage undo 472566e [Reza Zadeh] all files from old pr 555168f [Reza Zadeh] initial files --- docs/mllib-guide.md | 1 + docs/mllib-linear-algebra.md | 13 + .../spark/examples/mllib/SparkPCA.scala | 51 +++ .../spark/examples/mllib/SparkSVD.scala | 4 +- .../apache/spark/mllib/linalg/MatrixRow.scala | 26 ++ .../org/apache/spark/mllib/linalg/PCA.scala | 120 ++++++ .../org/apache/spark/mllib/linalg/SVD.scala | 370 ++++++++++++++---- .../mllib/linalg/TallSkinnyDenseMatrix.scala | 30 ++ .../mllib/linalg/TallSkinnyMatrixSVD.scala | 31 ++ .../org/apache/spark/mllib/util/LAUtils.scala | 65 +++ .../apache/spark/mllib/linalg/PCASuite.scala | 124 ++++++ .../apache/spark/mllib/linalg/SVDSuite.scala | 98 +++-- 12 files changed, 819 insertions(+), 114 deletions(-) create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 76308ec9c0821..203d235bf9663 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -29,6 +29,7 @@ The following links provide a detailed explanation of the methods and usage exam * Gradient Descent and Stochastic Gradient Descent * Linear Algebra * Singular Value Decomposition + * Principal Component Analysis # Dependencies MLlib uses the [jblas](https://github.com/mikiobraun/jblas) linear algebra library, which itself diff --git a/docs/mllib-linear-algebra.md b/docs/mllib-linear-algebra.md index cc203d833d344..09598be7903ac 100644 --- a/docs/mllib-linear-algebra.md +++ b/docs/mllib-linear-algebra.md @@ -59,3 +59,16 @@ val = decomposed.S.data println("singular values = " + s.toArray.mkString) {% endhighlight %} + + +# Principal Component Analysis + +Computes the top k principal component coefficients for the m-by-n data matrix X. +Rows of X correspond to observations and columns correspond to variables. +The coefficient matrix is n-by-k. Each column of the return matrix contains coefficients +for one principal component, and the columns are in descending +order of component variance. This function centers the data and uses the +singular value decomposition (SVD) algorithm. + +All input and output is expected in DenseMatrix matrix format. See the examples directory +under "SparkPCA.scala" for example usage. diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala new file mode 100644 index 0000000000000..d4e08c5e12d81 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala @@ -0,0 +1,51 @@ +/* + * 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.examples.mllib + +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg.PCA +import org.apache.spark.mllib.linalg.MatrixEntry +import org.apache.spark.mllib.linalg.SparseMatrix +import org.apache.spark.mllib.util._ + + +/** + * Compute PCA of an example matrix. + */ +object SparkPCA { + def main(args: Array[String]) { + if (args.length != 3) { + System.err.println("Usage: SparkPCA m n") + System.exit(1) + } + val sc = new SparkContext(args(0), "PCA", + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + + val m = args(2).toInt + val n = args(3).toInt + + // Make example matrix + val data = Array.tabulate(m, n) { (a, b) => + (a + 2).toDouble * (b + 1) / (1 + a + b) } + + // recover top principal component + val coeffs = new PCA().setK(1).compute(sc.makeRDD(data)) + + println("top principal component = " + coeffs.mkString(", ")) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala index ce2b133368e85..2933cec497b37 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala @@ -38,7 +38,7 @@ object SparkSVD { System.exit(1) } val sc = new SparkContext(args(0), "SVD", - System.getenv("SPARK_HOME"), Seq(System.getenv("SPARK_EXAMPLES_JAR"))) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) // Load and parse the data file val data = sc.textFile(args(1)).map { line => @@ -49,7 +49,7 @@ object SparkSVD { val n = args(3).toInt // recover largest singular vector - val decomposed = SVD.sparseSVD(SparseMatrix(data, m, n), 1) + val decomposed = new SVD().setK(1).compute(SparseMatrix(data, m, n)) val u = decomposed.U.data val s = decomposed.S.data val v = decomposed.V.data diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala new file mode 100644 index 0000000000000..2608a67bfe260 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.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.mllib.linalg + +/** + * Class that represents a row of a dense matrix + * + * @param i row index (0 indexing used) + * @param data entries of the row + */ +case class MatrixRow(val i: Int, val data: Array[Double]) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala new file mode 100644 index 0000000000000..fe5b3f6c7e463 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala @@ -0,0 +1,120 @@ +/* + * 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.mllib.linalg + +import org.apache.spark.rdd.RDD + + +import org.jblas.DoubleMatrix + + +/** + * Class used to obtain principal components + */ +class PCA { + private var k = 1 + + /** + * Set the number of top-k principle components to return + */ + def setK(k: Int): PCA = { + this.k = k + this + } + + /** + * Compute PCA using the current set parameters + */ + def compute(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { + computePCA(matrix) + } + + /** + * Compute PCA using the parameters currently set + * See computePCA() for more details + */ + def compute(matrix: RDD[Array[Double]]): Array[Array[Double]] = { + computePCA(matrix) + } + + /** + * Computes the top k principal component coefficients for the m-by-n data matrix X. + * Rows of X correspond to observations and columns correspond to variables. + * The coefficient matrix is n-by-k. Each column of coeff contains coefficients + * for one principal component, and the columns are in descending + * order of component variance. + * This function centers the data and uses the + * singular value decomposition (SVD) algorithm. + * + * @param matrix dense matrix to perform PCA on + * @return An nxk matrix with principal components in columns. Columns are inner arrays + */ + private def computePCA(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { + val m = matrix.m + val n = matrix.n + + if (m <= 0 || n <= 0) { + throw new IllegalArgumentException("Expecting a well-formed matrix: m=$m n=$n") + } + + computePCA(matrix.rows.map(_.data)) + } + + /** + * Computes the top k principal component coefficients for the m-by-n data matrix X. + * Rows of X correspond to observations and columns correspond to variables. + * The coefficient matrix is n-by-k. Each column of coeff contains coefficients + * for one principal component, and the columns are in descending + * order of component variance. + * This function centers the data and uses the + * singular value decomposition (SVD) algorithm. + * + * @param matrix dense matrix to perform pca on + * @return An nxk matrix of principal components + */ + private def computePCA(matrix: RDD[Array[Double]]): Array[Array[Double]] = { + val n = matrix.first.size + + // compute column sums and normalize matrix + val colSumsTemp = matrix.map((_, 1)).fold((Array.ofDim[Double](n), 0)) { + (a, b) => + val am = new DoubleMatrix(a._1) + val bm = new DoubleMatrix(b._1) + am.addi(bm) + (a._1, a._2 + b._2) + } + + val m = colSumsTemp._2 + val colSums = colSumsTemp._1.map(x => x / m) + + val data = matrix.map { + x => + val row = Array.ofDim[Double](n) + var i = 0 + while (i < n) { + row(i) = x(i) - colSums(i) + i += 1 + } + row + } + + val (u, s, v) = new SVD().setK(k).compute(data) + v + } +} + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index e4a26eeb07c60..3e7cc648d1d37 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -23,12 +23,16 @@ import org.apache.spark.rdd.RDD import org.jblas.{DoubleMatrix, Singular, MatrixFunctions} - /** * Class used to obtain singular value decompositions */ class SVD { - private var k: Int = 1 + private var k = 1 + private var computeU = true + + // All singular values smaller than rCond * sigma(0) + // are treated as zero, where sigma(0) is the largest singular value. + private var rCond = 1e-9 /** * Set the number of top-k singular vectors to return @@ -38,54 +42,235 @@ class SVD { this } - /** + /** + * Sets the reciprocal condition number (rCond). All singular values + * smaller than rCond * sigma(0) are treated as zero, + * where sigma(0) is the largest singular value. + */ + def setReciprocalConditionNumber(smallS: Double): SVD = { + this.rCond = smallS + this + } + + /** + * Should U be computed? + */ + def setComputeU(compU: Boolean): SVD = { + this.computeU = compU + this + } + + /** * Compute SVD using the current set parameters */ - def compute(matrix: SparseMatrix) : MatrixSVD = { - SVD.sparseSVD(matrix, k) + def compute(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { + denseSVD(matrix) } -} + /** + * Compute SVD using the current set parameters + * Returns (U, S, V) such that A = USV^T + * U is a row-by-row dense matrix + * S is a simple double array of singular values + * V is a 2d array matrix + * See [[denseSVD]] for more documentation + */ + def compute(matrix: RDD[Array[Double]]): + (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { + denseSVD(matrix) + } -/** - * Top-level methods for calling Singular Value Decomposition - * NOTE: All matrices are in 0-indexed sparse format RDD[((int, int), value)] - */ -object SVD { -/** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require n^2 doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * All input and output is expected in sparse matrix format, 0-indexed - * as tuples of the form ((i,j),value) all in RDDs using the - * SparseMatrix class - * - * @param matrix sparse matrix to factorize - * @param k Recover k singular values and vectors - * @return Three sparse matrices: U, S, V such that A = USV^T - */ - def sparseSVD( - matrix: SparseMatrix, - k: Int) - : MatrixSVD = - { + /** + * See full paramter definition of sparseSVD for more description. + * + * @param matrix sparse matrix to factorize + * @return Three sparse matrices: U, S, V such that A = USV^T + */ + def compute(matrix: SparseMatrix): MatrixSVD = { + sparseSVD(matrix) + } + + /** + * Singular Value Decomposition for Tall and Skinny matrices. + * Given an m x n matrix A, this will compute matrices U, S, V such that + * A = U * S * V' + * + * There is no restriction on m, but we require n^2 doubles to fit in memory. + * Further, n should be less than m. + * + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), + * from which we recover S and V. + * Then we compute U via easy matrix multiplication + * as U = A * V * S^-1 + * + * Only the k largest singular values and associated vectors are found. + * If there are k such values, then the dimensions of the return will be: + * + * S is k x k and diagonal, holding the singular values on diagonal + * U is m x k and satisfies U'U = eye(k) + * V is n x k and satisfies V'V = eye(k) + * + * @param matrix dense matrix to factorize + * @return See [[TallSkinnyMatrixSVD]] for the output matrices and arrays + */ + private def denseSVD(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { + val m = matrix.m + val n = matrix.n + + if (m < n || m <= 0 || n <= 0) { + throw new IllegalArgumentException("Expecting a tall and skinny matrix m=$m n=$n") + } + + if (k < 1 || k > n) { + throw new IllegalArgumentException("Request up to n singular values n=$n k=$k") + } + + val rowIndices = matrix.rows.map(_.i) + + // compute SVD + val (u, sigma, v) = denseSVD(matrix.rows.map(_.data)) + + if (computeU) { + // prep u for returning + val retU = TallSkinnyDenseMatrix( + u.zip(rowIndices).map { + case (row, i) => MatrixRow(i, row) + }, + m, + k) + + TallSkinnyMatrixSVD(retU, sigma, v) + } else { + TallSkinnyMatrixSVD(null, sigma, v) + } + } + + /** + * Singular Value Decomposition for Tall and Skinny matrices. + * Given an m x n matrix A, this will compute matrices U, S, V such that + * A = U * S * V' + * + * There is no restriction on m, but we require n^2 doubles to fit in memory. + * Further, n should be less than m. + * + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), + * from which we recover S and V. + * Then we compute U via easy matrix multiplication + * as U = A * V * S^-1 + * + * Only the k largest singular values and associated vectors are found. + * If there are k such values, then the dimensions of the return will be: + * + * S is k x k and diagonal, holding the singular values on diagonal + * U is m x k and satisfies U'U = eye(k) + * V is n x k and satisfies V'V = eye(k) + * + * The return values are as lean as possible: an RDD of rows for U, + * a simple array for sigma, and a dense 2d matrix array for V + * + * @param matrix dense matrix to factorize + * @return Three matrices: U, S, V such that A = USV^T + */ + private def denseSVD(matrix: RDD[Array[Double]]): + (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { + val n = matrix.first.size + + if (k < 1 || k > n) { + throw new IllegalArgumentException( + "Request up to n singular values k=$k n=$n") + } + + // Compute A^T A + val fullata = matrix.mapPartitions { + iter => + val localATA = Array.ofDim[Double](n, n) + while (iter.hasNext) { + val row = iter.next() + var i = 0 + while (i < n) { + var j = 0 + while (j < n) { + localATA(i)(j) += row(i) * row(j) + j += 1 + } + i += 1 + } + } + Iterator(localATA) + }.fold(Array.ofDim[Double](n, n)) { + (a, b) => + var i = 0 + while (i < n) { + var j = 0 + while (j < n) { + a(i)(j) += b(i)(j) + j += 1 + } + i += 1 + } + a + } + + // Construct jblas A^T A locally + val ata = new DoubleMatrix(fullata) + + // Since A^T A is small, we can compute its SVD directly + val svd = Singular.sparseSVD(ata) + val V = svd(0) + val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x / svd(1).get(0) > rCond) + + val sk = Math.min(k, sigmas.size) + val sigma = sigmas.take(sk) + + // prepare V for returning + val retV = Array.tabulate(n, sk)((i, j) => V.get(i, j)) + + if (computeU) { + // Compute U as U = A V S^-1 + // Compute VS^-1 + val vsinv = new DoubleMatrix(Array.tabulate(n, sk)((i, j) => V.get(i, j) / sigma(j))) + val retU = matrix.map { + x => + val v = new DoubleMatrix(Array(x)) + v.mmul(vsinv).data + } + (retU, sigma, retV) + } else { + (null, sigma, retV) + } + } + + /** + * Singular Value Decomposition for Tall and Skinny sparse matrices. + * Given an m x n matrix A, this will compute matrices U, S, V such that + * A = U * S * V' + * + * There is no restriction on m, but we require O(n^2) doubles to fit in memory. + * Further, n should be less than m. + * + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), + * from which we recover S and V. + * Then we compute U via easy matrix multiplication + * as U = A * V * S^-1 + * + * Only the k largest singular values and associated vectors are found. + * If there are k such values, then the dimensions of the return will be: + * + * S is k x k and diagonal, holding the singular values on diagonal + * U is m x k and satisfies U'U = eye(k) + * V is n x k and satisfies V'V = eye(k) + * + * All input and output is expected in sparse matrix format, 0-indexed + * as tuples of the form ((i,j),value) all in RDDs using the + * SparseMatrix class + * + * @param matrix sparse matrix to factorize + * @return Three sparse matrices: U, S, V such that A = USV^T + */ + private def sparseSVD(matrix: SparseMatrix): MatrixSVD = { val data = matrix.data val m = matrix.m val n = matrix.n @@ -100,11 +285,16 @@ object SVD { // Compute A^T A, assuming rows are sparse enough to fit in memory val rows = data.map(entry => - (entry.i, (entry.j, entry.mval))).groupByKey() - val emits = rows.flatMap{ case (rowind, cols) => - cols.flatMap{ case (colind1, mval1) => - cols.map{ case (colind2, mval2) => - ((colind1, colind2), mval1*mval2) } } + (entry.i, (entry.j, entry.mval))).groupByKey() + val emits = rows.flatMap { + case (rowind, cols) => + cols.flatMap { + case (colind1, mval1) => + cols.map { + case (colind2, mval2) => + ((colind1, colind2), mval1 * mval2) + } + } }.reduceByKey(_ + _) // Construct jblas A^T A locally @@ -116,11 +306,12 @@ object SVD { // Since A^T A is small, we can compute its SVD directly val svd = Singular.sparseSVD(ata) val V = svd(0) + // This will be updated to rcond val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x > 1e-9) if (sigmas.size < k) { - throw new Exception("Not enough singular values to return") - } + throw new Exception("Not enough singular values to return k=" + k + " s=" + sigmas.size) + } val sigma = sigmas.take(k) @@ -128,56 +319,73 @@ object SVD { // prepare V for returning val retVdata = sc.makeRDD( - Array.tabulate(V.rows, sigma.length){ (i,j) => - MatrixEntry(i, j, V.get(i,j)) }.flatten) + Array.tabulate(V.rows, sigma.length) { + (i, j) => + MatrixEntry(i, j, V.get(i, j)) + }.flatten) val retV = SparseMatrix(retVdata, V.rows, sigma.length) - - val retSdata = sc.makeRDD(Array.tabulate(sigma.length){ - x => MatrixEntry(x, x, sigma(x))}) + + val retSdata = sc.makeRDD(Array.tabulate(sigma.length) { + x => MatrixEntry(x, x, sigma(x)) + }) val retS = SparseMatrix(retSdata, sigma.length, sigma.length) // Compute U as U = A V S^-1 // turn V S^-1 into an RDD as a sparse matrix - val vsirdd = sc.makeRDD(Array.tabulate(V.rows, sigma.length) - { (i,j) => ((i, j), V.get(i,j) / sigma(j)) }.flatten) - - // Multiply A by VS^-1 - val aCols = data.map(entry => (entry.j, (entry.i, entry.mval))) - val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) - val retUdata = aCols.join(bRows).map( {case (key, ( (rowInd, rowVal), (colInd, colVal))) - => ((rowInd, colInd), rowVal*colVal)}).reduceByKey(_ + _) - .map{ case ((row, col), mval) => MatrixEntry(row, col, mval)} - val retU = SparseMatrix(retUdata, m, sigma.length) - - MatrixSVD(retU, retS, retV) - } + val vsirdd = sc.makeRDD(Array.tabulate(V.rows, sigma.length) { + (i, j) => ((i, j), V.get(i, j) / sigma(j)) + }.flatten) + if (computeU) { + // Multiply A by VS^-1 + val aCols = data.map(entry => (entry.j, (entry.i, entry.mval))) + val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) + val retUdata = aCols.join(bRows).map { + case (key, ((rowInd, rowVal), (colInd, colVal))) => + ((rowInd, colInd), rowVal * colVal) + }.reduceByKey(_ + _).map { + case ((row, col), mval) => MatrixEntry(row, col, mval) + } + val retU = SparseMatrix(retUdata, m, sigma.length) + MatrixSVD(retU, retS, retV) + } else { + MatrixSVD(null, retS, retV) + } + } +} + +/** + * Top-level methods for calling sparse Singular Value Decomposition + * NOTE: All matrices are 0-indexed + */ +object SVD { def main(args: Array[String]) { if (args.length < 8) { println("Usage: SVD " + - " ") + " ") System.exit(1) } - val (master, inputFile, m, n, k, output_u, output_s, output_v) = + val (master, inputFile, m, n, k, output_u, output_s, output_v) = (args(0), args(1), args(2).toInt, args(3).toInt, - args(4).toInt, args(5), args(6), args(7)) - + args(4).toInt, args(5), args(6), args(7)) + val sc = new SparkContext(master, "SVD") - - val rawdata = sc.textFile(inputFile) - val data = rawdata.map { line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble) + + val rawData = sc.textFile(inputFile) + val data = rawData.map { + line => + val parts = line.split(',') + MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble) } - val decomposed = SVD.sparseSVD(SparseMatrix(data, m, n), k) + val decomposed = new SVD().setK(k).compute(SparseMatrix(data, m, n)) val u = decomposed.U.data val s = decomposed.S.data val v = decomposed.V.data - + println("Computed " + s.collect().length + " singular values and vectors") u.saveAsTextFile(output_u) s.saveAsTextFile(output_s) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala new file mode 100644 index 0000000000000..e4ef3c58e8680 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.mllib.linalg + +import org.apache.spark.rdd.RDD + + +/** + * Class that represents a dense matrix + * + * @param rows RDD of rows + * @param m number of rows + * @param n number of columns + */ +case class TallSkinnyDenseMatrix(val rows: RDD[MatrixRow], val m: Int, val n: Int) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala new file mode 100644 index 0000000000000..b3a450e92394e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala @@ -0,0 +1,31 @@ +/* + * 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.mllib.linalg + +/** + * Class that represents the singular value decomposition of a matrix + * + * @param U such that A = USV^T is a TallSkinnyDenseMatrix + * @param S such that A = USV^T is a simple double array + * @param V such that A = USV^T, V is a 2d array matrix that holds + * singular vectors in columns. Columns are inner arrays + * i.e. V(i)(j) is standard math notation V_{ij} + */ +case class TallSkinnyMatrixSVD(val U: TallSkinnyDenseMatrix, + val S: Array[Double], + val V: Array[Array[Double]]) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala new file mode 100644 index 0000000000000..afe081295bfae --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala @@ -0,0 +1,65 @@ +/* + * 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.mllib.util + +import org.apache.spark.SparkContext._ + +import org.apache.spark.mllib.linalg._ + +/** + * Helper methods for linear algebra + */ +object LAUtils { + /** + * Convert a SparseMatrix into a TallSkinnyDenseMatrix + * + * @param sp Sparse matrix to be converted + * @return dense version of the input + */ + def sparseToTallSkinnyDense(sp: SparseMatrix): TallSkinnyDenseMatrix = { + val m = sp.m + val n = sp.n + val rows = sp.data.map(x => (x.i, (x.j, x.mval))).groupByKey().map { + case (i, cols) => + val rowArray = Array.ofDim[Double](n) + var j = 0 + while (j < cols.size) { + rowArray(cols(j)._1) = cols(j)._2 + j += 1 + } + MatrixRow(i, rowArray) + } + TallSkinnyDenseMatrix(rows, m, n) + } + + /** + * Convert a TallSkinnyDenseMatrix to a SparseMatrix + * + * @param a matrix to be converted + * @return sparse version of the input + */ + def denseToSparse(a: TallSkinnyDenseMatrix): SparseMatrix = { + val m = a.m + val n = a.n + val data = a.rows.flatMap { + mrow => Array.tabulate(n)(j => MatrixEntry(mrow.i, j, mrow.data(j))) + .filter(x => x.mval != 0) + } + SparseMatrix(data, m, n) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala new file mode 100644 index 0000000000000..5e5086b1bf73e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala @@ -0,0 +1,124 @@ +/* + * 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.mllib.linalg + +import scala.util.Random + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD + +import org.apache.spark.mllib.util._ + +import org.jblas._ + +class PCASuite extends FunSuite with BeforeAndAfterAll { + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + val EPSILON = 1e-3 + + // Return jblas matrix from sparse matrix RDD + def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { + val data = matrix.data + val ret = DoubleMatrix.zeros(matrix.m, matrix.n) + matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) + ret + } + + def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { + assert(a.rows == b.rows && a.columns == b.columns, + "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") + for (i <- 0 until a.columns) { + val aCol = a.getColumn(i) + val bCol = b.getColumn(i) + val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) + assert(diff < EPSILON, "matrix mismatch: " + diff) + } + } + + test("full rank matrix pca") { + val m = 5 + val n = 3 + val dataArr = Array.tabulate(m,n){ (a, b) => + MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten + val data = sc.makeRDD(dataArr, 3) + val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) + + val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), + (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), + (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) + val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3)), 3) + + val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) + + assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) + } + + test("sparse matrix full rank matrix pca") { + val m = 5 + val n = 3 + // the entry that gets dropped is zero to test sparse support + val dataArr = Array.tabulate(m,n){ (a, b) => + MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten.drop(1) + val data = sc.makeRDD(dataArr, 3) + val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) + + val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), + (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), + (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) + val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) + + val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) + + assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) + } + + test("truncated matrix pca") { + val m = 5 + val n = 3 + val dataArr = Array.tabulate(m,n){ (a, b) => + MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten + + val data = sc.makeRDD(dataArr, 3) + val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) + + val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), + (1,0,-0.1448), (1,1,0.7483), + (2,0,0.9553), (2,1,-0.0649)) + val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) + + val k = 2 + val coeffs = new DoubleMatrix(new PCA().setK(k).compute(a)) + + assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,k)), coeffs) + } +} + + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala index a92386865a189..20e2b0f84be06 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala @@ -28,6 +28,8 @@ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.util._ + import org.jblas._ class SVDSuite extends FunSuite with BeforeAndAfterAll { @@ -54,43 +56,77 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { ret } - def assertMatrixEquals(a: DoubleMatrix, b: DoubleMatrix) { - assert(a.rows == b.rows && a.columns == b.columns, "dimension mismatch") - val diff = DoubleMatrix.zeros(a.rows, a.columns) - Array.tabulate(a.rows, a.columns){(i, j) => - diff.put(i, j, - Math.min(Math.abs(a.get(i, j) - b.get(i, j)), - Math.abs(a.get(i, j) + b.get(i, j)))) } - assert(diff.norm1 < EPSILON, "matrix mismatch: " + diff.norm1) + def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { + assert(a.rows == b.rows && a.columns == b.columns, + "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") + for (i <- 0 until a.columns) { + val aCol = a.getColumn(i) + val bCol = b.getColumn(i) + val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) + assert(diff < EPSILON, "matrix mismatch: " + diff) + } } test("full rank matrix svd") { val m = 10 val n = 3 - val data = sc.makeRDD(Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten ) + val datarr = Array.tabulate(m,n){ (a, b) => + MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten + val data = sc.makeRDD(datarr, 3) val a = SparseMatrix(data, m, n) - val decomposed = SVD.sparseSVD(a, n) + val decomposed = new SVD().setK(n).compute(a) val u = decomposed.U val v = decomposed.V val s = decomposed.S - val densea = getDenseMatrix(a) - val svd = Singular.sparseSVD(densea) + val denseA = getDenseMatrix(a) + val svd = Singular.sparseSVD(denseA) val retu = getDenseMatrix(u) val rets = getDenseMatrix(s) val retv = getDenseMatrix(v) - + + + // check individual decomposition + assertMatrixApproximatelyEquals(retu, svd(0)) + assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) + assertMatrixApproximatelyEquals(retv, svd(2)) + + // check multiplication guarantee + assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) + } + + test("dense full rank matrix svd") { + val m = 10 + val n = 3 + val datarr = Array.tabulate(m,n){ (a, b) => + MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten + val data = sc.makeRDD(datarr, 3) + + val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) + + val decomposed = new SVD().setK(n).setComputeU(true).compute(a) + val u = LAUtils.denseToSparse(decomposed.U) + val v = decomposed.V + val s = decomposed.S + + val denseA = getDenseMatrix(LAUtils.denseToSparse(a)) + val svd = Singular.sparseSVD(denseA) + + val retu = getDenseMatrix(u) + val rets = DoubleMatrix.diag(new DoubleMatrix(s)) + val retv = new DoubleMatrix(v) + + // check individual decomposition - assertMatrixEquals(retu, svd(0)) - assertMatrixEquals(rets, DoubleMatrix.diag(svd(1))) - assertMatrixEquals(retv, svd(2)) + assertMatrixApproximatelyEquals(retu, svd(0)) + assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) + assertMatrixApproximatelyEquals(retv, svd(2)) // check multiplication guarantee - assertMatrixEquals(retu.mmul(rets).mmul(retv.transpose), densea) + assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) } test("rank one matrix svd") { @@ -102,7 +138,7 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val a = SparseMatrix(data, m, n) - val decomposed = SVD.sparseSVD(a, k) + val decomposed = new SVD().setK(k).compute(a) val u = decomposed.U val s = decomposed.S val v = decomposed.V @@ -110,20 +146,20 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { assert(retrank == 1, "rank returned not one") - val densea = getDenseMatrix(a) - val svd = Singular.sparseSVD(densea) + val denseA = getDenseMatrix(a) + val svd = Singular.sparseSVD(denseA) val retu = getDenseMatrix(u) val rets = getDenseMatrix(s) val retv = getDenseMatrix(v) // check individual decomposition - assertMatrixEquals(retu, svd(0).getColumn(0)) - assertMatrixEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixEquals(retv, svd(2).getColumn(0)) + assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) + assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) + assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) // check multiplication guarantee - assertMatrixEquals(retu.mmul(rets).mmul(retv.transpose), densea) + assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) } test("truncated with k") { @@ -135,14 +171,14 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { val k = 1 // only one svalue above this - val decomposed = SVD.sparseSVD(a, k) + val decomposed = new SVD().setK(k).compute(a) val u = decomposed.U val s = decomposed.S val v = decomposed.V val retrank = s.data.collect().length - val densea = getDenseMatrix(a) - val svd = Singular.sparseSVD(densea) + val denseA = getDenseMatrix(a) + val svd = Singular.sparseSVD(denseA) val retu = getDenseMatrix(u) val rets = getDenseMatrix(s) @@ -151,8 +187,8 @@ class SVDSuite extends FunSuite with BeforeAndAfterAll { assert(retrank == 1, "rank returned not one") // check individual decomposition - assertMatrixEquals(retu, svd(0).getColumn(0)) - assertMatrixEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixEquals(retv, svd(2).getColumn(0)) + assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) + assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) + assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) } } From ca76423e23f5f626c56041cecbc38a93ae1e0298 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 20 Mar 2014 14:13:16 -0700 Subject: [PATCH 114/397] [Hot Fix #42] Do not stop SparkUI if bind() is not called This is a bug fix for #42 (79d07d66040f206708e14de393ab0b80020ed96a). In Master, we do not bind() each SparkUI because we do not want to start a server for each finished application. However, when we remove the associated application, we call stop() on the SparkUI, which throws an assertion failure. This fix ensures we don't call stop() on a SparkUI that was never bind()'ed. Author: Andrew Or Closes #188 from andrewor14/ui-fix and squashes the following commits: 94a925f [Andrew Or] Do not stop SparkUI if bind() is not called --- .../scala/org/apache/spark/deploy/master/Master.scala | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 1fd211416976e..9ed49e01be639 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -149,7 +149,6 @@ private[spark] class Master( override def postStop() { webUi.stop() - appIdToUI.values.foreach(_.stop()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -622,10 +621,7 @@ private[spark] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach( a => { - appIdToUI.remove(a.id).foreach { ui => - ui.stop() - webUi.detachUI(ui) - } + appIdToUI.remove(a.id).foreach { ui => webUi.detachUI(ui) } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) @@ -681,10 +677,7 @@ private[spark] class Master( // Do not call ui.bind() to avoid creating a new server for each application ui.start() val success = replayerBus.replay(eventLogDir) - if (!success) { - ui.stop() - None - } else Some(ui) + if (success) Some(ui) else None } /** Generate a new app ID given a app's submission date */ From 9aadcffabd226557174f3ff566927f873c71672e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 20 Mar 2014 18:03:20 -0700 Subject: [PATCH 115/397] SPARK-1251 Support for optimizing and executing structured queries MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This pull request adds support to Spark for working with structured data using a simple SQL dialect, HiveQL and a Scala Query DSL. *This is being contributed as a new __alpha component__ to Spark and does not modify Spark core or other components.* The code is broken into three primary components: - 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 structured scala queries 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. A more complete design of this new component can be found in [the associated JIRA](https://spark-project.atlassian.net/browse/SPARK-1251). [An updated version of the Spark documentation, including API Docs for all three sub-components,](http://people.apache.org/~pwendell/catalyst-docs/sql-programming-guide.html) is also available for review. With this PR comes support for inferring the schema of existing RDDs that contain case classes. Using this information, developers can now express structured queries that are automatically compiled into RDD operations. ```scala // Define the schema using a case class. case class Person(name: String, age: Int) val people: RDD[Person] = sc.textFile("people.txt").map(_.split(",")).map(p => Person(p(0), p(1).toInt)) // The following is the same as 'SELECT name FROM people WHERE age >= 10 && age <= 19' val teenagers = people.where('age >= 10).where('age <= 19).select('name).toRdd ``` RDDs can also be registered as Tables, allowing SQL queries to be written over them. ```scala people.registerAsTable("people") val teenagers = sql("SELECT name FROM people WHERE age >= 10 && age <= 19") ``` The results of queries are themselves RDDs and support standard RDD operations: ```scala teenagers.map(t => "Name: " + t(0)).collect().foreach(println) ``` Finally, with the optional Hive support, users can read and write data located in existing Apache Hive deployments using HiveQL. ```scala 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 sql("SELECT key, value FROM src").collect().foreach(println) ``` ## Relationship to Shark Unlike Shark, Spark SQL does not act as a drop in replacement for Hive or the HiveServer. Instead this new feature is intended to make it easier for Spark developers to run queries over structured data, using either SQL or the query DSL. After this sub-project graduates from Alpha status it will likely become a new optimizer/backend for the Shark project. Author: Michael Armbrust Author: Yin Huai Author: Reynold Xin Author: Lian, Cheng Author: Andre Schumacher Author: Yin Huai Author: Timothy Chen Author: Cheng Lian Author: Timothy Chen Author: Henry Cook Author: Mark Hamstra Closes #146 from marmbrus/catalyst and squashes the following commits: 458bd1b [Michael Armbrust] Update people.txt 0d638c3 [Michael Armbrust] Typo fix from @ash211. bdab185 [Michael Armbrust] Address another round of comments: * Doc examples can now copy/paste into spark-shell. * SQLContext is serializable * Minor parser bugs fixed * Self-joins of RDDs now handled correctly. * Removed deprecated examples * Removed deprecated parquet docs * Made more of the API private * Copied all the DSLQuery tests and rewrote them as SQLQueryTests 778299a [Michael Armbrust] Fix some old links to spark-project.org fead0b6 [Michael Armbrust] Create a new RDD type, SchemaRDD, that is now the return type for all SQL operations. This improves the old API by reducing the number of implicits that are required, and avoids throwing away schema information when returning an RDD to the user. This change also makes it slightly less verbose to run language integrated queries. fee847b [Michael Armbrust] Merge remote-tracking branch 'origin/master' into catalyst, integrating changes to serialization for ShuffledRDD. 48a99bc [Michael Armbrust] Address first round of feedback. 461581c [Michael Armbrust] Blacklist test that depends on JVM specific rounding behaviour adcf1a4 [Henry Cook] Update sql-programming-guide.md 9dffbfa [Michael Armbrust] Style fixes. Add downloading of test cases to jenkins. 6978dd8 [Michael Armbrust] update docs, add apache license 1d0eb63 [Michael Armbrust] update changes with spark core e5e1d6b [Michael Armbrust] Remove travis configuration. c2efad6 [Michael Armbrust] First draft of SQL documentation. 013f62a [Michael Armbrust] Fix documentation / code style. c01470f [Michael Armbrust] Clean up example 2f22454 [Michael Armbrust] WIP: Parquet example. ce8073b [Michael Armbrust] clean up implicits. f7d992d [Michael Armbrust] Naming / spelling. 9eb0294 [Michael Armbrust] Bring expressions implicits into SqlContext. d2d9678 [Michael Armbrust] Make sure hive isn't in the assembly jar. Create a separate, optional Hive assembly that is used when present. 8b35e0a [Michael Armbrust] address feedback, work on DSL 5d71074 [Michael Armbrust] Merge pull request #62 from AndreSchumacher/parquet_file_fixes f93aa39 [Andre Schumacher] Better handling of path names in ParquetRelation 1a4bbd9 [Michael Armbrust] Merge pull request #60 from marmbrus/maven 3386e4f [Michael Armbrust] Merge pull request #58 from AndreSchumacher/parquet_fixes 3447c3e [Michael Armbrust] Don't override the metastore / warehouse in non-local/test hive context. 7233a74 [Michael Armbrust] initial support for maven builds f0ba39e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into maven 7386a9f [Michael Armbrust] Initial example programs using spark sql. aeaef54 [Andre Schumacher] Removing unnecessary Row copying and reverting some changes to MutableRow 7ca4b4e [Andre Schumacher] Improving checks in Parquet tests 5bacdc0 [Andre Schumacher] Moving towards mutable rows inside ParquetRowSupport 54637ec [Andre Schumacher] First part of second round of code review feedback c2a658d [Michael Armbrust] Merge pull request #55 from marmbrus/mutableRows ba28849 [Michael Armbrust] code review comments. d994333 [Michael Armbrust] Remove copies before shuffle, this required changing the default shuffle serialization. 9049cf0 [Michael Armbrust] 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. 959bdf0 [Michael Armbrust] Don't silently swallow all KryoExceptions, only the one that indicates the end of a stream. d371393 [Michael Armbrust] Add a framework for dealing with mutable rows to reduce the number of object allocations that occur in the critical path. c9f8fb3 [Michael Armbrust] Merge pull request #53 from AndreSchumacher/parquet_support 3c3f962 [Michael Armbrust] Fix a bug due to array reuse. This will need to be revisited after we merge the mutable row PR. 7d0f13e [Michael Armbrust] Update parquet support with master. 9d419a6 [Michael Armbrust] Merge remote-tracking branch 'catalyst/catalystIntegration' into parquet_support 0040ae6 [Andre Schumacher] Feedback from code review 1ce01c7 [Michael Armbrust] Merge pull request #56 from liancheng/unapplySeqForRow 70e489d [Cheng Lian] Fixed a spelling typo 6d315bb [Cheng Lian] Added Row.unapplySeq to extract fields from a Row object. 8d5da5e [Michael Armbrust] modify compute-classpath.sh to include datanucleus jars explicitly 99e61fb [Michael Armbrust] Merge pull request #51 from marmbrus/expressionEval 7b9d142 [Michael Armbrust] Update travis to increase permgen size. da9afbd [Michael Armbrust] Add byte wrappers for hive UDFS. 6fdefe6 [Michael Armbrust] Port sbt improvements from master. 296fe50 [Michael Armbrust] Address review feedback. d7fbc3a [Michael Armbrust] Several performance enhancements and simplifications of the expression evaluation framework. 3bda72d [Andre Schumacher] Adding license banner to new files 3ac9eb0 [Andre Schumacher] Rebasing to new main branch c863bed [Andre Schumacher] Codestyle checks 61e3bfb [Andre Schumacher] Adding WriteToFile operator and rewriting ParquetQuerySuite 3321195 [Andre Schumacher] Fixing one import in ParquetQueryTests.scala 3a0a552 [Andre Schumacher] Reorganizing Parquet table operations 18fdc44 [Andre Schumacher] Reworking Parquet metadata in relation and adding CREATE TABLE AS for Parquet tables 75262ee [Andre Schumacher] Integrating operations on Parquet files into SharkStrategies f347273 [Andre Schumacher] Adding ParquetMetaData extraction, fixing schema projection 6a6bf98 [Andre Schumacher] Added column projections to ParquetTableScan 0f17d7b [Andre Schumacher] Rewriting ParquetRelation tests with RowWriteSupport a11e364 [Andre Schumacher] Adding Parquet RowWriteSupport 6ad05b3 [Andre Schumacher] Moving ParquetRelation to spark.sql core eb0e521 [Andre Schumacher] Fixing package names and other problems that came up after the rebase 99a9209 [Andre Schumacher] Expanding ParquetQueryTests to cover all primitive types b33e47e [Andre Schumacher] First commit of Parquet import of primitive column types c334386 [Michael Armbrust] Initial support for generating schema's based on case classes. 608a29e [Michael Armbrust] Add hive as a repl dependency 7413ac2 [Michael Armbrust] make test downloading quieter. 4d57d0e [Michael Armbrust] Fix test execution on travis. 5f2963c [Michael Armbrust] naming and continuous compilation fixes. f5e7492 [Michael Armbrust] Add Apache license. Make naming more consistent. 3ac9416 [Michael Armbrust] Merge support for working with schema-ed RDDs using catalyst in as a spark subproject. 2225431 [Michael Armbrust] Merge pull request #48 from marmbrus/minorFixes d393d2a [Michael Armbrust] Review Comments: Add comment to map that adds a sub query. 24eaa79 [Michael Armbrust] fix > 100 chars 6e04e5b [Michael Armbrust] Add insertIntoTable to the DSL. df88f01 [Michael Armbrust] add a simple test for aggregation 18a861b [Michael Armbrust] Correctly convert nested products into nested rows when turning scala data into catalyst data. b922511 [Michael Armbrust] Fix insertion of nested types into hive tables. 5fe7de4 [Michael Armbrust] Move table creation out of rule into a separate function. a430895 [Michael Armbrust] Planning for logical Repartition operators. 532dd37 [Michael Armbrust] Allow the local warehouse path to be specified. 4905b2b [Michael Armbrust] Add more efficient TopK that avoids global sort for logical Sort => StopAfter. 8c01c24 [Michael Armbrust] Move definition of Row out of execution to top level sql package. c9116a6 [Michael Armbrust] Add combiner to avoid NPE when spark performs external aggregation. 29effad [Michael Armbrust] Include alias in attributes that are produced by overridden tables. 9990ec7 [Michael Armbrust] Merge pull request #28 from liancheng/columnPruning f22df3a [Michael Armbrust] Merge pull request #37 from yhuai/SerDe cf4db59 [Lian, Cheng] Added golden answers for PruningSuite 54f165b [Lian, Cheng] Fixed spelling typo in two golden answer file names 2682f72 [Lian, Cheng] Merge remote-tracking branch 'origin/master' into columnPruning c5a4fab [Lian, Cheng] Merge branch 'master' into columnPruning f670c8c [Yin Huai] Throw a NotImplementedError for not supported clauses in a CTAS query. 128a9f8 [Yin Huai] Minor changes. 017872c [Yin Huai] Remove stats20 from whitelist. a1a4776 [Yin Huai] Update comments. feb022c [Yin Huai] Partitioning key should be case insensitive. 555fb1d [Yin Huai] Correctly set the extension for a text file. d00260b [Yin Huai] Strips backticks from partition keys. 334aace [Yin Huai] New golden files. a40d6d6 [Yin Huai] Loading the static partition specified in a INSERT INTO/OVERWRITE query. 428aff5 [Yin Huai] Distinguish `INSERT INTO` and `INSERT OVERWRITE`. eea75c5 [Yin Huai] Correctly set codec. 45ffb86 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SerDeNew e089627 [Yin Huai] Code style. 563bb22 [Yin Huai] Set compression info in FileSinkDesc. 35c9a8a [Michael Armbrust] Merge pull request #46 from marmbrus/reviewFeedback bdab5ed [Yin Huai] Add a TODO for loading data into partitioned tables. 5495fab [Yin Huai] Remove cloneRecords which is no longer needed. 1596e1b [Yin Huai] Cleanup imports to make IntelliJ happy. 3bb272d [Michael Armbrust] move org.apache.spark.sql package.scala to the correct location. 8506c17 [Michael Armbrust] Address review feedback. 3cb4f2e [Michael Armbrust] Merge pull request #45 from tnachen/master 9ad474d [Michael Armbrust] Merge pull request #44 from marmbrus/sampling 566fd66 [Timothy Chen] Whitelist tests and add support for Binary type 69adf72 [Yin Huai] Set cloneRecords to false. a9c3188 [Timothy Chen] Fix udaf struct return 346f828 [Yin Huai] Move SharkHadoopWriter to the correct location. 59e37a3 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SerDeNew ed3a1d1 [Yin Huai] Load data directly into Hive. 7f206b5 [Michael Armbrust] Add support for hive TABLESAMPLE PERCENT. b6de691 [Michael Armbrust] Merge pull request #43 from liancheng/fixMakefile 1f6260d [Lian, Cheng] Fixed package name and test suite name in Makefile 5ae010f [Michael Armbrust] Merge pull request #42 from markhamstra/non-ascii 678341a [Mark Hamstra] Replaced non-ascii text 887f928 [Yin Huai] Merge remote-tracking branch 'upstream/master' into SerDeNew 1f7d00a [Reynold Xin] Merge pull request #41 from marmbrus/splitComponents 7588a57 [Michael Armbrust] Break into 3 major components and move everything into the org.apache.spark.sql package. bc9a12c [Michael Armbrust] Move hive test files. 5720d2b [Lian, Cheng] Fixed comment typo f0c3742 [Lian, Cheng] Refactored PhysicalOperation f235914 [Lian, Cheng] Test case udf_regex and udf_like need BooleanWritable registered cf691df [Lian, Cheng] Added the PhysicalOperation to generalize ColumnPrunings 2407a21 [Lian, Cheng] Added optimized logical plan to debugging output a7ad058 [Michael Armbrust] Merge pull request #40 from marmbrus/includeGoldens 9329820 [Michael Armbrust] add golden answer files to repository dce0593 [Michael Armbrust] move golden answer to the source code directory. 964368f [Michael Armbrust] Merge pull request #39 from marmbrus/lateralView 7785ee6 [Michael Armbrust] Tighten visibility based on comments. 341116c [Michael Armbrust] address comments. 0e6c1d7 [Reynold Xin] Merge pull request #38 from yhuai/parseDBNameInCTAS 2897deb [Michael Armbrust] fix scaladoc 7123225 [Yin Huai] Correctly parse the db name and table name in INSERT queries. b376d15 [Michael Armbrust] fix newlines at EOF 5cc367c [Michael Armbrust] use berkeley instead of cloudbees ff5ea3f [Michael Armbrust] new golden db92adc [Michael Armbrust] more tests passing. clean up logging. 740febb [Michael Armbrust] Tests for tgfs. 0ce61b0 [Michael Armbrust] Docs for GenericHiveUdtf. ba8897f [Michael Armbrust] Merge remote-tracking branch 'yin/parseDBNameInCTAS' into lateralView dd00b7e [Michael Armbrust] initial implementation of generators. ea76cf9 [Michael Armbrust] Add NoRelation to planner. bea4b7f [Michael Armbrust] Add SumDistinct. 016b489 [Michael Armbrust] fix typo. acb9566 [Michael Armbrust] Correctly type attributes of CTAS. 8841eb8 [Michael Armbrust] Rename Transform -> ScriptTransformation. 02ff8e4 [Yin Huai] Correctly parse the db name and table name in a CTAS query. 5e4d9b4 [Michael Armbrust] Merge pull request #35 from marmbrus/smallFixes 5479066 [Reynold Xin] Merge pull request #36 from marmbrus/partialAgg 8017afb [Michael Armbrust] fix copy paste error. dc6353b [Michael Armbrust] turn off deprecation cab1a84 [Michael Armbrust] Fix PartialAggregate inheritance. 883006d [Michael Armbrust] improve tests. 32b615b [Michael Armbrust] add override to asPartial. e1999f9 [Yin Huai] Use Deserializer and Serializer instead of AbstractSerDe. f94345c [Michael Armbrust] fix doc link d8cb805 [Michael Armbrust] Implement partial aggregation. ccdb07a [Michael Armbrust] Fix bug where averages of strings are turned into sums of strings. Remove a blank line. b4be6a5 [Michael Armbrust] better logging when applying rules. 67128b8 [Reynold Xin] Merge pull request #30 from marmbrus/complex cb57459 [Michael Armbrust] blacklist machine specific test. 2f27604 [Michael Armbrust] Address comments / style errors. 389525d [Michael Armbrust] update golden, blacklist mr. e3c10bd [Michael Armbrust] update whitelist. 44d343c [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into complex 42ec4af [Michael Armbrust] improve complex type support in hive udfs/udafs. ab5bff3 [Michael Armbrust] Support for get item of map types. 1679554 [Michael Armbrust] add toString for if and IS NOT NULL. ab9a131 [Michael Armbrust] when UDFs fail they should return null. 25288d0 [Michael Armbrust] Implement [] for arrays and maps. e7933e9 [Michael Armbrust] fix casting bug when working with fractional expressions. 010accb [Michael Armbrust] add tinyint to metastore type parser. 7a0f543 [Michael Armbrust] Avoid propagating types from unresolved nodes. ac9d7de [Michael Armbrust] Resolve *s in Transform clauses. 692a477 [Michael Armbrust] Support for wrapping arrays to be written into hive tables. 92e4158 [Reynold Xin] Merge pull request #32 from marmbrus/tooManyProjects 9c06778 [Michael Armbrust] fix serialization issues, add JavaStringObjectInspector. 72a003d [Michael Armbrust] revert regex change 7661b6c [Michael Armbrust] blacklist machines specific tests aa430e7 [Michael Armbrust] Update .travis.yml e4def6b [Michael Armbrust] set dataType for HiveGenericUdfs. 5e54aa6 [Michael Armbrust] quotes for struct field names. bbec500 [Michael Armbrust] update test coverage, new golden 3734a94 [Michael Armbrust] only quote string types. 3f9e519 [Michael Armbrust] use names w/ boolean args 5b3d2c8 [Michael Armbrust] implement distinct. 5b33216 [Michael Armbrust] work on decimal support. 2c6deb3 [Michael Armbrust] improve printing compatibility. 35a70fb [Michael Armbrust] multi-letter field names. a9388fb [Michael Armbrust] printing for map types. c3feda7 [Michael Armbrust] use toArray. c654f19 [Michael Armbrust] Support for list and maps in hive table scan. cf8d992 [Michael Armbrust] Use built in functions for creating temp directory. 1579eec [Michael Armbrust] Only cast unresolved inserts. 6420c7c [Michael Armbrust] Memoize the ordinal in the GetField expression. da7ae9d [Michael Armbrust] Add boolean writable that was breaking udf_regexp test. Not sure how this was passing before... 6709441 [Michael Armbrust] Evaluation for accessing nested fields. dc6463a [Michael Armbrust] Support for resolving access to nested fields using "." notation. d670e41 [Michael Armbrust] Print nested fields like hive does. efa7217 [Michael Armbrust] Support for reading structs in HiveTableScan. 9c22b4e [Michael Armbrust] Support for parsing nested types. 82163e3 [Michael Armbrust] special case handling of partitionKeys when casting insert into tables ea6f37f [Michael Armbrust] fix style. 7845364 [Michael Armbrust] deactivate concurrent test. b649c20 [Michael Armbrust] fix test logging / caching. 1590568 [Michael Armbrust] add log4j.properties 19bfd74 [Michael Armbrust] store hive output in circular buffer dfb67aa [Michael Armbrust] add test case cb775ac [Michael Armbrust] get rid of SharkContext singleton 2de89d0 [Michael Armbrust] Merge pull request #13 from tnachen/master 63003e9 [Michael Armbrust] Fix spacing. 41b41f3 [Michael Armbrust] Only cast unresolved inserts. 6eb5960 [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into udafs 5b7afd8 [Michael Armbrust] Merge pull request #10 from yhuai/exchangeOperator b1151a8 [Timothy Chen] Fix load data regex 8e0931f [Michael Armbrust] Cast to avoid using deprecated hive API. e079f2b [Timothy Chen] Add GenericUDAF wrapper and HiveUDAFFunction 45b334b [Yin Huai] fix comments 235cbb4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator fc67b50 [Yin Huai] Check for a Sort operator with the global flag set instead of an Exchange operator with a RangePartitioning. 6015f93 [Michael Armbrust] Merge pull request #29 from rxin/style 271e483 [Michael Armbrust] Update build status icon. d3a3d48 [Michael Armbrust] add testing to travis 807b2d7 [Michael Armbrust] check style and publish docs with travis d20b565 [Michael Armbrust] fix if style bce024d [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into style Disable if brace checking as it errors in single line functional cases unlike the style guide. d91e276 [Michael Armbrust] 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. f47c2f6 [Yin Huai] set outputPartitioning in BroadcastNestedLoopJoin 41bbee6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator 7e24436 [Reynold Xin] Removed dependency on JDK 7 (nio.file). 5c1e600 [Reynold Xin] Added hash code implementation for AttributeReference 7213a2c [Reynold Xin] style fix for Hive.scala. 08e4d05 [Reynold Xin] First round of style cleanup. 605255e [Reynold Xin] Added scalastyle checker. 61e729c [Lian, Cheng] Added ColumnPrunings strategy and test cases 2486fb7 [Lian, Cheng] Fixed spelling 8ee41be [Lian, Cheng] Minor refactoring ebb56fa [Michael Armbrust] add travis config 4c89d6e [Reynold Xin] Merge pull request #27 from marmbrus/moreTests d4f539a [Michael Armbrust] blacklist mr and user specific tests. 677eb07 [Michael Armbrust] Update test whitelist. 5dab0bc [Michael Armbrust] Merge pull request #26 from liancheng/serdeAndPartitionPruning c263c84 [Michael Armbrust] Only push predicates into partitioned table scans. ab77882 [Michael Armbrust] upgrade spark to RC5. c98ede5 [Lian, Cheng] Response to comments from @marmbrus 83d4520 [Yin Huai] marmbrus's comments 70994a3 [Lian, Cheng] Revert unnecessary Scaladoc changes 9ebff47 [Yin Huai] remove unnecessary .toSeq e811d1a [Yin Huai] markhamstra's comments 4802f69 [Yin Huai] The outputPartitioning of a UnaryNode inherits its child's outputPartitioning by default. Also, update the logic in AddExchange to avoid unnecessary shuffling operations. 040fbdf [Yin Huai] AddExchange is the only place to add Exchange operators. 9fb357a [Yin Huai] use getSpecifiedDistribution to create Distribution. ClusteredDistribution and OrderedDistribution do not take Nil as inptu expressions. e9347fc [Michael Armbrust] Remove broken scaladoc links. 99c6707 [Michael Armbrust] upgrade spark 57799ad [Lian, Cheng] Added special treat for HiveVarchar in InsertIntoHiveTable cb49af0 [Lian, Cheng] Fixed Scaladoc links 4e5e4d4 [Lian, Cheng] Added PreInsertionCasts to do necessary casting before insertion 111ffdc [Lian, Cheng] More comments and minor reformatting 9e0d840 [Lian, Cheng] Added partition pruning optimization 761bbb8 [Lian, Cheng] Generalized BindReferences to run against any query plan 04eb5da [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator 9dd3b26 [Michael Armbrust] Fix scaladoc. 6f44cac [Lian, Cheng] Made TableReader & HadoopTableReader private to catalyst 7c92a41 [Lian, Cheng] Added Hive SerDe support ce5fdd6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator 2957f31 [Yin Huai] addressed comments on PR 907db68 [Michael Armbrust] Space after while. 04573a0 [Reynold Xin] Merge pull request #24 from marmbrus/binaryCasts 4e50679 [Reynold Xin] Merge pull request #25 from marmbrus/rowOrderingWhile 5bc1dc2 [Yin Huai] Merge remote-tracking branch 'upstream/master' into exchangeOperator be1fff7 [Michael Armbrust] Replace foreach with while in RowOrdering. Fixes #23 fd084a4 [Michael Armbrust] implement casts binary <=> string. 0b31176 [Michael Armbrust] Merge pull request #22 from rxin/type 548e479 [Yin Huai] merge master into exchangeOperator and fix code style 5b11db0 [Reynold Xin] Added Void to Boolean type widening. 9e3d989 [Reynold Xin] Made HiveTypeCoercion.WidenTypes more clear. 9bb1979 [Reynold Xin] Merge pull request #19 from marmbrus/variadicUnion a2beb38 [Michael Armbrust] Merge pull request #21 from liancheng/fixIssue20 b20a4d4 [Lian, Cheng] Fix issue #20 6d6cb58 [Michael Armbrust] add source links that point to github to the scala doc. 4285962 [Michael Armbrust] Remove temporary test cases 167162f [Michael Armbrust] more merge errors, cleanup. e170ccf [Michael Armbrust] Improve documentation and remove some spurious changes that were introduced by the merge. 6377d0b [Michael Armbrust] Drop empty files, fix if (). c0b0e60 [Michael Armbrust] cleanup broken doc links. 330a88b [Michael Armbrust] Fix bugs in AddExchange. 4f345f2 [Michael Armbrust] Remove SortKey, use RowOrdering. 043e296 [Michael Armbrust] Make physical union nodes variadic. ece15e1 [Michael Armbrust] update unit tests 5c89d2e [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into exchangeOperator Fix deprecated use of combineValuesByKey. Get rid of test where the answer is dependent on the plan execution width. 9804eb5 [Michael Armbrust] upgrade spark 053a371 [Michael Armbrust] Merge pull request #15 from marmbrus/orderedRow 5ab18be [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into orderedRow ca2ff68 [Michael Armbrust] Merge pull request #17 from marmbrus/unionTypes bf9161c [Michael Armbrust] Merge pull request #18 from marmbrus/noSparkAgg 563053f [Michael Armbrust] Address @rxin's comments. 6537c66 [Michael Armbrust] Address @rxin's comments. 2a76fc6 [Michael Armbrust] add notes from @rxin. 685bfa1 [Michael Armbrust] fix spelling 69ed98f [Michael Armbrust] Output a single row for empty Aggregations with no grouping expressions. 7859a86 [Michael Armbrust] Remove SparkAggregate. Its kinda broken and breaks RDD lineage. fc22e01 [Michael Armbrust] whitelist newly passing union test. 3f547b8 [Michael Armbrust] Add support for widening types in unions. 53b95f8 [Michael Armbrust] coercion should not occur until children are resolved. b892e32 [Michael Armbrust] Union is not resolved until the types match up. 95ab382 [Michael Armbrust] Use resolved instead of custom function. This is better because some nodes override the notion of resolved. 81a109d [Michael Armbrust] fix link. f143f61 [Michael Armbrust] Implement sampling. Fixes a flaky test where the JVM notices that RAND as a Comparison method "violates its general contract!" 6cd442b [Michael Armbrust] Use numPartitions variable, fix grammar. c800798 [Michael Armbrust] Add build status icon. 0cf5a75 [Michael Armbrust] Merge pull request #16 from marmbrus/filterPushDown 05d3a0d [Michael Armbrust] Refactor to avoid serializing ordering details with every row. f2fdd77 [Michael Armbrust] fix required distribtion for aggregate. 658866e [Michael Armbrust] Pull back in changes made by @yhuai eliminating CoGroupedLocallyRDD.scala 583a337 [Michael Armbrust] break apart distribution and partitioning. e8d41a9 [Michael Armbrust] Merge remote-tracking branch 'yin/exchangeOperator' into exchangeOperator 0ff8be7 [Michael Armbrust] Cleanup spurious changes and fix doc links. 73c70de [Yin Huai] add a first set of unit tests for data properties. fbfa437 [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into filterPushDown Minor doc improvements. 2b9d80f [Yin Huai] initial commit of adding exchange operators to physical plans. fcbc03b [Michael Armbrust] Fix if (). 7b9080c [Michael Armbrust] Create OrderedRow class to allow ordering to be used by multiple operators. b4adb0f [Michael Armbrust] Merge pull request #14 from marmbrus/castingAndTypes b2a1ec5 [Michael Armbrust] add comment on how using numeric implicitly complicates spark serialization. e286d20 [Michael Armbrust] address code review comments. 80d0681 [Michael Armbrust] fix scaladoc links. de0c248 [Michael Armbrust] Print the executed plan in SharkQuery toString. 3413e61 [Michael Armbrust] Add mapChildren and withNewChildren methods to TreeNode. 404d552 [Michael Armbrust] Better exception when unbound attributes make it to evaluation. fb84ae4 [Michael Armbrust] Refactor DataProperty into Distribution. 2abb0bc [Michael Armbrust] better debug messages, use exists. 098dfc4 [Michael Armbrust] Implement Long sorting again. 60f3a9a [Michael Armbrust] More aggregate functions out of the aggregate class to make things more readable. a1ef62e [Michael Armbrust] Print the executed plan in SharkQuery toString. dfce426 [Michael Armbrust] Add mapChildren and withNewChildren methods to TreeNode. 037a2ed [Michael Armbrust] Better exception when unbound attributes make it to evaluation. ec90620 [Michael Armbrust] Support for Sets as arguments to TreeNode classes. b21f803 [Michael Armbrust] Merge pull request #11 from marmbrus/goldenGen 83adb9d [Yin Huai] add DataProperty 5a26292 [Michael Armbrust] Rules to bring casting more inline with Hive semantics. f0e0161 [Michael Armbrust] Move numeric types into DataTypes simplifying evaluator. This can probably also be use for codegen... 6d2924d [Michael Armbrust] add support for If. Not integrated in HiveQL yet. ccc4dbf [Michael Armbrust] Add optimization rule to simplify casts. 058ec15 [Michael Armbrust] handle more writeables. ffa9f25 [Michael Armbrust] blacklist some more MR tests. aa2239c [Michael Armbrust] filter test lines containing Owner: f71a325 [Michael Armbrust] Update golden jar. a3003ae [Michael Armbrust] Update makefile to use better sharding support. 568d150 [Michael Armbrust] Updates to white/blacklist. 8351f25 [Michael Armbrust] Add an ignored test to remind us we don't do empty aggregations right. c4104ec [Michael Armbrust] Numerous improvements to testing infrastructure. See comments for details. 09c6300 [Michael Armbrust] Add nullability information to StructFields. 5460b2d [Michael Armbrust] load srcpart by default. 3695141 [Michael Armbrust] Lots of parser improvements. 965ac9a [Michael Armbrust] Add expressions that allow access into complex types. 3ba53c9 [Michael Armbrust] Output type suffixes on AttributeReferences. 8777489 [Michael Armbrust] Initial support for operators that allow the user to specify partitioning. e57f97a [Michael Armbrust] more decimal/null support. e1440ed [Michael Armbrust] Initial support for function specific type conversions. 1814ed3 [Michael Armbrust] use childrenResolved function. f2ec57e [Michael Armbrust] Begin supporting decimal. 6924e6e [Michael Armbrust] Handle NullTypes when resolving HiveUDFs 7fcfa8a [Michael Armbrust] Initial support for parsing unspecified partition parameters. d0124f3 [Michael Armbrust] Correctly type null literals. b65626e [Michael Armbrust] Initial support for parsing BigDecimal. a90efda [Michael Armbrust] utility function for outputing string stacktraces. 7102f33 [Michael Armbrust] methods with side-effects should use (). 3ccaef7 [Michael Armbrust] add renaming TODO. bc282c7 [Michael Armbrust] fix bug in getNodeNumbered c8e89d5 [Michael Armbrust] memoize inputSet calculation. 6aefa46 [Michael Armbrust] Skip folding literals. a72e540 [Michael Armbrust] Add IN operator. 04f885b [Michael Armbrust] literals are only non-nullable if they are not null. 35d2948 [Michael Armbrust] correctly order partition and normal attributes in hive relation output. 12fd52d [Michael Armbrust] support for sorting longs. 0606520 [Michael Armbrust] drop old comment. 859200a [Michael Armbrust] support for reading more types from the metastore. 1fedd18 [Michael Armbrust] coercion from null to numeric types 71e902d [Michael Armbrust] fix test cases. cc06b6c [Michael Armbrust] Merge remote-tracking branch 'databricks/master' into interviewAnswer 8a8b521 [Reynold Xin] Merge pull request #8 from marmbrus/testImprovment 86355a6 [Michael Armbrust] throw error if there are unexpected join clauses. c5842d2 [Michael Armbrust] don't throw an error when a select clause outputs multiple copies of the same attribute. 0e975ea [Michael Armbrust] parse bucket sampling as percentage sampling a92919d [Michael Armbrust] add alter view as to native commands f58d5a5 [Michael Armbrust] support for parsing SELECT DISTINCT f0faa26 [Michael Armbrust] add sample and distinct operators. ef7b943 [Michael Armbrust] add metastore support for float e9f4588 [Michael Armbrust] fix > 100 char. 755b229 [Michael Armbrust] blacklist some ddl tests. 9ae740a [Michael Armbrust] blacklist more tests that require MR. 4cfc11a [Michael Armbrust] more test coverage. 0d9d56a [Michael Armbrust] add more native commands to parser 78d730d [Michael Armbrust] Load src test table on RESET. 8364ec2 [Michael Armbrust] whitelist all possible partition values. b01468d [Michael Armbrust] support path rewrites when the query begins with a comment. 4c6b454 [Michael Armbrust] add option for recomputing the cached golden answer when tests fail. 4c5fb0f [Michael Armbrust] makefile target for building new whitelist. 4b6fed8 [Michael Armbrust] support for parsing both DESTINATION and INSERT_INTO. 516481c [Michael Armbrust] Ignore requests to explain native commands. 68aa2e6 [Michael Armbrust] Stronger type for Token extractor. ca4ea26 [Michael Armbrust] Support for parsing UDF(*). 1aafea3 [Michael Armbrust] Configure partition whitelist in TestShark reset. 9627616 [Michael Armbrust] Use current database as default database. 9b02b44 [Michael Armbrust] Fix spelling error. Add failFast mode. 6f64cee [Michael Armbrust] don't line wrap string literal eafaeed [Michael Armbrust] add type documentation f54c94c [Michael Armbrust] make golden answers file a test dependency 5362365 [Michael Armbrust] push conditions into join 0d2388b [Michael Armbrust] Point at databricks hosted scaladoc. 73b29cd [Michael Armbrust] fix bad casting 9aa06c5 [Michael Armbrust] Merge pull request #7 from marmbrus/docFixes 7eff191 [Michael Armbrust] link all the expression names. 83227e4 [Michael Armbrust] fix scaladoc list syntax, add docs for some rules 9de6b74 [Michael Armbrust] fix language feature and deprecation warnings. 0b1960a [Michael Armbrust] Fix broken scala doc links / warnings. b1acb36 [Michael Armbrust] Merge pull request #3 from yhuai/evalauteLiteralsInExpressions 01c00c2 [Michael Armbrust] new golden 5c14857 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions b749b51 [Michael Armbrust] Merge pull request #5 from marmbrus/testCaching 66adceb [Michael Armbrust] Merge pull request #6 from marmbrus/joinWork 1a393da [Yin Huai] folded -> foldable 1e964ea [Yin Huai] update a43d41c [Michael Armbrust] more tests passing! 8ca38d0 [Michael Armbrust] begin support for varchar / binary types. ab8bbd1 [Michael Armbrust] parsing % operator c16c8b5 [Michael Armbrust] case insensitive checking for hooks in tests. 3a90a5f [Michael Armbrust] simpler output when running a single test from the commandline. 5332fee [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions 367fb9e [Yin Huai] update 0cd5cc6 [Michael Armbrust] add BIGINT cast parsing 61b266f [Michael Armbrust] comment for eliminate subqueries. d72a5a2 [Michael Armbrust] add long to literal factory object. b3bd15f [Michael Armbrust] blacklist more mr requiring tests. e06fd38 [Michael Armbrust] black list map reduce tests. 8e7ce30 [Michael Armbrust] blacklist some env specific tests. 6250cbd [Michael Armbrust] Do not exit on test failure b22b220 [Michael Armbrust] also look for cached hive test answers on the classpath. b6e4899 [Yin Huai] formatting e75c90d [Reynold Xin] Merge pull request #4 from marmbrus/hive12 5fabbec [Michael Armbrust] ignore partitioned scan test. scan seems to be working but there is some error about the table already existing? 9e190f5 [Michael Armbrust] drop unneeded () 68b58c1 [Michael Armbrust] drop a few more tests. b0aa400 [Michael Armbrust] update whitelist. c99012c [Michael Armbrust] skip tests with hooks db00ebf [Michael Armbrust] more types for hive udfs dbc3678 [Michael Armbrust] update ghpages repo 138f53d [Yin Huai] addressed comments and added a space after a space after the defining keyword of every control structure. 6f954ee [Michael Armbrust] export the hadoop classpath when starting sbt, required to invoke hive during tests. 46bf41b [Michael Armbrust] add a makefile for priming the test answer cache in parallel. usage: "make -j 8 -i" 8d47ed4 [Yin Huai] comment 2795f05 [Yin Huai] comment e003728 [Yin Huai] move OptimizerSuite to the package of catalyst.optimizer 2941d3a [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions 0bd1688 [Yin Huai] update 6a7bd75 [Michael Armbrust] fix partition column delimiter configuration. e942da1 [Michael Armbrust] Begin upgrade to Hive 0.12.0. b8cd7e3 [Michael Armbrust] Merge pull request #7 from rxin/moreclean 52864da [Reynold Xin] Added executeCollect method to SharkPlan. f0e1cbf [Reynold Xin] Added resolved lazy val to LogicalPlan. b367e36 [Reynold Xin] Replaced the use of ??? with UnsupportedOperationException. 38124bd [Yin Huai] formatting 2924468 [Yin Huai] add two tests for testing pre-order and post-order tree traversal, respectively 555d839 [Reynold Xin] More cleaning ... d48d0e1 [Reynold Xin] Code review feedback. aa2e694 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions 5c421ac [Reynold Xin] Imported SharkEnv, SharkContext, and HadoopTableReader to remove Shark dependency. 479e055 [Reynold Xin] 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) da16e45 [Reynold Xin] Merge pull request #3 from rxin/packagename e36caf5 [Reynold Xin] Renamed Rule.name to Rule.ruleName since name is used too frequently in the code base and is shadowed often by local scope. 72426ed [Reynold Xin] Rename shark2 package to execution. 0892153 [Reynold Xin] Merge pull request #2 from rxin/packagename e58304a [Reynold Xin] Merge pull request #1 from rxin/gitignore 3f9fee1 [Michael Armbrust] rewrite push filter through join optimization. c6527f5 [Reynold Xin] Moved the test src files into the catalyst directory. c9777d8 [Reynold Xin] Put all source files in a catalyst directory. 019ea74 [Reynold Xin] Updated .gitignore to include IntelliJ files. 80ca4be [Timothy Chen] Address comments 0079392 [Michael Armbrust] support for multiple insert commands in a single query 75b5a01 [Michael Armbrust] remove space. 4283400 [Timothy Chen] Add limited predicate push down e547e50 [Michael Armbrust] implement First. e77c9b6 [Michael Armbrust] more work on unique join. c795e06 [Michael Armbrust] improve star expansion a26494e [Michael Armbrust] allow aliases to have qualifiers d078333 [Michael Armbrust] remove extra space a75c023 [Michael Armbrust] implement Coalesce 3a018b6 [Michael Armbrust] fix up docs. ab6f67d [Michael Armbrust] import the string "null" as actual null. 5377c04 [Michael Armbrust] don't call dataType until checking if children are resolved. 191ce3e [Michael Armbrust] analyze rewrite test query. 60b1526 [Michael Armbrust] don't call dataType until checking if children are resolved. 2ab5a32 [Michael Armbrust] stop using uberjar as it has its own set of issues. e42f75a [Michael Armbrust] Merge remote-tracking branch 'origin/master' into HEAD c086a35 [Michael Armbrust] docs, spacing c4060e4 [Michael Armbrust] cleanup 3b85462 [Michael Armbrust] more tests passing bcfc8c5 [Michael Armbrust] start supporting partition attributes when inserting data. c944a95 [Michael Armbrust] First aggregate expression. 1e28311 [Michael Armbrust] make tests execute in alpha order again a287481 [Michael Armbrust] spelling 8492548 [Michael Armbrust] beginning of UNIQUEJOIN parsing. a6ab6c7 [Michael Armbrust] add != 4529594 [Michael Armbrust] draft of coalesce 70f253f [Michael Armbrust] more tests passing! 7349e7b [Michael Armbrust] initial support for test thrift table d3c9305 [Michael Armbrust] fix > 100 char line 93b64b0 [Michael Armbrust] load test tables that are args to "DESCRIBE" 06b2aba [Michael Armbrust] don't be case sensitive when fixing load paths 6355d0e [Michael Armbrust] match actual return type of count with expected cda43ab [Michael Armbrust] don't throw an exception when one of the join tables is empty. fd4b096 [Michael Armbrust] fix casing of null strings as well. 4632695 [Michael Armbrust] support for megastore bigint 67b88cf [Michael Armbrust] more verbose debugging of evaluation return types c680e0d [Michael Armbrust] Failed string => number conversion should return null. 2326be1 [Michael Armbrust] make getClauses case insensitive. dac2786 [Michael Armbrust] correctly handle null values when going from string to numeric types. 045ac4b [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions fb5ddfd [Michael Armbrust] move ViewExamples to examples/ 83833e8 [Michael Armbrust] more tests passing! 47c98d6 [Michael Armbrust] add query tests for like and hash. 1724c16 [Michael Armbrust] clear lines that contain last updated times. cfd6bbc [Michael Armbrust] Quick skipping of tests that we can't even parse. 9b2642b [Michael Armbrust] make the blacklist support regexes 1d50af6 [Michael Armbrust] more datatypes, fix nonserializable instance variables in udfs 910e33e [Michael Armbrust] basic support for building an assembly jar. d55bb52 [Michael Armbrust] add local warehouse/metastore to gitignore. 495d9dc [Michael Armbrust] Add an expression for when we decide to support LIKE natively instead of using the HIVE udf. 65f4e69 [Michael Armbrust] remove incorrect comments 0831a3c [Michael Armbrust] support for parsing some operator udfs. 6c27aa7 [Michael Armbrust] more cast parsing. 43db061 [Michael Armbrust] significant generalization of hive udf functionality. 3fe24ec [Michael Armbrust] better implementation of 3vl in Evaluate, fix some > 100 char lines. e5690a6 [Michael Armbrust] add BinaryType adab892 [Michael Armbrust] Clear out functions that are created during tests when reset is called. d408021 [Michael Armbrust] support for printing out arrays in the output in the same form as hive (e.g., [e1, e1]). 8d5f504 [Michael Armbrust] Example of schema RDD using scala's dynamic trait, resulting in a more standard ORM style of usage. 21f0d91 [Michael Armbrust] Simple example of schemaRdd with scala filter function. 0daaa0e [Michael Armbrust] Promote booleans that appear in comparisons. 2b70abf [Michael Armbrust] true and false literals. ef8b0a5 [Michael Armbrust] more tests. 14d070f [Michael Armbrust] add support for correctly extracting partition keys. 0afbe73 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions 69a0bd4 [Michael Armbrust] promote strings in predicates with number too. 3946e31 [Michael Armbrust] don't build strings unless assertion fails. 90c453d [Michael Armbrust] more tests passing! 6e6417a [Michael Armbrust] correct handling of nulls in boolean logic and sorting. 8000504 [Michael Armbrust] Improve type coercion. 9087152 [Michael Armbrust] fix toString of Not. 58b111c [Michael Armbrust] fix bad scaladoc tag. d5c05c6 [Michael Armbrust] For now, ignore the big data benchmark tests when the data isn't there. ac6376d [Michael Armbrust] Split out general shark query execution driver from test harness. 1d0ae1e [Michael Armbrust] Switch from IndexSeq[Any] to Row interface that will allow us unboxed access to primitive types. d873b2b [Yin Huai] Remove numbers associated with test cases. 8545675 [Yin Huai] Merge remote-tracking branch 'upstream/master' into evalauteLiteralsInExpressions b34a9eb [Michael Armbrust] Merge branch 'master' into filterPushDown d1e7b8e [Michael Armbrust] Update README.md c8b1553 [Michael Armbrust] Update README.md 9307ef9 [Michael Armbrust] update list of passing tests. 934c18c [Michael Armbrust] Filter out non-deterministic lines when comparing test answers. a045c9c [Michael Armbrust] SparkAggregate doesn't actually support sum right now. ae0024a [Yin Huai] update cf80545 [Yin Huai] Merge remote-tracking branch 'origin/evalauteLiteralsInExpressions' into evalauteLiteralsInExpressions 21976ae [Yin Huai] update b4999fe [Yin Huai] Merge remote-tracking branch 'upstream/filterPushDown' into evalauteLiteralsInExpressions dedbf0c [Yin Huai] support Boolean literals eaac9e2 [Yin Huai] explain the limitation of the current EvaluateLiterals 37817b5 [Yin Huai] add a comment to EvaluateLiterals. 468667f [Yin Huai] 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. b1d1843 [Michael Armbrust] more work on big data benchmark tests. cc9a957 [Michael Armbrust] support for creating test tables outside of TestShark 7d7fa9f [Michael Armbrust] support for create table as 5f54f03 [Michael Armbrust] parsing for ASC d42b725 [Michael Armbrust] Sum of strings requires cast 34b30fa [Michael Armbrust] not all attributes need to be bound (e.g. output attributes that are contained in non-leaf operators.) 81659cb [Michael Armbrust] implement transform operator. 5cd76d6 [Michael Armbrust] break up the file based test case code for reuse 1031b65 [Michael Armbrust] support for case insensitive resolution. 320df04 [Michael Armbrust] add snapshot repo for databricks (has shark/spark snapshots) b6f083e [Michael Armbrust] support for publishing scala doc to github from sbt d9d18b4 [Michael Armbrust] debug logging implicit. 669089c [Yin Huai] support Boolean literals ef3321e [Yin Huai] explain the limitation of the current EvaluateLiterals 73a05fd [Yin Huai] add a comment to EvaluateLiterals. 191eb7d [Yin Huai] 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. 80039cc [Yin Huai] Merge pull request #1 from yhuai/master cbe1ca1 [Yin Huai] add explicit result type to the overloaded sideBySide 5c518e4 [Michael Armbrust] fix bug in test. b50dd0e [Michael Armbrust] fix return type of overloaded method 05679b7 [Michael Armbrust] download assembly jar for easy compiling during interview. 8c60cc0 [Michael Armbrust] Update README.md 03b9526 [Michael Armbrust] First draft of optimizer tests. f392755 [Michael Armbrust] Add flatMap to TreeNode 6cbe8d1 [Michael Armbrust] fix bug in side by side, add support for working with unsplit strings 15a53fc [Michael Armbrust] more generic sum calculation and better binding of grouping expressions. 06749d0 [Michael Armbrust] add expression enumerations for query plan operators and recursive version of transform expression. 4b0a888 [Michael Armbrust] implement string comparison and more casts. 356b321 [Michael Armbrust] Update README.md 3776395 [Michael Armbrust] Update README.md 304d17d [Michael Armbrust] Create README.md b7d8be0 [Michael Armbrust] more tests passing. b82481f [Michael Armbrust] add todo comment. 02e6dee [Michael Armbrust] add another test that breaks the harness to the blacklist. cc5efe3 [Michael Armbrust] First draft of broadcast nested loop join with full outer support. c43a259 [Michael Armbrust] comments 15ff448 [Michael Armbrust] better error message when a dsl test throws an exception 76ec650 [Michael Armbrust] fix join conditions e10df99 [Michael Armbrust] Create new expr ids for local relations that exist more than once in a query plan. 91573a4 [Michael Armbrust] initial type promotion e2ef4a5 [Michael Armbrust] logging e43dc1e [Michael Armbrust] add string => int cast evaluation f1f7e96 [Michael Armbrust] fix incorrect generation of join keys 2b27230 [Michael Armbrust] add depth based subtree access 0f6279f [Michael Armbrust] broken tests. 389bc0b [Michael Armbrust] support for partitioned columns in output. 12584f4 [Michael Armbrust] better errors for missing clauses. support for matching multiple clauses with the same name. b67a225 [Michael Armbrust] better errors when types don't match up. 9e74808 [Michael Armbrust] add children resolved. 6d03ce9 [Michael Armbrust] defaults for unresolved relation 2469b00 [Michael Armbrust] skip nodes with unresolved children when doing coersions be5ae2c [Michael Armbrust] better resolution logging cb7b5af [Michael Armbrust] views example 420e05b [Michael Armbrust] more tests passing! 6916c63 [Michael Armbrust] Reading from partitioned hive tables. a1245f9 [Michael Armbrust] more tests passing 956e760 [Michael Armbrust] extended explain 5f14c35 [Michael Armbrust] more test tables supported 175c43e [Michael Armbrust] better errors for parse exceptions 480ade5 [Michael Armbrust] don't use partial cached results. 8a9d21c [Michael Armbrust] fix evaluation 7aee69c [Michael Armbrust] parsing for joins, boolean logic 7fcf480 [Michael Armbrust] test for and logic 3ea9b00 [Michael Armbrust] don't use simpleString if there are no new lines. 6902490 [Michael Armbrust] fix boolean logic evaluation 4d5eba7 [Michael Armbrust] add more dsl for expression arithmetic and boolean logic 8b2a2ee [Michael Armbrust] more tests passing! ad1f3b4 [Michael Armbrust] toString for null literals a5c0a1b [Michael Armbrust] more test harness improvements: * regex whitelist * side by side answer comparison (still needs formatting work) 60ec19d [Michael Armbrust] initial support for udfs c45b440 [Michael Armbrust] support for is (not) null and boolean logic 7f4a1dc [Michael Armbrust] add NoRelation logical operator 72e183b [Michael Armbrust] support for null values in tree node args. ad596d2 [Michael Armbrust] add sc to Union's otherCopyArgs e5c9d1a [Michael Armbrust] use nonEmpty dcc4fe1 [Michael Armbrust] support for src1 test table. c78b587 [Michael Armbrust] casting. 75c3f3f [Michael Armbrust] add support for logging with scalalogging. da2c011 [Michael Armbrust] make it more obvious when results are being truncated. 96b73ba [Michael Armbrust] more docs in TestShark 18524fd [Michael Armbrust] add method to SharkSqlQuery for directly executing the same query on hive. e6d063b [Michael Armbrust] more join tests. 664c1c3 [Michael Armbrust] make parsing of function names case insensitive. 0967d4e [Michael Armbrust] fix hardcoded path to hiveDevHome. 1a6db68 [Michael Armbrust] spelling 7638cb4 [Michael Armbrust] simple join execution with dsl tests. no hive tests yes. 859d4c9 [Michael Armbrust] better argString printing of nested trees. fc53615 [Michael Armbrust] add same instance comparisons for tree nodes. a026e6b [Michael Armbrust] move out hive specific operators fff4d1c [Michael Armbrust] add simple query execution debugging e2120ab [Michael Armbrust] sorting for strings da06eb6 [Michael Armbrust] Parsing for sortby and joins 9eb5c5e [Michael Armbrust] override equality in Attribute references to compare exprId. 8eb2460 [Michael Armbrust] add system property to override whitelist. 88124bb [Michael Armbrust] make strategy evaluation lazy. 74a3a21 [Michael Armbrust] implement outputSet d25b171 [Michael Armbrust] Add AND and OR expressions 67f0a4a [Michael Armbrust] dsl improvements: string to attribute, subquery, unionAll 12acf0a [Michael Armbrust] add .DS_Store for macs f7da6ce [Michael Armbrust] add agg with grouping expr in select test 36805b3 [Michael Armbrust] pull out and improve aggregation 75613e1 [Michael Armbrust] better evaluations failure messages. 4789a35 [Michael Armbrust] weaken type since its hard to create pure references. e89dd36 [Michael Armbrust] no newline for online trees d0590d4 [Michael Armbrust] include stack trace for catalyst failures. 081c0d9 [Michael Armbrust] more generic computation of agg functions. 31af3a0 [Michael Armbrust] fail when clauses are unhandeled in the parser ecd45b2 [Michael Armbrust] Add more passing tests. 97d5419 [Michael Armbrust] fix alignment. 565cc13 [Michael Armbrust] make the canary query optional. a95e65c [Michael Armbrust] support for resolving qualified attribute references. e1dfa0c [Michael Armbrust] better error reporting for comparison tests when hive works but catalyst fails. 4640a0b [Michael Armbrust] handle test tables when database is specified. bef12e3 [Michael Armbrust] Add Subquery node and trivial optimizer to remove it after analysis. fec5158 [Michael Armbrust] add hive / idea files to .gitignore 3f97ffe [Michael Armbrust] Rename Hive => HiveQl 656b836 [Michael Armbrust] Support for parsing select clause aliases. 3ca7414 [Michael Armbrust] StopAfter needs otherCopyArgs. 3ffde66 [Michael Armbrust] When the child of an alias is unresolved it should return an unresolved attribute instead of throwing an exception. 8cbef8a [Michael Armbrust] spelling aa8c37c [Michael Armbrust] Better toString for SortOrder 1bb8b45 [Michael Armbrust] fix error message for UnresolvedExceptions a2e0327 [Michael Armbrust] add a bunch of tests. 4a3e1ea [Michael Armbrust] docs and use shark for data loading. 339bb8f [Michael Armbrust] better docs, Not support 1d7b2d9 [Michael Armbrust] Add NaN conversions. 46a2534 [Michael Armbrust] only run canary query on failure. 8996066 [Michael Armbrust] remove protected from makeCopy 53bcf41 [Michael Armbrust] testing improvements: * reset hive vars * delete indexes and tables * delete database * reset to use default database * record tests that pass 04a372a [Michael Armbrust] add a flag for running all tests. 3b2235b [Michael Armbrust] More general implementation of arithmetic. edd7795 [Michael Armbrust] More testing improvements: * Check that results match for native commands * Ensure explain commands can be planned * Cache hive "golden" results da6c577 [Michael Armbrust] add string <==> file utility functions. 3adf5ca [Michael Armbrust] Initial support for groupBy and count. 7bcd8a4 [Michael Armbrust] Improvements to comparison tests: * Sort answer when query doesn't contain an order by. * Display null values the same as Hive. * Print full query results in easy to read format when they differ. a52e7c9 [Michael Armbrust] Transform children that are present in sequences of the product. d66ba7e [Michael Armbrust] drop printlns. 88f2efd [Michael Armbrust] Add sum / count distinct expressions. 05adedc [Michael Armbrust] rewrite relative paths when loading data in TestShark 07784b3 [Michael Armbrust] add support for rewriting paths and running 'set' commands. b8a9910 [Michael Armbrust] quote tests passing. 8e5e267 [Michael Armbrust] handle aliased select expressions. 4286a96 [Michael Armbrust] drop debugging println ac34aeb [Michael Armbrust] proof of concept for hive ast transformations. 2238b00 [Michael Armbrust] better error when makeCopy functions fails due to incorrect arguments ff1eab8 [Michael Armbrust] start trying to make insert into hive table more general. 74a6337 [Michael Armbrust] use fastEquals when doing transformations. 1184a23 [Michael Armbrust] add native test for escapes. b972b18 [Michael Armbrust] create BaseRelation class fa6bce9 [Michael Armbrust] implement union 6391a87 [Michael Armbrust] count aggregate. d47c317 [Michael Armbrust] add unary minus, more tests passing. c7114e4 [Michael Armbrust] first draft of star expansion. 044c43d [Michael Armbrust] better support for numeric literal parsing. 1d0f072 [Michael Armbrust] use native drop table as it doesn't appear to fail when the "table" is actually a view. 61503c5 [Michael Armbrust] add cached toRdd 2036883 [Michael Armbrust] skip explain queries when testing. ebac4b1 [Michael Armbrust] fix bug in sort reference calculation ca0dee0 [Michael Armbrust] docs. 1ee0471 [Michael Armbrust] string literal parsing. 357278b [Michael Armbrust] add limit support 9b3e479 [Michael Armbrust] creation of string literals. 02efa30 [Michael Armbrust] alias evaluation cb68b33 [Michael Armbrust] parsing for random sample in hive ql. 126dd36 [Michael Armbrust] include query plans in failure output bb59ae9 [Michael Armbrust] doc fixes 7e68286 [Michael Armbrust] fix confusing naming 768bb25 [Michael Armbrust] handle errors in shark query toString 829c3ce [Michael Armbrust] Auto loading of test data on demand. Add reset method to test shark. Make test shark a singleton to avoid weirdness with the hive megastore. ad02e41 [Michael Armbrust] comment jdo dependency 7bc89fe [Michael Armbrust] add collect to TreeNode. 438cf74 [Michael Armbrust] create explicit treeString function in addition to toString override. docs. 09679ee [Michael Armbrust] fix bug in TreeNode foreach 2930b27 [Michael Armbrust] more specific name for del query tests. 8842549 [Michael Armbrust] docs. da81f81 [Michael Armbrust] Implementation and tests for simple AVG query in Hive SQL. a8969b9 [Michael Armbrust] Factor out hive query comparison test framework. 1a7efb0 [Michael Armbrust] specialize spark aggregate for global aggregations. a36dd9a [Michael Armbrust] evaluation for other > data types. cae729b [Michael Armbrust] remove unnecessary lazy vals. d8e12af [Michael Armbrust] docs 3a60d67 [Michael Armbrust] implement average, placeholder for count f05c106 [Michael Armbrust] checkAnswer handles single row results. 2730534 [Michael Armbrust] implement inputSet a9aa79d [Michael Armbrust] debugging for sort exec 8bec3c9 [Michael Armbrust] better tree makeCopy when there are two constructors. 554b4b2 [Michael Armbrust] BoundAttribute pretty printing. 754f5fa [Michael Armbrust] dsl for setting nullability a206d7a [Michael Armbrust] clean up query tests. 84ad6ef [Michael Armbrust] better sort implementation and tests. de24923 [Michael Armbrust] add double type. 9611a2c [Michael Armbrust] literal creation for doubles. 7358313 [Michael Armbrust] sort order returns child type. b544715 [Michael Armbrust] implement eval for rand, and > for doubles 7013bad [Michael Armbrust] asc, desc should work for expressions and unresolved attributes (symbols) 1c1a35e [Michael Armbrust] add simple Rand expression. 3ca51de [Michael Armbrust] add orderBy to dsl 7ae41ab [Michael Armbrust] more literal implicit conversions b18b675 [Michael Armbrust] First cut at native query tests for shark. d392e29 [Michael Armbrust] add toRdd implicit conversion for logical plans in TestShark. 5eac895 [Michael Armbrust] better error when descending is specified. 2b16f86 [Michael Armbrust] add todo e527bb8 [Michael Armbrust] remove arguments to binary predicate constructor as they seem to break serialization 9dde3c8 [Michael Armbrust] add project and filter operations. ad9037b [Michael Armbrust] Add support for local relations. 6227143 [Michael Armbrust] evaluation of Equals. 7526290 [Michael Armbrust] BoundReference should also be an Attribute. bd33e26 [Michael Armbrust] more documentation 5de0ea3 [Michael Armbrust] Move all shark specific into a separate package. Lots of documentation improvements. 0ae292b [Michael Armbrust] implement calculation of sort expressions. 9fd5011 [Michael Armbrust] First cut at expression evaluation. 6259e3a [Michael Armbrust] cleanup 787e5a2 [Michael Armbrust] use fastEquals f90da36 [Michael Armbrust] better printing of optimization exceptions b05dd67 [Michael Armbrust] Application of rules to fixed point. bb2e0db [Michael Armbrust] pretty print for literals. 1ec3287 [Michael Armbrust] Add extractor for IntegerLiterals. d3a3687 [Michael Armbrust] add fastEquals 2b4935b [Michael Armbrust] set sbt.version explicitly 46dfd7f [Michael Armbrust] first cut at checking answer for HiveCompatability tests. c79f2fd [Michael Armbrust] insert operator should return an empty rdd. 14c22ec [Michael Armbrust] implement sorting when the sort expression is the first attribute of the input. ae7b4c3 [Michael Armbrust] remove implicit dependencies. now compiles without copying things into lib/ manually. 84082f9 [Michael Armbrust] add sbt binaries and scripts 15371a8 [Michael Armbrust] First draft of simple Hive DDL parser. 063bf44 [Michael Armbrust] Periods should end all comments. e1f7f4c [Michael Armbrust] Remove "NativePlaceholder" hack. ed3633e [Michael Armbrust] start consolidating Hive/Shark specific code. first hive compatibility test case passing! b34a770 [Michael Armbrust] Add data sink strategy, make strategy application a little more robust. e7174ec [Michael Armbrust] fix schema, add docs, make helper method protected. 26f410a [Michael Armbrust] physical traits should extend PhysicalPlan. dc72469 [Michael Armbrust] beginning of hive compatibility testing framework. 0763490 [Michael Armbrust] support for hive native command pass-through. d8a924f [Michael Armbrust] scaladoc 29a7163 [Michael Armbrust] Insert into hive table physical operator. 633cebc [Michael Armbrust] better error message when there is no appropriate planning strategy. 59ac444 [Michael Armbrust] add unary expression 3aa1b28 [Michael Armbrust] support for table names in the form 'database.tableName' 665f7d0 [Michael Armbrust] add logical nodes for hive data sinks. 64d2923 [Michael Armbrust] Add classes for representing sorts. f72b7ce [Michael Armbrust] first trivial end to end query execution. 5c7d244 [Michael Armbrust] first draft of references implementation. 7bff274 [Michael Armbrust] point at new shark. c7cd57f [Michael Armbrust] docs for util function. 910811c [Michael Armbrust] check each item of the sequence ef21a0b [Michael Armbrust] line up comments. 4b765d5 [Michael Armbrust] docs, drop println 6f9bafd [Michael Armbrust] empty output for unresolved relation to avoid exception in resolution. a703c49 [Michael Armbrust] this order works better until fixed point is implemented. ec1d7c0 [Michael Armbrust] Simple attribute resolution. 069df02 [Michael Armbrust] parsing binary predicates a1cf754 [Michael Armbrust] add joins and equality. 3f5bc98 [Michael Armbrust] add optiq to sbt. 54f3460 [Michael Armbrust] initial optiq parsing. d9161ce [Michael Armbrust] add join operator 1e423eb [Michael Armbrust] placeholders in LogicalPlan, docs 24ef6fb [Michael Armbrust] toString for alias. ae7d776 [Michael Armbrust] add nullability changing function d49dc02 [Michael Armbrust] scaladoc for named exprs 7c45dd7 [Michael Armbrust] pretty printing of trees. 78e34bf [Michael Armbrust] simple git ignore. 7ba19be [Michael Armbrust] First draft of interface to hive metastore. 7e7acf0 [Michael Armbrust] physical placeholder. 1c11136 [Michael Armbrust] first draft of error handling / plans for debugging. 3766a41 [Michael Armbrust] rearrange utility functions. 7fb3d5e [Michael Armbrust] docs and equality improvements. 45da47b [Michael Armbrust] flesh out plans and expressions a little. first cut at named expressions. 002d4d4 [Michael Armbrust] default to no alias. be25003 [Michael Armbrust] add repl initialization to sbt. 0608a00 [Michael Armbrust] tighten public interface a1a8b38 [Michael Armbrust] test that ids don't change for no-op transforms. daa71ca [Michael Armbrust] foreach, maps, and scaladoc 6a158cb [Michael Armbrust] simple transform working. db0299f [Michael Armbrust] basic analysis of relations minus transform function. f74c4ee [Michael Armbrust] parsing a simple query. 08e4f57 [Michael Armbrust] upgrade scala include shark. d3c6404 [Michael Armbrust] initial commit --- assembly/pom.xml | 5 + bin/compute-classpath.sh | 31 +- dev/download-hive-tests.sh | 4 + dev/run-tests | 3 + docs/_layouts/global.html | 9 + docs/_plugins/copy_api_dirs.rb | 13 + docs/index.md | 1 + docs/sql-programming-guide.md | 143 +++ examples/pom.xml | 6 + examples/src/main/resources/kv1.txt | 500 +++++++++ examples/src/main/resources/people.txt | 3 + .../spark/sql/examples/HiveFromSpark.scala | 64 ++ .../spark/sql/examples/RDDRelation.scala | 71 ++ graphx/pom.xml | 2 +- pom.xml | 5 + project/SparkBuild.scala | 76 +- sql/README.md | 80 ++ sql/catalyst/pom.xml | 66 ++ .../apache/spark/sql/catalyst/SqlParser.scala | 328 ++++++ .../sql/catalyst/analysis/Analyzer.scala | 185 ++++ .../spark/sql/catalyst/analysis/Catalog.scala | 107 ++ .../catalyst/analysis/FunctionRegistry.scala | 37 + .../catalyst/analysis/HiveTypeCoercion.scala | 275 +++++ .../analysis/MultiInstanceRelation.scala | 54 + .../spark/sql/catalyst/analysis/package.scala | 25 + .../sql/catalyst/analysis/unresolved.scala | 109 ++ .../spark/sql/catalyst/dsl/package.scala | 224 ++++ .../spark/sql/catalyst/errors/package.scala | 57 ++ .../catalyst/expressions/BoundAttribute.scala | 83 ++ .../spark/sql/catalyst/expressions/Cast.scala | 79 ++ .../sql/catalyst/expressions/Expression.scala | 196 ++++ .../sql/catalyst/expressions/Projection.scala | 127 +++ .../spark/sql/catalyst/expressions/Rand.scala | 29 + .../spark/sql/catalyst/expressions/Row.scala | 214 ++++ .../sql/catalyst/expressions/ScalaUdf.scala | 41 + .../sql/catalyst/expressions/SortOrder.scala | 34 + .../catalyst/expressions/WrapDynamic.scala | 49 + .../sql/catalyst/expressions/aggregates.scala | 265 +++++ .../sql/catalyst/expressions/arithmetic.scala | 89 ++ .../catalyst/expressions/complexTypes.scala | 96 ++ .../sql/catalyst/expressions/generators.scala | 116 +++ .../sql/catalyst/expressions/literals.scala | 73 ++ .../expressions/namedExpressions.scala | 156 +++ .../catalyst/expressions/nullFunctions.scala | 75 ++ .../sql/catalyst/expressions/package.scala | 51 + .../sql/catalyst/expressions/predicates.scala | 213 ++++ .../expressions/stringOperations.scala | 29 + .../sql/catalyst/optimizer/Optimizer.scala | 167 +++ .../sql/catalyst/planning/QueryPlanner.scala | 64 ++ .../spark/sql/catalyst/planning/package.scala | 24 + .../sql/catalyst/planning/patterns.scala | 117 +++ .../spark/sql/catalyst/plans/QueryPlan.scala | 128 +++ .../spark/sql/catalyst/plans/joinTypes.scala | 26 + .../catalyst/plans/logical/BaseRelation.scala | 28 + .../catalyst/plans/logical/LogicalPlan.scala | 132 +++ .../plans/logical/ScriptTransformation.scala | 38 + .../catalyst/plans/logical/TestRelation.scala | 47 + .../plans/logical/basicOperators.scala | 158 +++ .../catalyst/plans/logical/partitioning.scala | 46 + .../spark/sql/catalyst/plans/package.scala | 25 + .../plans/physical/partitioning.scala | 201 ++++ .../spark/sql/catalyst/rules/Rule.scala | 33 + .../sql/catalyst/rules/RuleExecutor.scala | 79 ++ .../spark/sql/catalyst/rules/package.scala | 24 + .../spark/sql/catalyst/trees/TreeNode.scala | 364 +++++++ .../spark/sql/catalyst/trees/package.scala | 38 + .../spark/sql/catalyst/types/dataTypes.scala | 137 +++ .../spark/sql/catalyst/types/package.scala | 24 + .../spark/sql/catalyst/util/package.scala | 122 +++ .../scala/org/apache/spark/sql/package.scala | 49 + .../spark/sql/catalyst/AnalysisSuite.scala | 41 + .../sql/catalyst/DistributionSuite.scala | 175 ++++ .../catalyst/ExpressionEvaluationSuite.scala | 115 +++ .../sql/catalyst/HiveTypeCoercionSuite.scala | 74 ++ .../sql/catalyst/RuleExecutorSuite.scala | 57 ++ .../spark/sql/catalyst/TreeNodeSuite.scala | 81 ++ .../optimizer/ConstantFoldingSuite.scala | 176 ++++ .../optimizer/FilterPushdownSuite.scala | 222 ++++ .../catalyst/optimizer/OptimizerTest.scala | 44 + sql/core/pom.xml | 76 ++ .../rdd/PartitionLocalRDDFunctions.scala | 99 ++ .../org/apache/spark/sql/SQLContext.scala | 174 ++++ .../org/apache/spark/sql/SchemaRDD.scala | 342 +++++++ .../apache/spark/sql/execution/Exchange.scala | 170 +++ .../apache/spark/sql/execution/Generate.scala | 69 ++ .../execution/QueryExecutionException.scala | 21 + .../spark/sql/execution/SparkPlan.scala | 89 ++ .../spark/sql/execution/SparkStrategies.scala | 229 +++++ .../spark/sql/execution/aggregates.scala | 117 +++ .../spark/sql/execution/basicOperators.scala | 137 +++ .../apache/spark/sql/execution/debug.scala | 46 + .../apache/spark/sql/execution/joins.scala | 158 +++ .../apache/spark/sql/execution/package.scala | 29 + .../spark/sql/parquet/ParquetRelation.scala | 276 +++++ .../sql/parquet/ParquetTableOperations.scala | 212 ++++ .../sql/parquet/ParquetTableSupport.scala | 220 ++++ .../spark/sql/parquet/ParquetTestData.scala | 103 ++ .../spark/sql/test/TestSQLContext.scala | 24 + sql/core/src/test/resources/log4j.properties | 52 + .../org/apache/spark/sql/DslQuerySuite.scala | 201 ++++ .../org/apache/spark/sql/PlannerSuite.scala | 62 ++ .../org/apache/spark/sql/QueryTest.scala | 75 ++ .../org/apache/spark/sql/SQLQuerySuite.scala | 211 ++++ .../scala/org/apache/spark/sql/TestData.scala | 72 ++ .../scala/org/apache/spark/sql/TgfSuite.scala | 71 ++ .../spark/sql/parquet/ParquetQuerySuite.scala | 126 +++ sql/hive/pom.xml | 81 ++ .../hadoop/mapred/SparkHadoopWriter.scala | 198 ++++ .../apache/spark/sql/hive/HiveContext.scala | 287 ++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 246 +++++ .../org/apache/spark/sql/hive/HiveQl.scala | 966 ++++++++++++++++++ .../spark/sql/hive/HiveStrategies.scala | 164 +++ .../spark/sql/hive/ScriptTransformation.scala | 76 ++ .../apache/spark/sql/hive/TableReader.scala | 243 +++++ .../org/apache/spark/sql/hive/TestHive.scala | 341 +++++++ .../apache/spark/sql/hive/hiveOperators.scala | 356 +++++++ .../org/apache/spark/sql/hive/hiveUdfs.scala | 467 +++++++++ sql/hive/src/test/resources/log4j.properties | 47 + .../execution/BigDataBenchmarkSuite.scala | 126 +++ .../hive/execution/ConcurrentHiveSuite.scala | 38 + .../hive/execution/HiveComparisonTest.scala | 379 +++++++ .../execution/HiveCompatibilitySuite.scala | 708 +++++++++++++ .../hive/execution/HiveQueryFileTest.scala | 70 ++ .../sql/hive/execution/HiveQuerySuite.scala | 144 +++ .../hive/execution/HiveResolutionSuite.scala | 65 ++ .../sql/hive/execution/HiveSerDeSuite.scala | 33 + .../execution/HiveTypeCoercionSuite.scala | 32 + .../sql/hive/execution/PruningSuite.scala | 164 +++ .../spark/sql/parquet/HiveParquetSuite.scala | 161 +++ .../spark/streaming/StreamingContext.scala | 4 +- .../streaming/receivers/ActorReceiver.scala | 2 +- 131 files changed, 16398 insertions(+), 15 deletions(-) create mode 100755 dev/download-hive-tests.sh create mode 100644 docs/sql-programming-guide.md create mode 100644 examples/src/main/resources/kv1.txt create mode 100644 examples/src/main/resources/people.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/README.md create mode 100644 sql/catalyst/pom.xml create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala 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 create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala 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 create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala 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 create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala create mode 100644 sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala create mode 100644 sql/core/pom.xml create mode 100644 sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/package.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala create mode 100644 sql/core/src/test/resources/log4j.properties create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/TestData.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala create mode 100644 sql/hive/pom.xml create mode 100644 sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala create mode 100644 sql/hive/src/test/resources/log4j.properties create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala 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 create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala 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/bin/compute-classpath.sh b/bin/compute-classpath.sh index 278969655de48..0624117f4964f 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -33,23 +33,43 @@ fi # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" +# 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 entirely 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 @@ -62,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/dev/download-hive-tests.sh b/dev/download-hive-tests.sh new file mode 100755 index 0000000000000..6c412a8493c18 --- /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 zxf hiveTests.tgz \ No newline at end of file diff --git a/dev/run-tests b/dev/run-tests index cf0b940c09a81..b62a25f42599e 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -21,6 +21,9 @@ FWDIR="$(cd `dirname $0`/..; pwd)" cd $FWDIR +# Download Hive Compatability Files +dev/download-hive-tests.sh + # Remove work directory rm -rf ./work diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 49fd78ca98655..5d4dbb7a9c3a9 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -66,6 +66,7 @@
  • Spark in Python
  • Spark Streaming
  • +
  • Spark SQL
  • MLlib (Machine Learning)
  • Bagel (Pregel on Spark)
  • GraphX (Graph Processing)
  • @@ -79,6 +80,14 @@
  • Spark Core for Python
  • Spark Streaming
  • +
  • MLlib (Machine Learning)
  • Bagel (Pregel on Spark)
  • GraphX (Graph Processing)
  • diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 44d64057f4fb3..2245bcbc70f1e 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -22,6 +22,7 @@ # Build Scaladoc for Java/Scala core_projects = ["core", "examples", "repl", "bagel", "graphx", "streaming", "mllib"] external_projects = ["flume", "kafka", "mqtt", "twitter", "zeromq"] + sql_projects = ["catalyst", "core", "hive"] projects = core_projects + external_projects.map { |project_name| "external/" + project_name } @@ -49,6 +50,18 @@ cp_r(source + "/.", dest) end + sql_projects.each do |project_name| + source = "../sql/" + project_name + "/target/scala-2.10/api/" + dest = "api/sql/" + project_name + + puts "echo making directory " + dest + mkdir_p dest + + # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. + puts "cp -r " + source + "/. " + dest + cp_r(source + "/.", dest) + end + # Build Epydoc for Python puts "Moving to python directory and building epydoc." cd("../python") diff --git a/docs/index.md b/docs/index.md index 23311101e1712..7a13fa9a9a2b6 100644 --- a/docs/index.md +++ b/docs/index.md @@ -78,6 +78,7 @@ For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to bui * [Java Programming Guide](java-programming-guide.html): using Spark from Java * [Python Programming Guide](python-programming-guide.html): using Spark from Python * [Spark Streaming](streaming-programming-guide.html): Spark's API for processing data streams +* [Spark SQL](sql-programming-guide.html): Support for running relational queries on Spark * [MLlib (Machine Learning)](mllib-guide.html): Spark's built-in machine learning library * [Bagel (Pregel on Spark)](bagel-programming-guide.html): simple graph processing model * [GraphX (Graphs on Spark)](graphx-programming-guide.html): Spark's new API for graphs diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md new file mode 100644 index 0000000000000..b6f21a5dc62c3 --- /dev/null +++ b/docs/sql-programming-guide.md @@ -0,0 +1,143 @@ +--- +layout: global +title: Spark SQL Programming Guide +--- +**Spark SQL is currently an Alpha component. Therefore, the APIs may be changed in future releases.** + +* This will become a table of contents (this text will be scraped). +{:toc} + +# Overview +Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using +Spark. At the core of this component is a new type of RDD, +[SchemaRDD](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed +[Row](api/sql/catalyst/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with +a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table +in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet +file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). + +**All of the examples on this page use sample data included in the Spark distribution and can be run in the spark-shell.** + +*************************************************************************************************** + +# Getting Started + +The entry point into all relational functionallity in Spark is the +[SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its +decendents. To create a basic SQLContext, all you need is a SparkContext. + +{% highlight scala %} +val sc: SparkContext // An existing SparkContext. +val sqlContext = new org.apache.spark.sql.SQLContext(sc) + +// Importing the SQL context gives access to all the public SQL functions and implicit conversions. +import sqlContext._ +{% endhighlight %} + +## Running SQL on RDDs +One type of table that is supported by Spark SQL is an RDD of Scala case classetees. The case class +defines the schema of the table. The names of the arguments to the case class are read using +reflection and become the names of the columns. Case classes can also be nested or contain complex +types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be +registered as a table. Tables can used in subsequent SQL statements. + +{% highlight scala %} +val sqlContext = new org.apache.spark.sql.SQLContext(sc) +import sqlContext._ + +// Define the schema using a case class. +case class Person(name: String, age: Int) + +// Create an RDD of Person objects and register it as a table. +val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Person(p(0), p(1).trim.toInt)) +people.registerAsTable("people") + +// SQL statements can be run by using the sql methods provided by sqlContext. +val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + +// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The columns of a row in the result can be accessed by ordinal. +teenagers.map(t => "Name: " + t(0)).collect().foreach(println) +{% endhighlight %} + +**Note that Spark SQL currently uses a very basic SQL parser, and the keywords are case sensitive.** +Users that want a more complete dialect of SQL should look at the HiveQL support provided by +`HiveContext`. + +## Using Parquet + +Parquet is a columnar format that is supported by many other data processing systems. Spark SQL +provides support for both reading and writing parquet files that automatically preserves the schema +of the original data. Using the data from the above example: + +{% highlight scala %} +val sqlContext = new org.apache.spark.sql.SQLContext(sc) +import sqlContext._ + +val people: RDD[Person] // An RDD of case class objects, from the previous example. + +// The RDD is implicitly converted to a SchemaRDD, allowing it to be stored using parquet. +people.saveAsParquetFile("people.parquet") + +// Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. +// The result of loading a parquet file is also a SchemaRDD. +val parquetFile = sqlContext.parquetFile("people.parquet") + +//Parquet files can also be registered as tables and then used in SQL statements. +parquetFile.registerAsTable("parquetFile") +val teenagers = sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") +teenagers.collect().foreach(println) +{% endhighlight %} + +## Writing Language-Integrated Relational Queries + +Spark SQL also supports a domain specific language for writing queries. Once again, +using the data from the above examples: + +{% highlight scala %} +val sqlContext = new org.apache.spark.sql.SQLContext(sc) +import sqlContext._ +val people: RDD[Person] // An RDD of case class objects, from the first example. + +// The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' +val teenagers = people.where('age >= 10).where('age <= 19).select('name) +{% endhighlight %} + +The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers +prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are +evaluated by the SQL execution engine. A full list of the functions supported can be found in the +[ScalaDoc](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). + + + +# Hive Support + +Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). +However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. +In order to use Hive you must first run '`sbt/sbt hive/assembly`'. This command builds a new assembly +jar that includes Hive. When this jar is present, Spark will use the Hive +assembly instead of the normal Spark assembly. Note that this Hive assembly jar must also be present +on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries +(SerDes) in order to acccess data stored in Hive. + +Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. + +When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and +adds support for finding tables in in the MetaStore and writing queries using HiveQL. Users who do +not have an existing Hive deployment can also experiment with the `LocalHiveContext`, +which is similar to `HiveContext`, but creates a local copy of the `metastore` and `warehouse` +automatically. + +{% highlight scala %} +val sc: SparkContext // An existing SparkContext. +val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc) + +// Importing the SQL context gives access to all the public SQL functions and implicit conversions. +import hiveContext._ + +sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + +// Queries are expressed in HiveQL +sql("SELECT key, value FROM src").collect().foreach(println) +{% endhighlight %} \ No newline at end of file diff --git a/examples/pom.xml b/examples/pom.xml index 382a38d9400b9..a5569ff5e71f3 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -70,6 +70,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/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/resources/people.txt b/examples/src/main/resources/people.txt new file mode 100644 index 0000000000000..3bcace4a44c23 --- /dev/null +++ b/examples/src/main/resources/people.txt @@ -0,0 +1,3 @@ +Michael, 29 +Andy, 30 +Justin, 19 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..abcc1f04d4279 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala @@ -0,0 +1,64 @@ +/* + * 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.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..8210ad977f066 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/sql/examples/RDDRelation.scala @@ -0,0 +1,71 @@ +/* + * 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.examples + +import org.apache.spark.SparkContext +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. +case class Record(key: Int, value: String) + +object RDDRelation { + def main(args: Array[String]) { + val sc = new SparkContext("local", "RDDRelation") + val sqlContext = new SQLContext(sc) + + // 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"))) + // 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) + + // Queries can also be written using a LINQ-like Scala DSL. + rdd.where('key === 1).orderBy('value.asc).select('key).collect().foreach(println) + + // Write out an RDD as a parquet file. + rdd.saveAsParquetFile("pair.parquet") + + // Read in parquet file. Parquet files are self-describing so the schmema is preserved. + val parquetFile = sqlContext.parquetFile("pair.parquet") + + // Queries can be run using the DSL on parequet files just like the original RDD. + parquetFile.where('key === 1).select('value as 'a).collect().foreach(println) + + // These files can also be registered as tables. + parquetFile.registerAsTable("parquetFile") + sql("SELECT * FROM parquetFile").collect().foreach(println) + } +} diff --git a/graphx/pom.xml b/graphx/pom.xml index 894a7c2641e39..5a5022916d234 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -29,7 +29,7 @@ spark-graphx_2.10 jar Spark Project GraphX - http://spark-project.org/ + http://spark.apache.org/ diff --git a/pom.xml b/pom.xml index 524e5daff5388..9db34a01ba87e 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/project/SparkBuild.scala b/project/SparkBuild.scala index aff191c98b409..e4ad65912523e 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) + .dependsOn(core, graphx, bagel, mllib, sql) lazy val tools = Project("tools", file("tools"), settings = toolsSettings) dependsOn(core) dependsOn(streaming) @@ -60,12 +60,19 @@ 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) dependsOn(core) + + lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core, catalyst) + + // 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") @@ -131,13 +138,13 @@ 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, java8Tests and examples belong to packageProjects - lazy val packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx) ++ 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", @@ -164,7 +171,7 @@ object SparkBuild extends Build { // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), // Remove certain packages from Scaladoc - scalacOptions in (Compile,doc) := Seq("-skip-packages", Seq( + scalacOptions in (Compile,doc) := Seq("-groups", "-skip-packages", Seq( "akka", "org.apache.spark.network", "org.apache.spark.deploy", @@ -362,6 +369,61 @@ object SparkBuild extends Build { ) ) + 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, + libraryDependencies ++= Seq( + "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" + ) + ) + + // 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", + "org.apache.hive" % "hive-exec" % "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 + // only for this subproject. + scalacOptions <<= scalacOptions map { currentOpts: Seq[String] => + currentOpts.filterNot(_ == "-deprecation") + }, + 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.hive._ + |import org.apache.spark.sql.hive.TestHive._ + |import org.apache.spark.sql.parquet.ParquetTestData""".stripMargin + ) + def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", libraryDependencies ++= Seq( diff --git a/sql/README.md b/sql/README.md new file mode 100644 index 0000000000000..4192fecb92fb0 --- /dev/null +++ b/sql/README.md @@ -0,0 +1,80 @@ +Spark SQL +========= + +This module provides support for executing relational queries expressed in either SQL or a LINQ-like Scala DSL. + +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 Support (sql/hive) - Includes 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. + + +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" +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 hive/console`. From here you can execute queries and inspect the various stages of query optimization. + +```scala +catalyst$ sbt/sbt hive/console + +[info] Starting scala interpreter... +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.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. + +scala> val query = sql("SELECT * FROM (SELECT * FROM src) a") +query: org.apache.spark.sql.ExecutedQuery = +SELECT * FROM (SELECT * FROM src) a +=== Query Plan === +Project [key#6:0.0,value#7:0.1] + HiveTableScan [key#6,value#7], (MetastoreRelation default, src, None), 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.logicalPlan +res1: catalyst.plans.logical.LogicalPlan = +Project {key#0,value#1} + Project {key#0,value#1} + MetastoreRelation default, src, None + + +scala> query.logicalPlan transform { + | case Project(projectList, child) if projectList == child.output => child + | } +res2: catalyst.plans.logical.LogicalPlan = +Project {key#0,value#1} + MetastoreRelation default, src, None +``` diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml new file mode 100644 index 0000000000000..740f1fdc83299 --- /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.apache.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/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..d3b1070a58d58 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SqlParser.scala @@ -0,0 +1,328 @@ +/* + * 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.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 => Average(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/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala new file mode 100644 index 0000000000000..9eb992ee58f0a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala @@ -0,0 +1,185 @@ +/* + * 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._ +import plans.logical._ +import rules._ + +/** + * 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 [[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 { + + // TODO: pass this in as a parameter. + val fixedPoint = FixedPoint(100) + + val batches: Seq[Batch] = Seq( + Batch("MultiInstanceRelations", Once, + NewRelationInstances), + Batch("CaseInsensitiveAttributeReferences", Once, + (if (caseSensitive) Nil else LowercaseAttributeReferences :: Nil) : _*), + Batch("Resolution", fixedPoint, + ResolveReferences :: + ResolveRelations :: + NewRelationInstances :: + ImplicitGenerate :: + StarExpansion :: + ResolveFunctions :: + GlobalAggregates :: + typeCoercionRules :_*) + ) + + /** + * Replaces [[UnresolvedRelation]]s with concrete relations from the catalog. + */ + object ResolveRelations extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case UnresolvedRelation(databaseName, name, alias) => + catalog.lookupRelation(databaseName, name, alias) + } + } + + /** + * 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(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)) + case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase) + case Alias(c, name) => Alias(c, name.toLowerCase)() + } + } + } + + /** + * 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 { + case q: LogicalPlan if q.childrenResolved => + 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 + } + } + } + + /** + * Replaces [[UnresolvedFunction]]s with concrete [[expressions.Expression Expressions]]. + */ + object ResolveFunctions extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case q: LogicalPlan => + q transformExpressions { + case u @ UnresolvedFunction(name, children) if u.childrenResolved => + registry.lookupFunction(name, children) + } + } + } + + /** + * Turns projections that contain aggregate expressions into aggregations. + */ + object GlobalAggregates extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case Project(projectList, child) if containsAggregates(projectList) => + Aggregate(Nil, projectList, child) + } + + def containsAggregates(exprs: Seq[Expression]): Boolean = { + exprs.foreach(_.foreach { + case agg: AggregateExpression => return true + case _ => + }) + false + } + } + + /** + * 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 { + case Project(Seq(Alias(g: Generator, _)), child) => + Generate(g, join = false, outer = false, None, child) + } + } + + /** + * Expands any references to [[Star]] (*) in project operators. + */ + object StarExpansion extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Wait until children are resolved + case p: LogicalPlan if !p.childrenResolved => p + // If the projection list contains Stars, expand it. + case p @ Project(projectList, child) if containsStar(projectList) => + Project( + projectList.flatMap { + case s: Star => s.expand(child.output) + case o => o :: Nil + }, + child) + case t: ScriptTransformation 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( + 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[Expression]): Boolean = + exprs.collect { case _: Star => true }.nonEmpty + } +} 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 new file mode 100644 index 0000000000000..71e4dcdb1567c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala @@ -0,0 +1,107 @@ +/* + * 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 plans.logical.{LogicalPlan, Subquery} +import scala.collection.mutable + +/** + * An interface for looking up relations by name. Used by an [[Analyzer]]. + */ +trait Catalog { + def lookupRelation( + 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) + } +} + +/** + * 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... + val overrides = new mutable.HashMap[(Option[String],String), LogicalPlan]() + + abstract override def lookupRelation( + databaseName: Option[String], + tableName: String, + alias: Option[String] = None): LogicalPlan = { + + 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)) + } + + override def registerTable( + databaseName: Option[String], + tableName: String, + plan: LogicalPlan): Unit = { + 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. + */ +object EmptyCatalog extends Catalog { + def lookupRelation( + databaseName: Option[String], + tableName: String, + 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/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/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala new file mode 100644 index 0000000000000..a0105cd7cf4af --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -0,0 +1,275 @@ +/* + * 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._ +import plans.logical._ +import rules._ +import types._ + +/** + * 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 + + // 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 { + // 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] { + // 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, DecimalType) + // 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] = { + // 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 => + Average(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/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala new file mode 100644 index 0000000000000..fe18cc466fbff --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala @@ -0,0 +1,54 @@ +/* + * 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.catalyst +package analysis + +import plans.logical.LogicalPlan +import rules._ + +/** + * A trait that should be mixed into query operators where an single instance might appear multiple + * times in a logical query plan. It is invalid to have multiple copies of the same attribute + * produced by distinct operators in a query tree as this breaks the gurantee that expression + * ids, which are used to differentate attributes, are unique. + * + * Before analysis, all operators that include this trait will be asked to produce a new version + * of itself with globally unique expression ids. + */ +trait MultiInstanceRelation { + def newInstance: this.type +} + +/** + * If any MultiInstanceRelation 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 NewRelationInstances extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = { + val localRelations = plan collect { case l: MultiInstanceRelation => l} + val multiAppearance = localRelations + .groupBy(identity[MultiInstanceRelation]) + .filter { case (_, ls) => ls.size > 1 } + .map(_._1) + .toSet + + plan transform { + case l: MultiInstanceRelation if multiAppearance contains l => l.newInstance + } + } +} 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/sql/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 new file mode 100644 index 0000000000000..2ed2af1352a16 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala @@ -0,0 +1,109 @@ +/* + * 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._ +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. + */ +class UnresolvedException[TreeType <: TreeNode[_]](tree: TreeType, function: String) extends + 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], + tableName: String, + alias: Option[String] = None) extends BaseRelation { + def output = Nil + override lazy val resolved = false +} + +/** + * Holds the name of an attribute that has yet to be resolved. + */ +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") + 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" +} + +case class UnresolvedFunction(name: String, children: Seq[Expression]) extends Expression { + def exprId = throw new UnresolvedException(this, "exprId") + def dataType = throw new UnresolvedException(this, "dataType") + 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 + 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 ...". + * + * @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], + 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") + def nullable = throw new UnresolvedException(this, "nullable") + def qualifiers = throw new UnresolvedException(this, "qualifiers") + override lazy val resolved = false + + def newInstance = this + def withQualifiers(newQualifiers: Seq[String]) = this + + def expand(input: Seq[Attribute]): Seq[NamedExpression] = { + 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 { + case (n: NamedExpression, _) => n + case (e, originalAttribute) => + Alias(e, originalAttribute.name)(qualifiers = originalAttribute.qualifiers) + } + mappedAttributes + } + + override def toString = table.map(_ + ".").getOrElse("") + "*" +} 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 new file mode 100644 index 0000000000000..cd8de9d52fb74 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala @@ -0,0 +1,224 @@ +/* + * 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 scala.language.implicitConversions +import scala.reflect.runtime.universe.TypeTag + +import analysis.UnresolvedAttribute +import expressions._ +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. + * + * {{{ + * scala> import catalyst.dsl._ + * + * // Standard operators are added to expressions. + * scala> Literal(1) + Literal(1) + * res1: catalyst.expressions.Add = (1 + 1) + * + * // There is a conversion from 'symbols to unresolved attributes. + * scala> 'a.attr + * res2: catalyst.analysis.UnresolvedAttribute = 'a + * + * // These unresolved attributes can be used to create more complicated expressions. + * scala> 'a === 'b + * res3: catalyst.expressions.Equals = ('a = 'b) + * + * // SQL verbs can be used to construct logical query plans. + * scala> TestRelation('key.int, 'value.string).where('key === 1).select('value).analyze + * res4: catalyst.plans.logical.LogicalPlan = + * Project {value#1} + * Filter (key#0 = 1) + * TestRelation {key#0,value#1} + * }}} + */ +package object dsl { + 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) = 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) = Not(Equals(expr, other)) + + def asc = SortOrder(expr, Ascending) + def desc = SortOrder(expr, Descending) + + def as(s: Symbol) = Alias(expr, s.name)() + } + + 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 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 + + 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 string */ + def string = AttributeReference(s, StringType, nullable = false)() + } + + implicit class DslAttribute(a: AttributeReference) { + def notNull = a.withNullability(false) + def nullable = a.withNullability(true) + + // Protobuf terminology + def required = a.withNullability(false) + } + } + + + object expressions extends ExpressionConversions // scalastyle:ignore + + 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(logicalPlan, otherPlan, joinType, condition) + + 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, logicalPlan) + } + + def subquery(alias: Symbol) = Subquery(alias.name, logicalPlan) + + def unionAll(otherPlan: LogicalPlan) = Union(logicalPlan, otherPlan) + + def sfilter[T1](arg1: Symbol)(udf: (T1) => Boolean) = + Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan) + + def sfilter(dynamicUdf: (DynamicRow) => Boolean) = + 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, logicalPlan) + + def generate( + generator: Generator, + join: Boolean = false, + outer: Boolean = false, + alias: Option[String] = None) = + Generate(generator, join, outer, None, logicalPlan) + + def insertInto(tableName: String, overwrite: Boolean = false) = + InsertIntoTable( + analysis.UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite) + + def analyze = analysis.SimpleAnalyzer(logicalPlan) + } + + object plans { // scalastyle:ignore + implicit class DslLogicalPlan(val logicalPlan: LogicalPlan) extends LogicalPlanFunctions { + def writeToFile(path: String) = WriteToFile(path, logicalPlan) + } + } +} diff --git a/sql/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 new file mode 100644 index 0000000000000..c253587f67c4e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala @@ -0,0 +1,57 @@ +/* + * 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 trees._ + +/** + * Functions for attaching and retrieving trees that are associated with errors. + */ +package object errors { + + class TreeNodeException[TreeType <: TreeNode[_]] + (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 + s"${super.getMessage}, tree:${if (treeString contains "\n") "\n" else " "}$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 TreeNodeException(tree, msg, e) + } + } + + /** + * 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/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 new file mode 100644 index 0000000000000..3b6bac16ff4e6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala @@ -0,0 +1,83 @@ +/* + * 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 rules._ +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. + */ +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(ordinal, baseReference.newInstance) + def withQualifiers(newQualifiers: Seq[String]) = + BoundReference(ordinal, baseReference.withQualifiers(newQualifiers)) + + override def toString = s"$baseReference:$ordinal" + + override def apply(input: Row): Any = input(ordinal) +} + +class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] { + import BindReferences._ + + def apply(plan: TreeNode): TreeNode = { + plan.transform { + case leafNode if leafNode.children.isEmpty => leafNode + 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[Attribute]): Expression = { + expression.transform { case a: AttributeReference => + attachTree(a, "Binding attribute") { + 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) + } + } + } + } +} 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..608656d3a90f9 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala @@ -0,0 +1,79 @@ +/* + * 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._ + +/** 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/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala new file mode 100644 index 0000000000000..78aaaeebbd631 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala @@ -0,0 +1,196 @@ +/* + * 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 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 + * 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.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 + 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}") + + /** + * 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 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) + if(evalE1 == null) { + null + } else { + 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") + } + } + } + } + + @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) + if(evalE1 == null) { + null + } else { + 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") + } + } + } + } + + @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) + if(evalE1 == null) { + null + } else { + 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") + } + } + } + } +} + +abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { + self: Product => + + def symbol: String + + override def foldable = left.foldable && right.foldable + + def references = left.references ++ right.references + + override def toString = s"($left $symbol $right)" +} + +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 +} 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..8c407d2fddb42 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala @@ -0,0 +1,127 @@ +/* + * 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.catalyst +package expressions + +/** + * 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]) = + 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) + } +} + +/** + * 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. + */ +class JoinedRow extends 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 + 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) + + 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/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/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 new file mode 100644 index 0000000000000..352967546827d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala @@ -0,0 +1,214 @@ +/* + * 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._ + +/** + * 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. + */ +trait 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 getFloat(i: Int): Float + 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: Float) + def setString(ordinal: Int, value: String) + + /** + * 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 +} + +/** + * 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 = throw new UnsupportedOperationException + + def iterator = Iterator.empty + def length = 0 + def isNullAt(i: Int): Boolean = throw new UnsupportedOperationException + + 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 + def getString(i: Int): String = throw new UnsupportedOperationException + + def copy() = this +} + +/** + * 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(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 + + 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 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] + } + + def getString(i: Int): String = { + if (values(i) == null) sys.error("Failed to check null bit for primitive String value.") + values(i).asInstanceOf[String] + } + + def copy() = this +} + +class GenericMutableRow(size: Int) extends GenericRow(size) with MutableRow { + /** No-arg constructor for serialization. */ + def this() = this(0) + + 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: 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 } + + 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 + while (i < ordering.size) { + val order = ordering(i) + val left = order.child.apply(a) + val right = order.child.apply(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 + } + return 0 + } +} 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..a3c7ca1acdb0c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala @@ -0,0 +1,41 @@ +/* + * 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 { + + type EvaluatedType = Any + + def references = children.flatMap(_.references).toSet + def nullable = true + + override def apply(input: Row): Any = { + children.size match { + 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/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..2ad8d6f31d2e6 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala @@ -0,0 +1,49 @@ +/* + * 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 { + type EvaluatedType = DynamicRow + + def nullable = false + def references = children.toSet + def dataType = DynamicType + + 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: Array[Any]) + extends GenericRow(values) with Dynamic { + + def selectDynamic(attributeName: String): String = { + val ordinal = schema.indexWhere(_.name == attributeName) + values(ordinal).toString + } +} 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 new file mode 100644 index 0000000000000..2287a849e6831 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala @@ -0,0 +1,265 @@ +/* + * 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._ + +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 +} + +/** + * 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 [[NamedExpression]]s 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 one specific result. + */ +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 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) +} + +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) + } + + override def newInstance = new CountFunction(child, this) +} + +case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { + def children = expressions + def references = expressions.flatMap(_.references).toSet + 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] { + def references = child.references + def nullable = false + def dataType = DoubleType + override def toString = s"AVG($child)" + + override 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) + } + + override def newInstance = new AverageFunction(child, this) +} + +case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { + def references = child.references + def nullable = false + def dataType = child.dataType + override def toString = s"SUM($child)" + + override def asPartial: SplitEvaluation = { + val partialSum = Alias(Sum(child), "PartialSum")() + SplitEvaluation( + Sum(partialSum.toAttribute), + partialSum :: Nil) + } + + override def newInstance = new SumFunction(child, this) +} + +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)" + + override def newInstance = new SumDistinctFunction(child, this) +} + +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)" + + override def asPartial: SplitEvaluation = { + val partialFirst = Alias(First(child), "PartialFirst")() + SplitEvaluation( + 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. + + private var count: Long = _ + 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.apply(EmptyRow).asInstanceOf[Double] / count.toDouble + + def update(input: Row): Unit = { + count += 1 + sum.update(addFunction, input) + } +} + +case class CountFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + var count: Int = _ + + def update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.apply(input)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { + count += 1 + } + } + + override def apply(input: Row): Any = count +} + +case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { + def this() = this(null, null) // Required for serialization. + + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) + + private val addFunction = Add(sum, expr) + + 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) + extends AggregateFunction { + + def this() = this(null, null) // Required for serialization. + + val seen = new scala.collection.mutable.HashSet[Any]() + + def update(input: Row): Unit = { + val evaluatedExpr = expr.apply(input) + if (evaluatedExpr != null) { + seen += evaluatedExpr + } + } + + override def apply(input: Row): 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 update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.apply(input)) + if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { + seen += evaluatedExpr + } + } + + override def apply(input: Row): 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 update(input: Row): Unit = { + if (result == null) { + 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 new file mode 100644 index 0000000000000..db235645cd85c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala @@ -0,0 +1,89 @@ +/* + * 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.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 = + 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}") + } + left.dataType + } +} + +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 new file mode 100644 index 0000000000000..d3feb6c461b55 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala @@ -0,0 +1,96 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql +package catalyst +package expressions + +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 + 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] || 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 || o < 0) { + 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 + + 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 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 new file mode 100644 index 0000000000000..c367de2a3e570 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala @@ -0,0 +1,116 @@ +/* + * 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._ + +/** + * 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 => + + type EvaluatedType = TraversableOnce[Row] + + 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(attributeNames: Seq[String], child: Expression) + extends Generator with trees.UnaryNode[Expression] { + + override lazy val resolved = + child.resolved && + (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) + + 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() = + 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)() + } + } + + override def apply(input: Row): TraversableOnce[Row] = { + child.dataType match { + case ArrayType(_) => + val inputArray = child.apply(input).asInstanceOf[Seq[Any]] + if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Array(v))) + case MapType(_, _) => + val inputMap = child.apply(input).asInstanceOf[Map[Any,Any]] + if (inputMap == null) Nil else inputMap.map { case (k,v) => new GenericRow(Array(k,v)) } + } + } + + override def toString() = s"explode($child)" +} 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 new file mode 100644 index 0000000000000..229d8f7f7b763 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala @@ -0,0 +1,73 @@ +/* + * 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._ + +object Literal { + def apply(v: Any): Literal = v match { + 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) + case s: Short => Literal(s, ShortType) + case s: String => Literal(s, StringType) + case b: Boolean => Literal(b, BooleanType) + case null => Literal(null, NullType) + } +} + +/** + * Extractor for retrieving Int literals. + */ +object IntegerLiteral { + def unapply(a: Any): Option[Int] = a match { + case Literal(a: Int, IntegerType) => Some(a) + case _ => None + } +} + +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 new file mode 100644 index 0000000000000..0a06e85325a22 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala @@ -0,0 +1,156 @@ +/* + * 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.analysis.UnresolvedAttribute +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 qualifiers: Seq[String] + + def toAttribute: Attribute + + protected def typeSuffix = + if (resolved) { + dataType match { + case LongType => "L" + case _ => "" + } + } else { + "" + } +} + +abstract class Attribute extends NamedExpression { + self: Product => + + def withQualifiers(newQualifiers: Seq[String]): Attribute + + def references = Set(this) + def toAttribute = this + def newInstance: Attribute +} + +/** + * 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 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. + */ +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 + + def toAttribute = { + if (resolved) { + AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers) + } else { + UnresolvedAttribute(name) + } + } + + override def toString: String = s"$child AS $name#${exprId.id}$typeSuffix" + + override protected final def otherCopyArgs = exprId :: qualifiers :: Nil +} + +/** + * 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 [[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. + * @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) + extends Attribute with trees.LeafNode[Expression] { + + override def equals(other: Any) = other match { + case ar: AttributeReference => exprId == ar.exprId && dataType == ar.dataType + 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) + + /** + * Returns a copy of this [[AttributeReference]] with changed nullability. + */ + def withNullability(newNullability: Boolean) = { + if (nullable == newNullability) { + this + } else { + AttributeReference(name, dataType, newNullability)(exprId, qualifiers) + } + } + + /** + * Returns a copy of this [[AttributeReference]] with new qualifiers. + */ + def withQualifiers(newQualifiers: Seq[String]) = { + if (newQualifiers == qualifiers) { + this + } else { + AttributeReference(name, dataType, nullable)(exprId, newQualifiers) + } + } + + override def toString: String = s"$name#${exprId.id}$typeSuffix" +} 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 new file mode 100644 index 0000000000000..e869a4d9b0a52 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala @@ -0,0 +1,75 @@ +/* + * 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.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) + + def references = children.flatMap(_.references).toSet + // 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) + + 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.") + } + + 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 + } +} diff --git a/sql/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 new file mode 100644 index 0000000000000..76554e160b1d1 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala @@ -0,0 +1,51 @@ +/* + * 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 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. + * + * ==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`. + * + * 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. + */ +package object expressions 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 new file mode 100644 index 0000000000000..561396eb4373c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala @@ -0,0 +1,213 @@ +/* + * 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._ +import catalyst.analysis.UnresolvedException + +trait Predicate extends Expression { + self: Product => + + def dataType = BooleanType + + type EvaluatedType = Any +} + +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 => + def nullable = left.nullable || right.nullable +} + +case class Not(child: Expression) extends Predicate with trees.UnaryNode[Expression] { + def references = child.references + 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 + } + } +} + +/** + * 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("(", ",", ")")}" + + 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 { + self: Product => +} + +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 = ">=" + 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) + extends Expression { + + def children = predicate :: trueValue :: falseValue :: Nil + 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) { + throw new UnresolvedException( + this, + s"Can not resolve due to differing types ${trueValue.dataType}, ${falseValue.dataType}") + } + 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/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/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 new file mode 100644 index 0000000000000..4db280317345a --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala @@ -0,0 +1,167 @@ +/* + * 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 + +import catalyst.expressions._ +import catalyst.plans.logical._ +import catalyst.rules._ +import catalyst.types.BooleanType +import catalyst.plans.Inner + +object Optimizer extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Subqueries", Once, + EliminateSubqueries) :: + Batch("ConstantFolding", Once, + ConstantFolding, + BooleanSimplification, + SimplifyCasts) :: + Batch("Filter Pushdown", Once, + EliminateSubqueries, + CombineFilters, + PushPredicateThroughProject, + PushPredicateThroughInnerJoin) :: Nil +} + +/** + * 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 { + case Subquery(_, child) => child + } +} + +/** + * 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 { + case q: LogicalPlan => q transformExpressionsDown { + // Skip redundant folding of literals. + case l: Literal => l + case e if e.foldable => Literal(e.apply(null), e.dataType) + } + } +} + +/** + * 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) => { + (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 (_, _) => and + } + } + 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 (_, _) => or + } + } + } + } +} + +/** + * 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 { + case ff@Filter(fc, nf@Filter(nc, grandChild)) => Filter(And(nc, fc), grandChild) + } +} + +/** + * 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. + */ +object PushPredicateThroughProject extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + 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)) + } + + // + def replaceAlias(condition: Expression, sourceAliases: Map[Attribute, Expression]): Expression = { + condition transform { + case a: AttributeReference => sourceAliases.getOrElse(a, a) + } + } +} + +/** + * 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 { + 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) + Join(newLeft, newRight, Inner, joinConditions.reduceLeftOption(And)) + } +} + +/** + * 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 { + case Cast(e, dataType) if e.dataType == dataType => e + } +} diff --git a/sql/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 new file mode 100644 index 0000000000000..22f8ea005baf1 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala @@ -0,0 +1,64 @@ +/* + * 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 + + +import plans.logical.LogicalPlan +import trees._ + +/** + * 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. + * + * @tparam PhysicalPlan The type of physical plan produced by this [[QueryPlanner]] + */ +abstract class QueryPlanner[PhysicalPlan <: TreeNode[PhysicalPlan]] { + /** A list of execution strategies that can be used by the planner */ + def strategies: Seq[Strategy] + + /** + * 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 + * filled in automatically by the QueryPlanner using the other execution strategies that are + * available. + */ + protected def planLater(plan: LogicalPlan) = apply(plan).next() + + def apply(plan: LogicalPlan): Iterator[PhysicalPlan] = { + // Obviously a lot to do here still... + val iter = strategies.view.flatMap(_(plan)).toIterator + assert(iter.hasNext, s"No plan for $plan") + iter + } +} 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/sql/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 new file mode 100644 index 0000000000000..613b028ca8409 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala @@ -0,0 +1,117 @@ +/* + * 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 + +import scala.annotation.tailrec + +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. + * + * @return A tuple containing a sequence of conjunctive predicates that should be used to filter the + * output and a relational operator. + */ +object FilteredOperation extends PredicateHelper { + type ReturnType = (Seq[Expression], LogicalPlan) + + def unapply(plan: LogicalPlan): Option[ReturnType] = Some(collectFilters(Nil, plan)) + + @tailrec + private def collectFilters(filters: Seq[Expression], plan: LogicalPlan): ReturnType = plan match { + case Filter(condition, child) => + collectFilters(filters ++ splitConjunctivePredicates(condition), child) + case other => (filters, other) + } +} + +/** + * 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] = { + 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 + * examples for alias in-lining/substitution. Before: + * {{{ + * 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: + * {{{ + * SELECT key AS c1 FROM t1 WHERE key > 10 + * SELECT key AS c2 FROM t1 WHERE key > 10 + * }}} + */ + def collectProjectsAndFilters(plan: LogicalPlan): + (Option[Seq[NamedExpression]], Seq[Expression], LogicalPlan, Map[Attribute, Expression]) = + plan match { + case Project(fields, child) => + 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 (fields, filters, other, aliases) = collectProjectsAndFilters(child) + val substitutedCondition = substitute(aliases)(condition) + (fields, filters ++ splitConjunctivePredicates(substitutedCondition), other, aliases) + + case other => + (None, Nil, other, Map.empty) + } + + 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. + */ +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 + } +} diff --git a/sql/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 new file mode 100644 index 0000000000000..20f230c5c4f2c --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala @@ -0,0 +1,128 @@ +/* + * 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 + +import catalyst.expressions.{SortOrder, Attribute, Expression} +import catalyst.trees._ + +abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { + self: PlanType with Product => + + 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. + * 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. + */ + def transformExpressionsDown(rule: PartialFunction[Expression, Expression]): this.type = { + var changed = false + + @inline def transformExpressionDown(e: Expression) = { + val newE = e.transformDown(rule) + if (newE.id != e.id && newE != e) { + changed = true + newE + } else { + e + } + } + + val newArgs = productIterator.map { + case e: Expression => transformExpressionDown(e) + case Some(e: Expression) => Some(transformExpressionDown(e)) + case m: Map[_,_] => m + case seq: Traversable[_] => seq.map { + case e: Expression => transformExpressionDown(e) + case other => other + } + case other: AnyRef => other + }.toArray + + if (changed) makeCopy(newArgs) else this + } + + /** + * 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 transformExpressionsUp(rule: PartialFunction[Expression, Expression]): this.type = { + var changed = false + + @inline def transformExpressionUp(e: Expression) = { + val newE = e.transformUp(rule) + if (newE.id != e.id && newE != e) { + changed = true + newE + } else { + e + } + } + + val newArgs = productIterator.map { + case e: Expression => transformExpressionUp(e) + case Some(e: Expression) => Some(transformExpressionUp(e)) + case m: Map[_,_] => m + case seq: Traversable[_] => seq.map { + case e: Expression => transformExpressionUp(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 { + 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: Traversable[_] => seq.flatMap { + case e: Expression => e :: Nil + case other => Nil + } + case other => Nil + }.toSeq + } +} diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala new file mode 100644 index 0000000000000..9f2283ad4346e --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.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/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..48ff45c3d3ebb --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala @@ -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. + */ + +package org.apache.spark.sql +package catalyst +package plans +package logical + +abstract class BaseRelation extends LeafNode { + self: Product => + + def tableName: String + def isPartitioned: Boolean = false +} diff --git a/sql/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 new file mode 100644 index 0000000000000..bc7b6871df524 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala @@ -0,0 +1,132 @@ +/* + * 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 catalyst.expressions._ +import catalyst.errors._ +import catalyst.types.StructType + +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 this node takes as + * input from its children. + */ + 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 + * and false if it is still contains any unresolved placeholders. Implementations of LogicalPlan + * can override this (e.g. [[catalyst.analysis.UnresolvedRelation UnresolvedRelation]] should + * return `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) + + /** + * 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("\\.") + // 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 + } + + options.distinct match { + 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( + this, s"Ambiguous references to $name: ${ambiguousReferences.mkString(",")}") + } + } +} + +/** + * A logical plan node with no children. + */ +abstract class LeafNode extends LogicalPlan with trees.LeafNode[LogicalPlan] { + self: Product => + + // Leaf nodes by definition cannot reference any input attributes. + 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. + */ +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 => +} 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/sql/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 new file mode 100644 index 0000000000000..b5905a4456d12 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala @@ -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. + */ + +package org.apache.spark.sql +package catalyst +package plans +package logical + +import expressions._ +import rules._ + +object LocalRelation { + def apply(output: Attribute*) = + new LocalRelation(output) +} + +case class LocalRelation(output: Seq[Attribute], data: Seq[Product] = Nil) + extends LeafNode with analysis.MultiInstanceRelation { + + // 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. + */ + override final def newInstance: this.type = { + LocalRelation(output.map(_.newInstance), data).asInstanceOf[this.type] + } + + override protected def stringArgs = Iterator(output) +} 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 new file mode 100644 index 0000000000000..8e98aab736432 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala @@ -0,0 +1,158 @@ +/* + * 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._ + +case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { + def output = projectList.map(_.toAttribute) + 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, 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 { + + 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 +} + +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).exists { case (l,r) => l.dataType != r.dataType } + + def references = Set.empty +} + +case class Join( + left: LogicalPlan, + right: LogicalPlan, + joinType: JoinType, + condition: Option[Expression]) extends BinaryNode { + + def references = condition.map(_.references).getOrElse(Set.empty) + def output = left.output ++ right.output +} + +case class InsertIntoTable( + table: BaseRelation, + partition: Map[String, Option[String]], + child: LogicalPlan, + overwrite: Boolean) + 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 = child.output + + override lazy val resolved = childrenResolved && child.output.zip(table.output).forall { + case (childAttr, tableAttr) => childAttr.dataType == tableAttr.dataType + } +} + +case class InsertIntoCreatedTable( + databaseName: Option[String], + tableName: String, + child: LogicalPlan) extends UnaryNode { + def references = Set.empty + def output = child.output +} + +case class WriteToFile( + path: String, + child: LogicalPlan) 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 +} + +case class Aggregate( + groupingExpressions: Seq[Expression], + aggregateExpressions: Seq[NamedExpression], + child: LogicalPlan) + extends UnaryNode { + + def output = aggregateExpressions.map(_.toAttribute) + def references = child.references +} + +case class StopAfter(limit: Expression, child: LogicalPlan) extends UnaryNode { + def output = child.output + def references = limit.references +} + +case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { + def output = child.output.map(_.withQualifiers(alias :: Nil)) + def references = Set.empty +} + +case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, 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 +} 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/sql/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 new file mode 100644 index 0000000000000..2d8f3ad33517d --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala @@ -0,0 +1,201 @@ +/* + * 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 physical + +import expressions._ +import types._ + +/** + * 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. + * - 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 + +/** + * Represents a distribution where no promises are made about co-location of data. + */ +case object UnspecifiedDistribution extends Distribution + +/** + * Represents a distribution that only has a single partition and all tuples of the dataset + * are co-located. + */ +case object AllTuples extends Distribution + +/** + * 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 { + 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.") +} + +/** + * 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 { + 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 +} + +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 + * 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 + + /** + * Returns true iff all distribution guarantees made by this partitioning can also be made + * 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(numPartitions: Int) extends Partitioning { + override def satisfies(required: Distribution): Boolean = required match { + case UnspecifiedDistribution => true + case _ => false + } + + override def compatibleWith(other: Partitioning): Boolean = other match { + case UnknownPartitioning(_) => true + case _ => false + } +} + +case object SinglePartition extends Partitioning { + val numPartitions = 1 + + override def satisfies(required: Distribution): Boolean = true + + override def compatibleWith(other: Partitioning) = other match { + case SinglePartition => true + case _ => false + } +} + +case object BroadcastPartitioning extends Partitioning { + val numPartitions = 1 + + override def satisfies(required: Distribution): Boolean = true + + override def compatibleWith(other: Partitioning) = other match { + case SinglePartition => true + case _ => false + } +} + +/** + * 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], numPartitions: Int) + extends Expression + with Partitioning { + + def children = expressions + def references = expressions.flatMap(_.references).toSet + def nullable = false + def dataType = IntegerType + + lazy val clusteringSet = expressions.toSet + + override def satisfies(required: Distribution): Boolean = required match { + case UnspecifiedDistribution => true + case ClusteredDistribution(requiredClustering) => + clusteringSet.subsetOf(requiredClustering.toSet) + case _ => false + } + + override def compatibleWith(other: Partitioning) = other match { + case BroadcastPartitioning => true + case h: HashPartitioning if h == this => true + case _ => false + } +} + +/** + * 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], numPartitions: Int) + extends Expression + with Partitioning { + + def children = ordering + def references = ordering.flatMap(_.references).toSet + def nullable = false + def dataType = IntegerType + + lazy val clusteringSet = ordering.map(_.child).toSet + + override def satisfies(required: Distribution): Boolean = required match { + case UnspecifiedDistribution => 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 + } + + override def compatibleWith(other: Partitioning) = other match { + case BroadcastPartitioning => true + case r: RangePartitioning if r == this => true + case _ => false + } +} 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/sql/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 new file mode 100644 index 0000000000000..68ae30cde1c57 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/RuleExecutor.scala @@ -0,0 +1,79 @@ +/* + * 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._ +import util._ + +abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { + + /** + * 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 + + batches.foreach { batch => + var iteration = 1 + var lastPlan = curPlan + curPlan = batch.rules.foldLeft(curPlan) { case (curPlan, rule) => rule(curPlan) } + + // 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) => + 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 + } + } + + curPlan + } +} 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/sql/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 new file mode 100644 index 0000000000000..76ede87e4e6c0 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala @@ -0,0 +1,364 @@ +/* + * 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 + +import errors._ + +object TreeNode { + private val currentId = new java.util.concurrent.atomic.AtomicLong + protected def nextId() = currentId.getAndIncrement() +} + +/** Used by [[TreeNode.getNodeNumbered]] when traversing the tree for a given number */ +private class MutableInt(var i: Int) + +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` can be used to differentiate distinct but structurally + * identical branches of a tree. + */ + val id = TreeNode.nextId() + + /** + * 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 + } + + /** + * 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 + */ + def fastEquals(other: TreeNode[_]): Boolean = { + sameInstance(other) || this == other + } + + /** + * 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 = { + f(this) + children.foreach(_.foreach(f)) + } + + /** + * 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. + */ + def map[A](f: BaseType => A): Seq[A] = { + val ret = new collection.mutable.ArrayBuffer[A]() + foreach(ret += f(_)) + 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. + */ + def collect[B](pf: PartialFunction[BaseType, B]): Seq[B] = { + val ret = new collection.mutable.ArrayBuffer[B]() + val lifted = pf.lift + foreach(node => lifted(node).foreach(ret.+=)) + 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. + * 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 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]) + // Check if unchanged and then possibly return old copy to avoid gc churn. + if (this fastEquals afterRule) { + transformChildrenDown(rule) + } else { + afterRule.transformChildrenDown(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 used to transform this nodes children + */ + 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].transformDown(rule) + if (!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + 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)) { + changed = true + newChild + } else { + arg + } + case other => other + } + 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 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 = { + val afterRuleOnChildren = transformChildrenUp(rule); + if (this fastEquals afterRuleOnChildren) { + rule.applyOrElse(this, identity[BaseType]) + } else { + rule.applyOrElse(afterRuleOnChildren, identity[BaseType]) + } + } + + 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].transformUp(rule) + if (!(newChild fastEquals arg)) { + changed = true + newChild + } else { + arg + } + 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)) { + 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 + * @return + */ + protected def otherCopyArgs: Seq[AnyRef] = Nil + + /** + * 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. + */ + def makeCopy(newArgs: Array[AnyRef]): this.type = attachTree(this, "makeCopy") { + try { + 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 TreeNodeException( + this, s"Failed to copy node. Is otherCopyArgs specified correctly for $nodeName?") + } + } + + /** 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 + case tn: TreeNode[_] if tn.toString contains "\n" => s"(${tn.simpleString})" :: Nil + case seq: Seq[_] => seq.mkString("[", ",", "]") :: Nil + case set: Set[_] => set.mkString("{", ",", "}") :: Nil + case other => other :: Nil + }.mkString(", ") + + /** String representation of this node without any children */ + def simpleString = s"$nodeName $argString" + + 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 [[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") + + /** + * 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 getNodeNumbered(number: MutableInt): BaseType = { + if (number.i < 0) { + null.asInstanceOf[BaseType] + } else if (number.i == 0) { + this + } else { + number.i -= 1 + children.map(_.getNodeNumbered(number)).find(_ != null).getOrElse(null.asInstanceOf[BaseType]) + } + } + + /** 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) + builder.append("\n") + children.foreach(_.generateTreeString(depth + 1, builder)) + builder + } +} + +/** + * A [[TreeNode]] that has two children, [[left]] and [[right]]. + */ +trait BinaryNode[BaseType <: TreeNode[BaseType]] { + def left: BaseType + def right: 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 +} diff --git a/sql/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 new file mode 100644 index 0000000000000..e2da1d2439bbf --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.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 + +/** + * 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. + *
    • + *
    • 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") +} diff --git a/sql/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 new file mode 100644 index 0000000000000..6eb2b62eccb48 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala @@ -0,0 +1,137 @@ +/* + * 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 + +import expressions.Expression + +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 NullType extends DataType + +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 DataType { + type JvmType = Array[Byte] +} +case object BooleanType extends NativeType { + type JvmType = Boolean + val ordering = implicitly[Ordering[JvmType]] +} + +abstract class NumericType extends NativeType { + // 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] +} + +/** 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]] + 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 */ +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]] + 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 + +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 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/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 new file mode 100644 index 0000000000000..52adea2661363 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala @@ -0,0 +1,122 @@ +/* + * 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 java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, 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() + tempFile + } + + 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) + } + + 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) + out.close() + file + } + + 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(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 + } + } + + 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 + + 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() { + org.apache.log4j.Logger.getLogger(a.getClass.getName).setLevel(org.apache.log4j.Level.DEBUG) + } + } */ +} 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..9ec31689b5098 --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/package.scala @@ -0,0 +1,49 @@ +/* + * 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 + + object Row { + /** + * This method can be used to extract fields from a [[Row]] object in a pattern 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) + } +} 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..1fd0d26b6f8bb --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala @@ -0,0 +1,41 @@ +/* + * 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._ +import dsl.expressions._ + +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/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 new file mode 100644 index 0000000000000..fb25e1c246117 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala @@ -0,0 +1,175 @@ +/* + * 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 org.apache.spark.sql.catalyst.plans.physical._ + +/* Implicit conversions */ +import org.apache.spark.sql.catalyst.dsl.expressions._ + +class DistributionSuite extends FunSuite { + + protected def checkSatisfied( + inputPartitioning: Partitioning, + requiredDistribution: Distribution, + satisfied: Boolean) { + if (inputPartitioning.satisfies(requiredDistribution) != satisfied) + fail( + s""" + |== Input Partitioning == + |$inputPartitioning + |== Required Distribution == + |$requiredDistribution + |== Does input partitioning satisfy required distribution? == + |Expected $satisfied got ${inputPartitioning.satisfies(requiredDistribution)} + """.stripMargin) + } + + test("HashPartitioning is the output partitioning") { + // Cases which do not need an exchange between two data properties. + checkSatisfied( + HashPartitioning(Seq('a, 'b, 'c), 10), + UnspecifiedDistribution, + true) + + checkSatisfied( + HashPartitioning(Seq('a, 'b, 'c), 10), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) + + checkSatisfied( + HashPartitioning(Seq('b, 'c), 10), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) + + checkSatisfied( + SinglePartition, + ClusteredDistribution(Seq('a, 'b, 'c)), + true) + + checkSatisfied( + SinglePartition, + 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), + ClusteredDistribution(Seq('b, 'c)), + false) + + checkSatisfied( + HashPartitioning(Seq('a, 'b, 'c), 10), + ClusteredDistribution(Seq('d, 'e)), + false) + + checkSatisfied( + HashPartitioning(Seq('a, 'b, 'c), 10), + AllTuples, + false) + + checkSatisfied( + HashPartitioning(Seq('a, 'b, 'c), 10), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + checkSatisfied( + HashPartitioning(Seq('b, 'c), 10), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + false) + + // TODO: We should check functional dependencies + /* + checkSatisfied( + ClusteredDistribution(Seq('b)), + ClusteredDistribution(Seq('b + 1)), + true) + */ + } + + 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), + UnspecifiedDistribution, + true) + + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + OrderedDistribution(Seq('a.asc, 'b.asc, 'c.asc)), + true) + + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + OrderedDistribution(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)), + true) + + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + ClusteredDistribution(Seq('a, 'b, 'c)), + true) + + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + ClusteredDistribution(Seq('c, 'b, 'a)), + true) + + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + 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. + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + OrderedDistribution(Seq('a.asc, 'b.desc, 'c.asc)), + false) + + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + OrderedDistribution(Seq('b.asc, 'a.asc)), + false) + + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + ClusteredDistribution(Seq('a, 'b)), + false) + + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + ClusteredDistribution(Seq('c, 'd)), + false) + + checkSatisfied( + RangePartitioning(Seq('a.asc, 'b.asc, 'c.asc), 10), + AllTuples, + false) + } +} \ No newline at end of file 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 new file mode 100644 index 0000000000000..f06618ad11467 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala @@ -0,0 +1,115 @@ +/* + * 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 org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types._ + +/* Implict conversions */ +import org.apache.spark.sql.catalyst.dsl.expressions._ + +class ExpressionEvaluationSuite extends FunSuite { + + test("literals") { + assert((Literal(1) + Literal(1)).apply(null) === 2) + } + + /** + * 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 = expr.apply(null) + if (result != answer) + 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 = expr.apply(null) + if (result != answer) + fail(s"$expr should not evaluate to $result, expected: $answer") + } + } + } +} \ No newline at end of file diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala new file mode 100644 index 0000000000000..f595bf7e44af7 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala @@ -0,0 +1,74 @@ +/* + * 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 catalyst.types._ + + +class HiveTypeCoercionSuite extends FunSuite { + + val rules = new HiveTypeCoercion { } + import rules._ + + 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, + 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") + } + + // 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)) + 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)) + } +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala new file mode 100644 index 0000000000000..ff7c15b7187b7 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala @@ -0,0 +1,57 @@ +/* + * 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 + +import org.scalatest.FunSuite + +import expressions._ +import rules._ + +class RuleExecutorSuite extends FunSuite { + object DecrementLiterals extends Rule[Expression] { + def apply(e: Expression): Expression = e transform { + case IntegerLiteral(i) if i > 0 => Literal(i - 1) + } + } + + test("only once") { + object ApplyOnce extends RuleExecutor[Expression] { + val batches = Batch("once", Once, DecrementLiterals) :: Nil + } + + assert(ApplyOnce(Literal(10)) === Literal(9)) + } + + test("to fixed point") { + object ToFixedPoint extends RuleExecutor[Expression] { + val batches = Batch("fixedPoint", FixedPoint(100), DecrementLiterals) :: Nil + } + + assert(ToFixedPoint(Literal(10)) === Literal(0)) + } + + test("to maxIterations") { + object ToFixedPoint extends RuleExecutor[Expression] { + val batches = Batch("fixedPoint", FixedPoint(10), DecrementLiterals) :: Nil + } + + assert(ToFixedPoint(Literal(100)) === Literal(90)) + } +} \ No newline at end of file diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala new file mode 100644 index 0000000000000..98bb090c29d11 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala @@ -0,0 +1,81 @@ +/* + * 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 + +import scala.collection.mutable.ArrayBuffer + +import expressions._ + +import org.scalatest.{FunSuite} + +class TreeNodeSuite extends FunSuite { + + test("top node changed") { + 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 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 transform { case Literal(5, _) => Literal(1)} + + assert(before === after) + assert(before.map(_.id) === after.map(_.id)) + } + + test("collect") { + val tree = Add(Literal(1), Add(Literal(2), Add(Literal(3), Literal(4)))) + val literals = tree collect {case l: Literal => l} + + 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 transformDown { + 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 transformUp { + 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 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 new file mode 100644 index 0000000000000..7ce42b2b0a057 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala @@ -0,0 +1,176 @@ +/* + * 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 + +import types.IntegerType +import util._ +import plans.logical.{LogicalPlan, LocalRelation} +import rules._ +import expressions._ +import dsl.plans._ +import dsl.expressions._ + +class ConstantFoldingSuite extends OptimizerTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Subqueries", Once, + EliminateSubqueries) :: + Batch("ConstantFolding", Once, + ConstantFolding, + BooleanSimplification) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + + 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) + } + + /** + * Unit tests for constant folding in expressions. + */ + 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")) + + val optimized = Optimize(originalQuery.analyze) + + val correctAnswer = + testRelation + .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("Constant folding test: expressions have attribute references and literals in " + + "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")) + + 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")) + .analyze + + comparePlans(optimized, correctAnswer) + } + + test("Constant folding test: 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 optimized = Optimize(originalQuery.analyze) + + val correctAnswer = + testRelation + .where(('a > 1 || 'b > 1) && ('a < 10 && 'b < 10)) + .analyze + + 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 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 new file mode 100644 index 0000000000000..cd611b3fb3400 --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala @@ -0,0 +1,222 @@ +package org.apache.spark.sql +package catalyst +package optimizer + +import expressions._ +import plans.logical._ +import rules._ +import util._ + +import dsl.plans._ +import dsl.expressions._ + +class FilterPushdownSuite extends OptimizerTest { + + object Optimize extends RuleExecutor[LogicalPlan] { + val batches = + Batch("Subqueries", Once, + EliminateSubqueries) :: + Batch("Filter Pushdown", Once, + EliminateSubqueries, + CombineFilters, + PushPredicateThroughProject, + PushPredicateThroughInnerJoin) :: Nil + } + + val testRelation = LocalRelation('a.int, 'b.int, 'c.int) + + // 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) + .analyze + + 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) + + val originalQuery = { + x.join(y) + .where("x.b".attr === 1) + .where("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: 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") { + val x = testRelation.subquery('x) + val y = testRelation.subquery('y) + + val originalQuery = { + x.join(y, condition = Some("x.b".attr === "y.b".attr)) + } + val optimized = Optimize(originalQuery.analyze) + + comparePlans(optimizer.EliminateSubqueries(originalQuery.analyze), 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)) + } + + 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, condition = Some("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, condition = Some("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)) + } + + 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, condition = Some("x.b".attr === "y.b".attr)), + condition = Some("z.a".attr === "x.b".attr)) + .analyze + + comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) + } +} \ No newline at end of file diff --git a/sql/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 new file mode 100644 index 0000000000000..7b3653d0f998a --- /dev/null +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala @@ -0,0 +1,44 @@ +package org.apache.spark.sql +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 diff --git a/sql/core/pom.xml b/sql/core/pom.xml new file mode 100644 index 0000000000000..e367edfb1f562 --- /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.apache.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/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala b/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala new file mode 100644 index 0000000000000..b8b9e5839d0a5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala @@ -0,0 +1,99 @@ +/* + * 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 + +import scala.reflect._ +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark._ +import org.apache.spark.Aggregator +import org.apache.spark.SparkContext._ +import org.apache.spark.util.collection.AppendOnlyMap + +/** + * Extra functions on RDDs that perform only local operations. These can be used when data has + * already been partitioned correctly. + */ +private[spark] class PartitionLocalRDDFunctions[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, _ ++ _) + val bufs = self.mapPartitionsWithContext((context, iter) => { + new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) + }, 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) + } + } +} + +private[spark] object PartitionLocalRDDFunctions { + implicit def rddToPartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + new PartitionLocalRDDFunctions(rdd) +} + + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala new file mode 100644 index 0000000000000..587cc7487f31c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -0,0 +1,174 @@ +/* + * 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 +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.dsl +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.optimizer.Optimizer +import org.apache.spark.sql.catalyst.planning.QueryPlanner +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, NativeCommand, WriteToFile} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.execution._ + +/** + * ALPHA COMPONENT + * + * The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]] + * objects and the execution of SQL queries. + * + * @groupname userf Spark SQL Functions + * @groupname Ungrouped Support functions for language integrated queries. + */ +class SQLContext(@transient val sparkContext: SparkContext) + extends Logging + with dsl.ExpressionConversions + with Serializable { + + self => + + @transient + protected[sql] lazy val catalog: Catalog = new SimpleCatalog + @transient + protected[sql] lazy val analyzer: Analyzer = + new Analyzer(catalog, EmptyFunctionRegistry, caseSensitive = true) + @transient + protected[sql] val optimizer = Optimizer + @transient + protected[sql] val parser = new catalyst.SqlParser + + 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 } + + /** + * EXPERIMENTAL + * + * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan + * interface is considered internal, and thus not guranteed to be stable. As a result, using + * them directly is not reccomended. + */ + implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = new SchemaRDD(this, plan) + + /** + * Creates a SchemaRDD from an RDD of case classes. + * + * @group userf + */ + implicit def createSchemaRDD[A <: Product: TypeTag](rdd: RDD[A]) = + new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))) + + /** + * Loads a parequet file, returning the result as a [[SchemaRDD]]. + * + * @group userf + */ + def parquetFile(path: String): SchemaRDD = + new SchemaRDD(this, parquet.ParquetRelation("ParquetFile", path)) + + + /** + * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only + * during the lifetime of this instance of SQLContext. + * + * @group userf + */ + def registerRDDAsTable(rdd: SchemaRDD, tableName: String): Unit = { + catalog.registerTable(None, tableName, rdd.logicalPlan) + } + + /** + * Executes a SQL query using Spark, returning the result as a SchemaRDD. + * + * @group userf + */ + def sql(sqlText: String): SchemaRDD = { + val result = new SchemaRDD(this, parseSql(sqlText)) + // We force query optimization to happen right away instead of letting it happen lazily like + // when using the query DSL. This is so DDL commands behave as expected. This is only + // generates the RDD lineage for DML queries, but do not perform any execution. + result.queryExecution.toRdd + result + } + + protected[sql] class SparkPlanner extends SparkStrategies { + val sparkContext = self.sparkContext + + val strategies: Seq[Strategy] = + TopK :: + PartialAggregation :: + SparkEquiInnerJoin :: + BasicOperators :: + CartesianProduct :: + BroadcastNestedLoopJoin :: Nil + } + + @transient + protected[sql] val planner = new SparkPlanner + + /** + * Prepares a planned SparkPlan for execution by binding references to specific ordinals, and + * inserting shuffle operations as needed. + */ + @transient + protected[sql] val prepareForExecution = new 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. + */ + protected 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) + + /** Internal version of the RDD. Avoids copies and has no schema */ + 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)} + |== Optimized Logical Plan + |${stringOrError(optimizedPlan)} + |== 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/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala new file mode 100644 index 0000000000000..91c3aaa2b8d5b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -0,0 +1,342 @@ +/* +* 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.apache.spark.{OneToOneDependency, Dependency, Partition, TaskContext} +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.types.BooleanType + +/** + * ALPHA COMPONENT + * + * An RDD of [[Row]] objects that has an associated schema. In addition to standard RDD functions, + * SchemaRDDs can be used in relational queries, as shown in the examples below. + * + * Importing a SQLContext brings an implicit into scope that automatically converts a standard RDD + * whose elements are scala case classes into a SchemaRDD. This conversion can also be done + * explicitly using the `createSchemaRDD` function on a [[SQLContext]]. + * + * A `SchemaRDD` can also be created by loading data in from external sources, for example, + * by using the `parquetFile` method on [[SQLContext]]. + * + * == SQL Queries == + * A SchemaRDD can be registered as a table in the [[SQLContext]] that was used to create it. Once + * an RDD has been registered as a table, it can be used in the FROM clause of SQL statements. + * + * {{{ + * // 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) + * + * val sc: SparkContext // An existing spark context. + * val sqlContext = new SQLContext(sc) + * + * // 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"))) + * // 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") + * + * val results: SchemaRDD = sql("SELECT * FROM records") + * }}} + * + * == Language Integrated Queries == + * + * {{{ + * + * case class Record(key: Int, value: String) + * + * val sc: SparkContext // An existing spark context. + * val sqlContext = new SQLContext(sc) + * + * // 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, "val_" + i))) + * + * // Example of language integrated queries. + * rdd.where('key === 1).orderBy('value.asc).select('key).collect() + * }}} + * + * @todo There is currently no support for creating SchemaRDDs from either Java or Python RDDs. + * + * @groupname Query Language Integrated Queries + * @groupdesc Query Functions that create new queries from SchemaRDDs. The + * result of all query functions is also a SchemaRDD, allowing multiple operations to be + * chained using a builder pattern. + * @groupprio Query -2 + * @groupname schema SchemaRDD Functions + * @groupprio schema -1 + * @groupname Ungrouped Base RDD Functions + */ +class SchemaRDD( + @transient val sqlContext: SQLContext, + @transient val logicalPlan: LogicalPlan) + extends RDD[Row](sqlContext.sparkContext, Nil) { + + /** + * A lazily computed query execution workflow. All other RDD operations are passed + * through to the RDD that is produced by this workflow. + * + * We want this to be lazy because invoking the whole query optimization pipeline can be + * expensive. + */ + @transient + protected[spark] lazy val queryExecution = sqlContext.executePlan(logicalPlan) + + override def toString = + s"""${super.toString} + |== Query Plan == + |${queryExecution.executedPlan}""".stripMargin.trim + + // ========================================================================================= + // RDD functions: Copy the interal row representation so we present immutable data to users. + // ========================================================================================= + + override def compute(split: Partition, context: TaskContext): Iterator[Row] = + firstParent[Row].compute(split, context).map(_.copy()) + + override def getPartitions: Array[Partition] = firstParent[Row].partitions + + override protected def getDependencies: Seq[Dependency[_]] = + List(new OneToOneDependency(queryExecution.toRdd)) + + + // ======================================================================= + // Query DSL + // ======================================================================= + + /** + * Changes the output of this relation to the given expressions, similar to the `SELECT` clause + * in SQL. + * + * {{{ + * schemaRDD.select('a, 'b + 'c, 'd as 'aliasedName) + * }}} + * + * @param exprs a set of logical expression that will be evaluated for each input row. + * + * @group Query + */ + def select(exprs: NamedExpression*): SchemaRDD = + new SchemaRDD(sqlContext, Project(exprs, logicalPlan)) + + /** + * Filters the ouput, only returning those rows where `condition` evaluates to true. + * + * {{{ + * schemaRDD.where('a === 'b) + * schemaRDD.where('a === 1) + * schemaRDD.where('a + 'b > 10) + * }}} + * + * @group Query + */ + def where(condition: Expression): SchemaRDD = + new SchemaRDD(sqlContext, Filter(condition, logicalPlan)) + + /** + * Performs a relational join on two SchemaRDDs + * + * @param otherPlan the [[SchemaRDD]] that should be joined with this one. + * @param joinType One of `Inner`, `LeftOuter`, `RightOuter`, or `FullOuter`. Defaults to `Inner.` + * @param condition An optional condition for the join operation. This is equivilent to the `ON` + * clause in standard SQL. In the case of `Inner` joins, specifying a + * `condition` is equivilent to adding `where` clauses after the `join`. + * + * @group Query + */ + def join( + otherPlan: SchemaRDD, + joinType: JoinType = Inner, + condition: Option[Expression] = None): SchemaRDD = + new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, condition)) + + /** + * Sorts the results by the given expressions. + * {{{ + * schemaRDD.orderBy('a) + * schemaRDD.orderBy('a, 'b) + * schemaRDD.orderBy('a.asc, 'b.desc) + * }}} + * + * @group Query + */ + def orderBy(sortExprs: SortOrder*): SchemaRDD = + new SchemaRDD(sqlContext, Sort(sortExprs, logicalPlan)) + + /** + * Performs a grouping followed by an aggregation. + * + * {{{ + * schemaRDD.groupBy('year)(Sum('sales) as 'totalSales) + * }}} + * + * @group Query + */ + def groupBy(groupingExprs: Expression*)(aggregateExprs: Expression*): SchemaRDD = { + val aliasedExprs = aggregateExprs.map { + case ne: NamedExpression => ne + case e => Alias(e, e.toString)() + } + new SchemaRDD(sqlContext, Aggregate(groupingExprs, aliasedExprs, logicalPlan)) + } + + /** + * Applies a qualifier to the attributes of this relation. Can be used to disambiguate attributes + * with the same name, for example, when peforming self-joins. + * + * {{{ + * val x = schemaRDD.where('a === 1).subquery('x) + * val y = schemaRDD.where('a === 2).subquery('y) + * x.join(y).where("x.a".attr === "y.a".attr), + * }}} + * + * @group Query + */ + def subquery(alias: Symbol) = + new SchemaRDD(sqlContext, Subquery(alias.name, logicalPlan)) + + /** + * Combines the tuples of two RDDs with the same schema, keeping duplicates. + * + * @group Query + */ + def unionAll(otherPlan: SchemaRDD) = + new SchemaRDD(sqlContext, Union(logicalPlan, otherPlan.logicalPlan)) + + /** + * Filters tuples using a function over the value of the specified column. + * + * {{{ + * schemaRDD.sfilter('a)((a: Int) => ...) + * }}} + * + * @group Query + */ + def where[T1](arg1: Symbol)(udf: (T1) => Boolean) = + new SchemaRDD( + sqlContext, + Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) + + /** + * EXPERIMENTAL + * + * Filters tuples using a function over a `Dynamic` version of a given Row. DynamicRows use + * scala's Dynamic trait to emulate an ORM of in a dynamically typed language. Since the type of + * the column is not known at compile time, all attributes are converted to strings before + * being passed to the function. + * + * {{{ + * schemaRDD.where(r => r.firstName == "Bob" && r.lastName == "Smith") + * }}} + * + * @group Query + */ + def where(dynamicUdf: (DynamicRow) => Boolean) = + new SchemaRDD( + sqlContext, + Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) + + /** + * EXPERIMENTAL + * + * Returns a sampled version of the underlying dataset. + * + * @group Query + */ + def sample( + fraction: Double, + withReplacement: Boolean = true, + seed: Int = (math.random * 1000).toInt) = + new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) + + /** + * EXPERIMENTAL + * + * Applies the given Generator, or table generating function, to this relation. + * + * @param generator A table generating function. The API for such functions is likely to change + * in future releases + * @param join when set to true, each output row of the generator is joined with the input row + * that produced it. + * @param outer when set to true, at least one row will be produced for each input row, similar to + * an `OUTER JOIN` in SQL. When no output rows are produced by the generator for a + * given row, a single row will be output, with `NULL` values for each of the + * generated columns. + * @param alias an optional alias that can be used as qualif for the attributes that are produced + * by this generate operation. + * + * @group Query + */ + def generate( + generator: Generator, + join: Boolean = false, + outer: Boolean = false, + alias: Option[String] = None) = + new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) + + /** + * EXPERIMENTAL + * + * Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is + * no notion of persistent tables, and thus queries that contain this operator will fail to + * optimize. When working with an extension of a SQLContext that has a persistent catalog, such + * as a `HiveContext`, this operation will result in insertions to the table specified. + * + * @group schema + */ + def insertInto(tableName: String, overwrite: Boolean = false) = + new SchemaRDD( + sqlContext, + InsertIntoTable(UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite)) + + /** + * Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that + * are written out using this method can be read back in as a SchemaRDD using the ``function + * + * @group schema + */ + def saveAsParquetFile(path: String): Unit = { + sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd + } + + /** + * Registers this RDD as a temporary table using the given name. The lifetime of this temporary + * table is tied to the [[SQLContext]] that was used to create this SchemaRDD. + * + * @group schema + */ + def registerAsTable(tableName: String): Unit = { + sqlContext.registerRDDAsTable(this, tableName) + } + + /** + * Returns this RDD as a SchemaRDD. + * @group schema + */ + def toSchemaRDD = this + + def analyze = sqlContext.analyzer(logicalPlan) +} 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 new file mode 100644 index 0000000000000..72dc5ec6ad33c --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala @@ -0,0 +1,170 @@ +/* + * 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 + +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._ + +private 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 + } +} + +private 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. + val rdd = child.execute().mapPartitions { iter => + val hashExpressions = new MutableProjection(expressions) + val mutablePair = new MutablePair[Row, Row]() + iter.map(r => mutablePair.update(hashExpressions(r), r)) + } + val part = new HashPartitioner(numPartitions) + val shuffled = new ShuffledRDD[Row, Row, MutablePair[Row, Row]](rdd, part) + shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.map(_._2) + } + case RangePartitioning(sortingExpressions, numPartitions) => { + // TODO: RangePartitioner should take an Ordering. + implicit val ordering = new RowOrdering(sortingExpressions) + + val rdd = child.execute().mapPartitions { iter => + val mutablePair = new MutablePair[Row, Null](null, null) + iter.map(row => mutablePair.update(row, null)) + } + val part = new RangePartitioner(numPartitions, rdd, ascending = true) + val shuffled = new ShuffledRDD[Row, Null, MutablePair[Row, Null]](rdd, part) + shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + + shuffled.map(_._1) + } + case SinglePartition => + child.execute().coalesce(1, true) + + case _ => sys.error(s"Exchange not implemented for $newPartitioning") + // TODO: Handle BroadcastPartitioning. + } + } +} + +/** + * 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[SparkPlan] { + // TODO: Determine the number of partitions. + val numPartitions = 8 + + def apply(plan: SparkPlan): SparkPlan = plan.transformUp { + case operator: SparkPlan => + // Check if every child's outputPartitioning satisfies the corresponding + // required data distribution. + def meetsRequirements = + !operator.requiredChildDistribution.zip(operator.children).map { + case (required, child) => + val valid = child.outputPartitioning.satisfies(required) + logger.debug( + s"${if (valid) "Valid" else "Invalid"} distribution," + + s"required: $required current: ${child.outputPartitioning}") + valid + }.exists(!_) + + // 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 + .map(_.outputPartitioning) + .sliding(2) + .map { + case Seq(a) => true + case Seq(a,b) => a compatibleWith b + }.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. + def addExchangeIfNecessary(partitioning: Partitioning, child: SparkPlan) = + if (child.outputPartitioning != partitioning) Exchange(partitioning, child) else child + + if (meetsRequirements && compatible) { + operator + } else { + // 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) => + addExchangeIfNecessary(HashPartitioning(clustering, numPartitions), child) + case (OrderedDistribution(ordering), child) => + addExchangeIfNecessary(RangePartitioning(ordering, numPartitions), child) + case (UnspecifiedDistribution, child) => child + case (dist, _) => sys.error(s"Don't know how to ensure $dist") + } + operator.withNewChildren(repartitionedChildren) + } + } +} 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 new file mode 100644 index 0000000000000..c1da3653c5b8e --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala @@ -0,0 +1,69 @@ +/* + * 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 + +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, 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, + join: Boolean, + outer: Boolean, + child: SparkPlan) + extends UnaryNode { + + def output = + if (join) child.output ++ generator.output else generator.output + + def execute() = { + if (join) { + 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) { + outerProjection(row) :: Nil + } else { + outputRows.map(or => joinProjection(joinedRow(row, or))) + } + } + } + } else { + child.execute().mapPartitions(iter => iter.flatMap(generator)) + } + } +} 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/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala new file mode 100644 index 0000000000000..5626181d184e5 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -0,0 +1,89 @@ +/* + * 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 + +import org.apache.spark.rdd.RDD + +import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.trees + +abstract class SparkPlan extends QueryPlan[SparkPlan] 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 WIDTH! + /** Specifies any partition requirements on the input data for this operator. */ + def requiredChildDistribution: Seq[Distribution] = + Seq.fill(children.size)(UnspecifiedDistribution) + + /** + * Runs this query returning the result as an RDD. + */ + 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.toArray) +} + +/** + * Allows already planned SparkQueries to be linked into logical query plans. + * + * Note that in general it is not valid to use this class to link multiple copies of the same + * physical operator into the same query plan as this violates the uniqueness of expression ids. + * Special handling exists for ExistingRdd as these are already leaf operators and thus we can just + * replace the output attributes with new copies of themselves without breaking any attribute + * linking. + */ +case class SparkLogicalPlan(alreadyPlanned: SparkPlan) + extends logical.LogicalPlan with MultiInstanceRelation { + + def output = alreadyPlanned.output + def references = Set.empty + def children = Nil + + override final def newInstance: this.type = { + SparkLogicalPlan( + alreadyPlanned match { + case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) + case _ => sys.error("Multiple instance of the same relation detected.") + }).asInstanceOf[this.type] + } +} + +trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { + self: Product => +} + +trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] { + self: Product => + override def outputPartitioning: Partitioning = child.outputPartitioning +} + +trait BinaryNode extends SparkPlan with trees.BinaryNode[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 new file mode 100644 index 0000000000000..85035b8118936 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala @@ -0,0 +1,229 @@ +/* + * 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 + +import org.apache.spark.SparkContext + +import catalyst.expressions._ +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] { + + val sparkContext: SparkContext + + object SparkEquiInnerJoin extends Strategy { + 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 + // 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 _ => 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) + } + + // 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 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 + } + } else { + logger.debug(s"Avoiding spark join with no join keys.") + Nil + } + 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 + } + + object PartialAggregation extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = 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))(sparkContext))(sparkContext) :: Nil + } else { + Nil + } + case _ => Nil + } + } + + object BroadcastNestedLoopJoin extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case logical.Join(left, right, joinType, condition) => + execution.BroadcastNestedLoopJoin( + planLater(left), planLater(right), joinType, condition)(sparkContext) :: Nil + case _ => Nil + } + } + + object CartesianProduct extends Strategy { + 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)) => + execution.Filter(condition, + execution.CartesianProduct(planLater(left), planLater(right))) :: Nil + case _ => Nil + } + } + + protected lazy val singleRowRdd = + sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): 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).toArray) + case other => other + } + + object TopK extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + case logical.StopAfter(IntegerLiteral(limit), logical.Sort(order, child)) => + execution.TopK(limit, order, planLater(child))(sparkContext) :: Nil + case _ => Nil + } + } + + // 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( + 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 + case logical.SortPartitions(sortExprs, child) => + // 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) => + execution.Filter(condition, planLater(child)) :: Nil + case logical.Aggregate(group, agg, child) => + 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) => + val dataAsRdd = + sparkContext.parallelize(data.map(r => + new GenericRow(r.productIterator.map(convertToCatalyst).toArray): Row)) + execution.ExistingRdd(output, dataAsRdd) :: 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) => + 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 logical.WriteToFile(path, child) => + 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/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala new file mode 100644 index 0000000000000..51889c1988680 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala @@ -0,0 +1,117 @@ +/* + * 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 + +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.PartitionLocalRDDFunctions._ + +/** + * 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(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().mapPartitions { iter => + val buildGrouping = new Projection(groupingExpressions) + iter.map(row => (buildGrouping(row), row.copy())) + }.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 => + aggFunctions.foreach(_.update(row)) + } + buildRow(aggImplementations.map(_.apply(group))) + } + + // 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(_.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 new file mode 100644 index 0000000000000..c6d31d9abc9e8 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala @@ -0,0 +1,137 @@ +/* + * 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 + +import scala.reflect.runtime.universe.TypeTag + +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 +import catalyst.ScalaReflection + +case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { + def output = projectList.map(_.toAttribute) + + def execute() = child.execute().mapPartitions { iter => + @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.apply(_).asInstanceOf[Boolean]) + } +} + +case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan) + extends UnaryNode { + + def output = child.output + + // TODO: How to pick seed? + def execute() = child.execute().sample(withReplacement, fraction, seed) +} + +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())) + + override def otherCopyArgs = sc :: Nil +} + +case class StopAfter(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { + override def otherCopyArgs = sc :: Nil + + def output = child.output + + 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|. + 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().map(_.copy()).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, + child: SparkPlan) + extends UnaryNode { + override def requiredChildDistribution = + if (global) OrderedDistribution(sortOrder) :: Nil else UnspecifiedDistribution :: Nil + + @transient + lazy val ordering = new RowOrdering(sortOrder) + + def execute() = attachTree(this, "sort") { + // TODO: Optimize sorting operation? + child.execute() + .mapPartitions( + iterator => iterator.map(_.copy()).toArray.sorted(ordering).iterator, + preservesPartitioning = true) + } + + 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] = { + // 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) + } + + 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/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/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 new file mode 100644 index 0000000000000..5934fd1b03bfb --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala @@ -0,0 +1,158 @@ +/* + * 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 + +import scala.collection.mutable + +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.PartitionLocalRDDFunctions._ + +case class SparkEquiInnerJoin( + leftKeys: Seq[Expression], + rightKeys: Seq[Expression], + left: SparkPlan, + right: SparkPlan) extends BinaryNode { + + override def outputPartitioning: Partitioning = left.outputPartitioning + + override def requiredChildDistribution = + ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + + def output = left.output ++ right.output + + def execute() = attachTree(this, "execute") { + val leftWithKeys = left.execute().mapPartitions { iter => + val generateLeftKeys = new Projection(leftKeys, left.output) + 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.copy())) + } + + // Do the join. + val joined = filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys)) + // Drop join keys and merge input tuples. + 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[(Row, Row)]) = + rdd.filter { + case (key: Seq[_], _) => !key.exists(_ == null) + } +} + +case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { + def output = left.output ++ right.output + + def execute() = left.execute().map(_.copy()).cartesian(right.execute().map(_.copy())).map { + case (l: Row, r: Row) => buildRow(l ++ r) + } +} + +case class BroadcastNestedLoopJoin( + streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression]) + (@transient sc: SparkContext) + extends BinaryNode { + // TODO: Override requiredChildDistribution. + + override def outputPartitioning: Partitioning = streamed.outputPartitioning + + override def otherCopyArgs = sc :: Nil + + def output = left.output ++ right.output + + /** The Streamed Relation */ + def left = streamed + /** 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().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 + + 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(joinedRow(streamedRow, broadcastedRow)).asInstanceOf[Boolean]) { + matchedRows += buildRow(streamedRow ++ broadcastedRow) + matched = true + includedBroadcastTuples += i + } + i += 1 + } + + if (!matched && (joinType == LeftOuter || joinType == FullOuter)) { + matchedRows += buildRow(streamedRow ++ Array.fill(right.output.size)(null)) + } + } + Iterator((matchedRows, includedBroadcastTuples)) + } + + 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) => !allIncludedBroadcastTuples.contains(i) + }.map { + // TODO: Use projection. + case (row, _) => buildRow(Vector.fill(left.output.size)(null) ++ row) + } + } else { + Vector() + } + + // TODO: Breaks lineage. + sc.union( + streamedPlusMatches.flatMap(_._1), sc.makeRDD(rightOuterMatches)) + } +} 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..67f6f43f904b1 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/package.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 + +/** + * An execution engine for relational query plans that runs on top Spark and returns RDDs. + * + * Note that the operators in this package are created automatically by a query planner using a + * [[SQLContext]] and are not intended to be used directly by end users of Spark SQL. They are + * documented here in order to make it easier for others to understand the performance + * characteristics of query plans that are generated by Spark SQL. + */ +package object execution { +} 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 new file mode 100644 index 0000000000000..e87561fe1384f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala @@ -0,0 +1,276 @@ +/* + * 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.{IOException, FileNotFoundException} + +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.{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, 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.schema.Type.Repetition +import parquet.io.api.{Binary, RecordConsumer} +import parquet.hadoop.{Footer, ParquetFileWriter, ParquetFileReader} +import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} +import parquet.hadoop.util.ContextUtil + +import scala.collection.JavaConversions._ + +/** + * Relation that consists of data stored in a Parquet columnar format. + * + * Users should interact with parquet files though a SchemaRDD, created by a [[SQLContext]] instead + * of using this class directly. + * + * {{{ + * val parquetRDD = sqlContext.parquetFile("path/to/parequet.file") + * }}} + * + * @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 { + + /** Schema derived from ParquetFile **/ + def parquetSchema: MessageType = + ParquetTypesConverter + .readMetaData(new Path(path)) + .getFileMetaData + .getSchema + + /** Attributes **/ + val attributes = + ParquetTypesConverter + .convertToAttributes(parquetSchema) + + /** Output **/ + override val output = attributes + + // 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 { + + // The element type for the RDDs that this relation maps to. + type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow + + /** + * Creates a new ParquetRelation and underlying Parquetfile for the given + * 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. + * + * @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, + 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) + ParquetTypesConverter.writeMetaData(child.output, path, conf) + new ParquetRelation(name, path.toString) + } + + private def checkPath(pathStr: String, conf: Configuration): Path = { + 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 + .getUserAction + .implies(FsAction.READ_WRITE)) { + throw new IOException( + s"Unable to create ParquetRelation: path $path not read-writable") + } + 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 => { + // 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") + } + + 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 $ctype") + } + + 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)) + case FloatType => consumer.addFloat(record.getFloat(index)) + case BooleanType => consumer.addBoolean(record.getBoolean(index)) + case _ => sys.error(s"Unsupported datatype $ctype, cannot write to consumer") + } + } + + def getSchema(schemaString : String) : MessageType = + MessageTypeParser.parseMessageType(schemaString) + + def convertToAttributes(parquetSchema: MessageType) : Seq[Attribute] = { + parquetSchema.getColumns.map { + case (desc) => { + val ctype = toDataType(desc.getType) + val name: String = desc.getPath.mkString(".") + new AttributeReference(name, ctype, false)() + } + } + } + + // 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) + } + + 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 (fs.exists(metadataPath)) { + try { + 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.? + + val parquetSchema: MessageType = + ParquetTypesConverter.convertFromAttributes(attributes) + val metaData: FileMetaData = new FileMetaData( + parquetSchema, + extraMetadata, + "Spark") + + 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(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 = 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) + } 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) + } + } +} 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 new file mode 100644 index 0000000000000..61121103cbbfa --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableOperations.scala @@ -0,0 +1,212 @@ +/* + * 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 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.{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.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 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( + @transient output: Seq[Attribute], + @transient relation: ParquetRelation, + @transient columnPruningPred: Option[Expression])( + @transient val sc: SparkContext) + extends LeafNode { + + override def execute(): RDD[Row] = { + val job = new Job(sc.hadoopConfiguration) + ParquetInputFormat.setReadSupportClass( + job, + classOf[org.apache.spark.sql.parquet.RowReadSupport]) + val conf: Configuration = ContextUtil.getConfiguration(job) + conf.set( + 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]], + 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 { + sys.error("Warning: Could not validate Parquet schema projection in pruneColumns") + this + } + } + + /** + * 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) + try { + original.checkContains(candidate) + true + } catch { + case e: InvalidRecordException => { + false + } + } + } +} + +case class InsertIntoParquetTable( + @transient relation: ParquetRelation, + @transient child: SparkPlan)( + @transient val sc: SparkContext) + extends UnaryNode with SparkHadoopMapReduceUtil { + + /** + * 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 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(sc.hadoopConfiguration) + + ParquetOutputFormat.setWriteSupportClass( + job, + classOf[org.apache.spark.sql.parquet.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) + val fs = fspath.getFileSystem(conf) + + try { + fs.delete(fspath, true) + } catch { + case e: IOException => + throw new IOException( + s"Unable to clear output directory ${fspath.toString} prior" + + s" to InsertIntoParquetTable:\n${e.toString}") + } + 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 Product2 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 new file mode 100644 index 0000000000000..c2ae18b882d19 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala @@ -0,0 +1,220 @@ +/* + * 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 org.apache.hadoop.conf.Configuration + +import org.apache.spark.Logging + +import parquet.io.api._ +import parquet.schema.{MessageTypeParser, MessageType} +import parquet.hadoop.api.{WriteSupport, ReadSupport} +import parquet.hadoop.api.ReadSupport.ReadContext +import parquet.hadoop.ParquetOutputFormat +import parquet.column.ParquetProperties + +import org.apache.spark.sql.catalyst.expressions.{Row, Attribute} +import org.apache.spark.sql.catalyst.types._ + +/** + * 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))) + + override def getCurrentRecord: Row = root.getCurrentRecord + + 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, + 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 = { + 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) + } +} + +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 { + def setSchema(schema: MessageType, configuration: Configuration) { + // 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) + } + + def getSchema(configuration: Configuration): MessageType = { + return MessageTypeParser.parseMessageType( + configuration.get(RowWriteSupport.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 = { + var index = 0 + writer.startMessage() + 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() + } +} + +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. + * + * @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 { + + def this(schema: Seq[Attribute]) = this(schema, new ParquetRelation.RowType(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( + s"unable to convert datatype ${a.dataType.toString} in CatalystGroupConverter") + } + }.toArray + + override def getConverter(fieldIndex: Int): Converter = converters(fieldIndex) + + private[parquet] def getCurrentRecord: ParquetRelation.RowType = current + + override def start(): Unit = { + var i = 0 + while (i < schema.length) { + current.setNullAt(i) + i = i + 1 + } + } + + override def end(): Unit = {} +} + +/** + * 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 = + // 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) + + override def addDouble(value: Double): Unit = + parent.getCurrentRecord.setDouble(fieldIndex, value) + + override def addFloat(value: Float): Unit = + parent.getCurrentRecord.setFloat(fieldIndex, value) + + override def addInt(value: Int): Unit = + parent.getCurrentRecord.setInt(fieldIndex, value) + + override def addLong(value: Long): Unit = + parent.getCurrentRecord.setLong(fieldIndex, value) +} + +/** + * 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.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 new file mode 100644 index 0000000000000..bbe409fb9c7e0 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTestData.scala @@ -0,0 +1,103 @@ +/* + * 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 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 + +import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.catalyst.expressions.GenericRow +import java.nio.charset.Charset + +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 + + // 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 { + |optional boolean myboolean; + |optional int64 mylong; + |} + """.stripMargin + + // 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) + + 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, i.toLong << 33) + data.update(4, 2.5F) + data.update(5, 4.5D) + writer.write(new GenericRow(data.toArray)) + } + writer.close() + } +} + diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala new file mode 100644 index 0000000000000..ca56c4476bcc2 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.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 +package sql +package test + +/** A SQLContext that can be used for local testing. */ +object TestSQLContext + extends SQLContext(new SparkContext("local", "TestSQLContext", new SparkConf())) diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..7bb6789bd33a5 --- /dev/null +++ b/sql/core/src/test/resources/log4j.properties @@ -0,0 +1,52 @@ +# +# 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 + +# Parquet logging +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/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala new file mode 100644 index 0000000000000..37c90a18a0148 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -0,0 +1,201 @@ +/* + * 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} + +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.test._ + +/* Implicits */ +import TestSQLContext._ + +class DslQuerySuite extends QueryTest { + import TestData._ + + test("table scan") { + checkAnswer( + testData, + testData.collect().toSeq) + } + + test("agg") { + checkAnswer( + testData2.groupBy('a)('a, Sum('b)), + Seq((1,3),(2,3),(3,3)) + ) + } + + test("select *") { + checkAnswer( + testData.select(Star(None)), + testData.collect().toSeq) + } + + test("simple select") { + checkAnswer( + testData.where('key === 1).select('value), + Seq(Seq("1"))) + } + + test("sorting") { + checkAnswer( + testData2.orderBy('a.asc, 'b.asc), + Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2))) + + checkAnswer( + testData2.orderBy('a.asc, 'b.desc), + Seq((1,2), (1,1), (2,2), (2,1), (3,2), (3,1))) + + checkAnswer( + testData2.orderBy('a.desc, 'b.desc), + Seq((3,2), (3,1), (2,2), (2,1), (1,2), (1,1))) + + checkAnswer( + testData2.orderBy('a.desc, 'b.asc), + Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + } + + test("average") { + checkAnswer( + testData2.groupBy()(Average('a)), + 2.0) + } + + test("count") { + checkAnswer( + testData2.groupBy()(Count(1)), + testData2.count() + ) + } + + test("null count") { + checkAnswer( + testData3.groupBy('a)('a, Count('b)), + Seq((1,0), (2, 1)) + ) + + checkAnswer( + testData3.groupBy()(Count('a), Count('b), Count(1), CountDistinct('a :: Nil), CountDistinct('b :: Nil)), + (2, 1, 2, 2, 1) :: Nil + ) + } + + test("inner join where, one match per row") { + checkAnswer( + upperCaseData.join(lowerCaseData, Inner).where('n === 'N), + Seq( + (1, "A", 1, "a"), + (2, "B", 2, "b"), + (3, "C", 3, "c"), + (4, "D", 4, "d") + )) + } + + test("inner join ON, one match per row") { + checkAnswer( + upperCaseData.join(lowerCaseData, Inner, Some('n === 'N)), + Seq( + (1, "A", 1, "a"), + (2, "B", 2, "b"), + (3, "C", 3, "c"), + (4, "D", 4, "d") + )) + } + + test("inner join, where, multiple matches") { + val x = testData2.where('a === 1).subquery('x) + val y = testData2.where('a === 1).subquery('y) + checkAnswer( + x.join(y).where("x.a".attr === "y.a".attr), + (1,1,1,1) :: + (1,1,1,2) :: + (1,2,1,1) :: + (1,2,1,2) :: Nil + ) + } + + test("inner join, no matches") { + val x = testData2.where('a === 1).subquery('x) + val y = testData2.where('a === 2).subquery('y) + checkAnswer( + x.join(y).where("x.a".attr === "y.a".attr), + Nil) + } + + test("big inner join, 4 matches per row") { + val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData) + val bigDataX = bigData.subquery('x) + val bigDataY = bigData.subquery('y) + + checkAnswer( + bigDataX.join(bigDataY).where("x.key".attr === "y.key".attr), + testData.flatMap( + row => Seq.fill(16)((row ++ row).toSeq)).collect().toSeq) + } + + test("cartisian product join") { + checkAnswer( + testData3.join(testData3), + (1, null, 1, null) :: + (1, null, 2, 2) :: + (2, 2, 1, null) :: + (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) + } +} \ No newline at end of file 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 new file mode 100644 index 0000000000000..83908edf5a040 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala @@ -0,0 +1,62 @@ +/* + * 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 + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.test.TestSQLContext._ +import org.apache.spark.sql.test.TestSQLContext.planner._ + +class PlannerSuite extends FunSuite { + + + test("unions are collapsed") { + val query = testData.unionAll(testData).unionAll(testData).logicalPlan + val planned = 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("count is partially aggregated") { + val query = testData.groupBy('value)(Count('key)).analyze.logicalPlan + val planned = PartialAggregation(query).head + val aggregations = planned.collect { case a: Aggregate => a } + + assert(aggregations.size === 2) + } + + test("count distinct is not partially aggregated") { + val query = testData.groupBy('value)(CountDistinct('key :: Nil)).analyze.logicalPlan + val planned = PartialAggregation(query.logicalPlan) + assert(planned.isEmpty) + } + + test("mixed aggregates are not partially aggregated") { + val query = + testData.groupBy('value)(Count('value), CountDistinct('key :: Nil)).analyze.logicalPlan + val planned = PartialAggregation(query) + assert(planned.isEmpty) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala new file mode 100644 index 0000000000000..728fecededeb0 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -0,0 +1,75 @@ +/* + * 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} + +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.test._ + +/* Implicits */ +import TestSQLContext._ + +class QueryTest extends FunSuite { + /** + * 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(rdd: SchemaRDD, 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 = rdd.logicalPlan.collect { case s: logical.Sort => s}.nonEmpty + def prepareAnswer(answer: Seq[Any]) = if (!isSorted) answer.sortBy(_.toString) else answer + val sparkAnswer = try rdd.collect().toSeq catch { + case e: Exception => + fail( + s""" + |Exception thrown while executing query: + |${rdd.logicalPlan} + |== Exception == + |$e + """.stripMargin) + } + if(prepareAnswer(convertedAnswer) != prepareAnswer(sparkAnswer)) { + fail(s""" + |Results do not match for query: + |${rdd.logicalPlan} + |== Analyzed Plan == + |${rdd.queryExecution.analyzed} + |== RDD == + |$rdd + |== Results == + |${sideBySide( + prepareAnswer(convertedAnswer).map(_.toString), + prepareAnswer(sparkAnswer).map(_.toString)).mkString("\n")} + """.stripMargin) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala new file mode 100644 index 0000000000000..5728313d6dd3d --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -0,0 +1,211 @@ +/* + * 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} + +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.test._ + +/* Implicits */ +import TestSQLContext._ +import TestData._ + +class SQLQuerySuite extends QueryTest { + test("agg") { + checkAnswer( + sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"), + Seq((1,3),(2,3),(3,3)) + ) + } + + test("select *") { + checkAnswer( + sql("SELECT * FROM testData"), + testData.collect().toSeq) + } + + test("simple select") { + checkAnswer( + sql("SELECT value FROM testData WHERE key = 1"), + Seq(Seq("1"))) + } + + test("sorting") { + checkAnswer( + sql("SELECT * FROM testData2 ORDER BY a ASC, b ASC"), + Seq((1,1), (1,2), (2,1), (2,2), (3,1), (3,2))) + + checkAnswer( + sql("SELECT * FROM testData2 ORDER BY a ASC, b DESC"), + Seq((1,2), (1,1), (2,2), (2,1), (3,2), (3,1))) + + checkAnswer( + sql("SELECT * FROM testData2 ORDER BY a DESC, b DESC"), + Seq((3,2), (3,1), (2,2), (2,1), (1,2), (1,1))) + + checkAnswer( + sql("SELECT * FROM testData2 ORDER BY a DESC, b ASC"), + Seq((3,1), (3,2), (2,1), (2,2), (1,1), (1,2))) + } + + test("average") { + checkAnswer( + sql("SELECT AVG(a) FROM testData2"), + 2.0) + } + + test("count") { + checkAnswer( + sql("SELECT COUNT(*) FROM testData2"), + testData2.count() + ) + } + + // No support for primitive nulls yet. + ignore("null count") { + checkAnswer( + sql("SELECT a, COUNT(b) FROM testData3"), + Seq((1,0), (2, 1)) + ) + + checkAnswer( + testData3.groupBy()(Count('a), Count('b), Count(1), CountDistinct('a :: Nil), CountDistinct('b :: Nil)), + (2, 1, 2, 2, 1) :: Nil + ) + } + + test("inner join where, one match per row") { + checkAnswer( + sql("SELECT * FROM upperCaseData JOIN lowerCaseData WHERE n = N"), + Seq( + (1, "A", 1, "a"), + (2, "B", 2, "b"), + (3, "C", 3, "c"), + (4, "D", 4, "d") + )) + } + + test("inner join ON, one match per row") { + checkAnswer( + sql("SELECT * FROM upperCaseData JOIN lowerCaseData ON n = N"), + Seq( + (1, "A", 1, "a"), + (2, "B", 2, "b"), + (3, "C", 3, "c"), + (4, "D", 4, "d") + )) + } + + test("inner join, where, multiple matches") { + checkAnswer( + sql(""" + |SELECT * FROM + | (SELECT * FROM testData2 WHERE a = 1) x JOIN + | (SELECT * FROM testData2 WHERE a = 1) y + |WHERE x.a = y.a""".stripMargin), + (1,1,1,1) :: + (1,1,1,2) :: + (1,2,1,1) :: + (1,2,1,2) :: Nil + ) + } + + test("inner join, no matches") { + checkAnswer( + sql( + """ + |SELECT * FROM + | (SELECT * FROM testData2 WHERE a = 1) x JOIN + | (SELECT * FROM testData2 WHERE a = 2) y + |WHERE x.a = y.a""".stripMargin), + Nil) + } + + test("big inner join, 4 matches per row") { + + + checkAnswer( + sql( + """ + |SELECT * FROM + | (SELECT * FROM testData UNION ALL + | SELECT * FROM testData UNION ALL + | SELECT * FROM testData UNION ALL + | SELECT * FROM testData) x JOIN + | (SELECT * FROM testData UNION ALL + | SELECT * FROM testData UNION ALL + | SELECT * FROM testData UNION ALL + | SELECT * FROM testData) y + |WHERE x.key = y.key""".stripMargin), + testData.flatMap( + row => Seq.fill(16)((row ++ row).toSeq)).collect().toSeq) + } + + ignore("cartisian product join") { + checkAnswer( + testData3.join(testData3), + (1, null, 1, null) :: + (1, null, 2, 2) :: + (2, 2, 1, null) :: + (2, 2, 2, 2) :: Nil) + } + + test("left outer join") { + checkAnswer( + sql("SELECT * FROM upperCaseData LEFT OUTER JOIN lowerCaseData ON 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( + sql("SELECT * FROM lowerCaseData RIGHT OUTER JOIN upperCaseData ON 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") { + checkAnswer( + sql( + """ + |SELECT * FROM + | (SELECT * FROM upperCaseData WHERE N <= 4) left FULL OUTER JOIN + | (SELECT * FROM upperCaseData WHERE N >= 3) right + | ON left.N = right.N + """.stripMargin), + (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) + } +} \ No newline at end of file diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala new file mode 100644 index 0000000000000..640292571b125 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -0,0 +1,72 @@ +/* + * 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.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.test._ + +/* Implicits */ +import TestSQLContext._ + +object TestData { + case class TestData(key: Int, value: String) + val testData: SchemaRDD = TestSQLContext.sparkContext.parallelize( + (1 to 100).map(i => TestData(i, i.toString))) + testData.registerAsTable("testData") + + case class TestData2(a: Int, b: Int) + val testData2: SchemaRDD = + TestSQLContext.sparkContext.parallelize( + TestData2(1, 1) :: + TestData2(1, 2) :: + TestData2(2, 1) :: + TestData2(2, 2) :: + TestData2(3, 1) :: + TestData2(3, 2) :: Nil + ) + testData2.registerAsTable("testData2") + + // TODO: There is no way to express null primitives as case classes currently... + val testData3 = + logical.LocalRelation('a.int, 'b.int).loadData( + (1, null) :: + (2, 2) :: Nil + ) + + case class UpperCaseData(N: Int, L: String) + val upperCaseData = + TestSQLContext.sparkContext.parallelize( + UpperCaseData(1, "A") :: + UpperCaseData(2, "B") :: + UpperCaseData(3, "C") :: + UpperCaseData(4, "D") :: + UpperCaseData(5, "E") :: + UpperCaseData(6, "F") :: Nil + ) + upperCaseData.registerAsTable("upperCaseData") + + case class LowerCaseData(n: Int, l: String) + val lowerCaseData = + TestSQLContext.sparkContext.parallelize( + LowerCaseData(1, "a") :: + LowerCaseData(2, "b") :: + LowerCaseData(3, "c") :: + LowerCaseData(4, "d") :: Nil + ) + lowerCaseData.registerAsTable("lowerCaseData") +} \ No newline at end of file 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 new file mode 100644 index 0000000000000..08265b7a6a2bf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala @@ -0,0 +1,71 @@ +/* + * 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 + +import org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.test._ + + +import TestSQLContext._ + +/** + * 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 + + override def apply(input: Row): TraversableOnce[Row] = { + val name = nameAttr.apply(input) + val age = ageAttr.apply(input).asInstanceOf[Int] + + Iterator( + new GenericRow(Array[Any](s"$name is $age years old")), + new GenericRow(Array[Any](s"Next year, $name will be ${age + 1} years old"))) + } +} + +class TgfSuite extends QueryTest { + 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)) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala new file mode 100644 index 0000000000000..8b2ccb52d82c3 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -0,0 +1,126 @@ +/* + * 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 org.scalatest.{BeforeAndAfterAll, FunSuite} + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.Row +import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.test.TestSQLContext + +import org.apache.hadoop.mapreduce.Job +import org.apache.hadoop.fs.{Path, FileSystem} + +import parquet.schema.MessageTypeParser +import parquet.hadoop.ParquetFileWriter +import parquet.hadoop.util.ContextUtil + +class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { + override def beforeAll() { + ParquetTestData.writeFile + } + + override def afterAll() { + ParquetTestData.testFile.delete() + } + + test("Import of simple Parquet file") { + val result = getRDD(ParquetTestData.testData).collect() + assert(result.size === 15) + result.zipWithIndex.foreach { + case (row, index) => { + val checkBoolean = + if (index % 3 == 0) + row(0) == true + else + row(0) == false + 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") + } + } + } + + 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))) + assert(projected.output.size === 2) + val result = projected + .execute() + .map(_.copy()) + .collect() + result.zipWithIndex.foreach { + case (row, index) => { + if (index % 3 == 0) + assert(row(0) === true, s"boolean field value in line $index did not match (every third row)") + else + 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") + } + } + } + + test("Writing metadata from scratch for table CREATE") { + val job = new Job() + 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) + 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, true) + } + + /** + * 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.output, + parquetRelation, + None)(TestSQLContext.sparkContext) + scanner + .execute + .map(_.copy()) + } +} + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml new file mode 100644 index 0000000000000..7b5ea98f27ff5 --- /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.apache.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 + + + + diff --git a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala new file mode 100644 index 0000000000000..08d390e8871ab --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala @@ -0,0 +1,198 @@ +/* + * 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 +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.{Utilities, FileSinkOperator} +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +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[apache] +class SparkHiveHadoopWriter( + @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 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( + conf.value, + fileSinkConf.getTableInfo, + conf.value.getOutputValueClass.asInstanceOf[Class[Writable]], + fileSinkConf, + path, + 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 SparkHiveHadoopWriter { + 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) + } +} 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 new file mode 100644 index 0000000000000..4aad876cc0364 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala @@ -0,0 +1,287 @@ +/* + * 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 + +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 org.apache.spark.sql.execution._ + +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) extends HiveContext(sc) { + + 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. + */ +class HiveContext(sc: SparkContext) extends SQLContext(sc) { + self => + + override def parseSql(sql: String): LogicalPlan = HiveQl.parseSql(sql) + override def executePlan(plan: LogicalPlan): this.QueryExecution = + new this.QueryExecution { val logical = plan } + + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. + @transient + protected 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 protected[hive] lazy val hiveconf = new HiveConf(classOf[SessionState]) + @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") + + /* A catalyst metadata catalog that points to the Hive Metastore. */ + @transient + override lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog + + /* An analyzer that uses the Hive metastore. */ + @transient + override lazy 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. + */ + 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... + 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. + */ + 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+") + 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 => + logger.error( + s""" + |====================== + |HIVE FAILURE OUTPUT + |====================== + |${outputBuffer.toString} + |====================== + |END HIVE FAILURE OUTPUT + |====================== + """.stripMargin) + throw e + } + } + + @transient + val hivePlanner = new SparkPlanner with HiveStrategies { + val hiveContext = self + + override val strategies: Seq[Strategy] = Seq( + TopK, + ColumnPrunings, + PartitionPrunings, + HiveTableScans, + DataSinks, + Scripts, + PartialAggregation, + SparkEquiInnerJoin, + BasicOperators, + CartesianProduct, + BroadcastNestedLoopJoin + ) + } + + @transient + override val planner = hivePlanner + + @transient + 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 { + // 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").asInstanceOf[Array[Any]])) + sparkContext.parallelize(asRows, 1) + } + case _ => + executedPlan.execute.map(_.copy()) + } + + 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/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 new file mode 100644 index 0000000000000..e4d50722ce9a8 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -0,0 +1,246 @@ +/* + * 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 + +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.ql.plan.TableDesc +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.serde2.Deserializer + +import catalyst.analysis.Catalog +import catalyst.expressions._ +import catalyst.plans.logical +import catalyst.plans.logical._ +import catalyst.rules._ +import catalyst.types._ + +import scala.collection.JavaConversions._ + +class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { + import HiveMetastoreTypes._ + + val client = Hive.get(hive.hiveconf) + + def lookupRelation( + db: Option[String], + tableName: String, + alias: Option[String]): LogicalPlan = { + val databaseName = db.getOrElse(hive.sessionState.getCurrentDatabase()) + val table = client.getTable(databaseName, tableName) + 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.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] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case InsertIntoCreatedTable(db, tableName, child) => + val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase()) + + createTable(databaseName, tableName, child.output) + + InsertIntoTable( + lookupRelation(Some(databaseName), tableName, None).asInstanceOf[BaseRelation], + Map.empty, + child, + overwrite = false) + } + } + + /** + * 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 = logical.Project(castedChildOutput, child)) + } + } + } + + /** + * 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 { + protected lazy val primitiveType: Parser[DataType] = + "string" ^^^ StringType | + "float" ^^^ FloatType | + "int" ^^^ IntegerType | + "tinyint" ^^^ ShortType | + "double" ^^^ DoubleType | + "bigint" ^^^ LongType | + "binary" ^^^ BinaryType | + "boolean" ^^^ BooleanType | + "decimal" ^^^ DecimalType | + "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-Z0-9]*".r ~ ":" ~ dataType ^^ { + case name ~ _ ~ tpe => StructField(name, tpe, nullable = 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") + } + + 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]) + (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 Table(table) + + def hiveQlPartitions = partitions.map { p => + new Partition(hiveQlTable, p) + } + + override def isPartitioned = hiveQlTable.isPartitioned + + val tableDesc = new TableDesc( + 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 + ) + + implicit class SchemaAttribute(f: FieldSchema) { + def toAttribute = AttributeReference( + f.getName, + HiveMetastoreTypes.toDataType(f.getType), + // Since data can be dumped in randomly with no validation, everything is nullable. + nullable = true + )(qualifiers = tableName +: alias.toSeq) + } + + // Must be a stable value since new attributes are born here. + val partitionKeys = hiveQlTable.getPartitionKeys.map(_.toAttribute) + + /** Non-partitionKey attributes */ + val attributes = table.getSd.getCols.map(_.toAttribute) + + val output = attributes ++ partitionKeys +} 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 new file mode 100644 index 0000000000000..4f33a293c3964 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala @@ -0,0 +1,966 @@ +/* + * 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 + +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._ +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 + * back for Hive to execute natively. Will be replaced with a native command that contains the + * cmd string. + */ +case object NativePlaceholder 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 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", + "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_ANALYZE", + "TOK_ALTERDATABASE_PROPERTIES", + "TOK_ALTERINDEX_PROPERTIES", + "TOK_ALTERINDEX_REBUILD", + "TOK_ALTERTABLE_ADDCOLS", + "TOK_ALTERTABLE_ADDPARTS", + "TOK_ALTERTABLE_ALTERPARTS", + "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_SKEWED", + "TOK_ALTERTABLE_TOUCH", + "TOK_ALTERTABLE_UNARCHIVE", + "TOK_ANALYZE", + "TOK_CREATEDATABASE", + "TOK_CREATEFUNCTION", + "TOK_CREATEINDEX", + "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_AS", + "TOK_ALTERVIEW_DROPPARTS", + "TOK_ALTERVIEW_PROPERTIES", + "TOK_ALTERVIEW_RENAME", + "TOK_CREATEVIEW", + "TOK_DROPVIEW", + + "TOK_EXPORT", + "TOK_IMPORT", + "TOK_LOAD", + + "TOK_SWITCHDATABASE" + ) + + /** + * 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. + */ + 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. + * @param rule the function use to transform this nodes children + */ + def transform(rule: PartialFunction[ASTNode, ASTNode]): ASTNode = { + try { + val afterRule = rule.applyOrElse(n, identity[ASTNode]) + afterRule.withChildren( + nilIfEmpty(afterRule.getChildren) + .asInstanceOf[Seq[ASTNode]] + .map(ast => Option(ast).map(_.transform(rule)).orNull)) + } catch { + case e: Exception => + println(dumpTree(n)) + throw e + } + } + + /** + * Returns a scala.Seq equivilent to [s] or Nil if [s] is null. + */ + private def nilIfEmpty[A](s: java.util.List[A]): Seq[A] = + Option(s).map(_.toSeq).getOrElse(Nil) + + /** + * Returns this ASTNode with the text changed to `newText``. + */ + def withText(newText: String): ASTNode = { + n.token.asInstanceOf[org.antlr.runtime.CommonToken].setText(newText) + n + } + + /** + * Returns this ASTNode with the children changed to `newChildren`. + */ + def withChildren(newChildren: Seq[ASTNode]): ASTNode = { + (1 to n.getChildCount).foreach(_ => n.deleteChild(0)) + n.addChildren(newChildren) + n + } + + /** + * Throws an error if this is not equal to other. + * + * Right now this function only checks the name, type, text and children of the node + * 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. " + + s"'${f(n)}' != '${f(other)}' left: ${dumpTree(n)}, right: ${dumpTree(other)}") + } + check("name", _.getName) + check("type", _.getType) + check("text", _.getText) + check("numChildren", n => nilIfEmpty(n.getChildren).size) + + val leftChildren = nilIfEmpty(n.getChildren).asInstanceOf[Seq[ASTNode]] + val rightChildren = nilIfEmpty(other.getChildren).asInstanceOf[Seq[ASTNode]] + leftChildren zip rightChildren foreach { + case (l,r) => l checkEquals r + } + } + } + + class ParseException(sql: String, cause: Throwable) + extends Exception(s"Failed to parse: $sql", cause) + + /** + * Returns the AST for the given SQL string. + */ + 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")) { + NativeCommand(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) match { + case NativePlaceholder => NativeCommand(sql) + case other => other + } + } + } + } catch { + case e: Exception => throw new ParseException(sql, e) + } + } + + 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. */ + object Token { + /** @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]])) + case _ => None + } + } + + protected def getClauses(clauseNames: Seq[String], nodeList: Seq[ASTNode]): Seq[Option[Node]] = { + 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) + matches.headOption + } + + assert(remainingNodes.isEmpty, + s"Unhandled clauses: ${remainingNodes.map(dumpTree(_)).mkString("\n")}") + clauses + } + + def getClause(clauseName: String, nodeList: Seq[Node]) = + 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 { + case Seq(oneMatch) => Some(oneMatch) + case Seq() => None + case _ => sys.error(s"Found multiple instances of clause $clauseName") + } + } + + 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), nullable = true) + case Token("TOK_TABCOL", + Token(fieldName, Nil) :: + dataType :: + _ /* comment */:: Nil) => + StructField(fieldName, nodeToDataType(dataType), nullable = true) + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for StructField:\n ${dumpTree(a).toString} ") + } + + protected def nameExpressions(exprs: Seq[Expression]): Seq[NamedExpression] = { + exprs.zipWithIndex.map { + case (ne: NamedExpression, _) => ne + case (e, i) => Alias(e, s"c_$i")() + } + } + + protected def nodeToPlan(node: Node): LogicalPlan = node match { + // Just fake explain for any of the native commands. + 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) + // TODO: support EXTENDED? + ExplainCommand(nodeToPlan(query)) + + case Token("TOK_CREATETABLE", children) + if children.collect { case t@Token("TOK_QUERY", _) => t }.nonEmpty => + // 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 { + 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 + + case Token("TOK_QUERY", + Token("TOK_FROM", fromClause :: Nil) :: + insertClauses) => + + // Return one query for each insert clause. + val queries = insertClauses.map { case Token("TOK_INSERT", singleInsert) => + val ( + intoClause :: + destClause :: + selectClause :: + selectDistinctClause :: + whereClause :: + groupByClause :: + orderByClause :: + sortByClause :: + clusterByClause :: + distributeByClause :: + limitClause :: + lateralViewClause :: 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", + "TOK_LATERAL_VIEW"), + singleInsert) + } + + val relations = nodeToRelation(fromClause) + val withWhere = whereClause.map { whereNode => + val Seq(whereExpr) = whereNode.getChildren.toSeq + 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 = select.getChildren.head match { + case Token("TOK_SELEXPR", + 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_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( + logical.ScriptTransformation( + inputExprs.map(nodeToExpr), + unescapedScript, + output, + withWhere)) + 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 { + // 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, withLateralView) + case None => + Project(selectExpressions, withLateralView) + } + } + + val withDistinct = + if (selectDistinctClause.isDefined) Distinct(withProject) else withProject + + val withSort = + (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)) + .getOrElse(withSort) + + // TOK_INSERT_INTO means to add files to the table. + // TOK_DESTINATION means to overwrite the table. + val resultDestination = + (intoClause orElse destClause).getOrElse(sys.error("No destination found.")) + val overwrite = if (intoClause.isEmpty) true else false + nodeToDest( + resultDestination, + withLimit, + overwrite) + } + + // 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)) + + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") + } + + 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. + val (nonAliasClauses, aliasClause) = + if (clauses.last.getText.startsWith("TOK")) { + (clauses, None) + } else { + (clauses.dropRight(1), Some(clauses.last)) + } + + val (Some(tableNameParts) :: + splitSampleClause :: + bucketSampleClause :: Nil) = { + getClauses(Seq("TOK_TABNAME", "TOK_TABLESPLITSAMPLE", "TOK_TABLEBUCKETSAMPLE"), + nonAliasClauses) + } + + 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(db, tableName, alias) + + // Apply sampling if requested. + (bucketSampleClause orElse splitSampleClause).map { + case Token("TOK_TABLESPLITSAMPLE", + 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) => + 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)) + } + + 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. + // 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(groups, Star(None, First(_)) :: Nil, joinedResult) + + case Token(allJoinTokens(joinToken), + relation1 :: + relation2 :: other) => + assert(other.size <= 1, s"Unhandled join child ${other}") + val joinType = joinToken match { + case "TOK_JOIN" => Inner + case "TOK_RIGHTOUTERJOIN" => RightOuter + case "TOK_LEFTOUTERJOIN" => LeftOuter + case "TOK_FULLOUTERJOIN" => FullOuter + } + assert(other.size <= 1, "Unhandled join clauses.") + Join(nodeToRelation(relation1), + nodeToRelation(relation2), + joinType, + other.headOption.map(nodeToExpr)) + + case a: ASTNode => + throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") + } + + 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} ") + } + + val destinationToken = "TOK_DESTINATION|TOK_INSERT_INTO".r + 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) => + query + + case Token(destinationToken(), + Token("TOK_TAB", + tableArgs) :: Nil) => + val Some(tableNameParts) :: partitionClause :: Nil = + getClauses(Seq("TOK_TABNAME", "TOK_PARTSPEC"), tableArgs) + + 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 { + // Parse partitions. We also make keys case insensitive. + case Token("TOK_PARTVAL", Token(key, Nil) :: Token(value, Nil) :: Nil) => + 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)) { + 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 => + throw new NotImplementedError(s"No parse rules for:\n ${dumpTree(a).toString} ") + } + + protected def selExprNodeToExpr(node: Node): Option[Expression] = node match { + case Token("TOK_SELEXPR", + e :: Nil) => + Some(nodeToExpr(e)) + + case Token("TOK_SELEXPR", + e :: Token(alias, Nil) :: Nil) => + 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} ") + } + + + protected val escapedIdentifier = "`([^`]+)`".r + /** Strips backticks from ident if present */ + protected def cleanIdentifier(ident: String): String = ident match { + case escapedIdentifier(i) => i + case plainIdent => plainIdent + } + + val numericAstTypes = Seq( + HiveParser.Number, + HiveParser.TinyintLiteral, + HiveParser.SmallintLiteral, + HiveParser.BigintLiteral) + + /* Case insensitive matches */ + val COUNT = "(?i)COUNT".r + 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 + val TRUE = "(?i)TRUE".r + val FALSE = "(?i)FALSE".r + + protected def nodeToExpr(node: Node): Expression = node match { + /* Attribute References */ + case Token("TOK_TABLE_OR_COL", + Token(name, Nil) :: Nil) => + 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 GetItem(UnresolvedAttribute(qualifierName), ordinal) => + GetItem(UnresolvedAttribute(qualifierName + "." + cleanIdentifier(attr)), ordinal) + } + + /* 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)) + + /* Aggregate Functions */ + case Token("TOK_FUNCTION", Token(AVG(), Nil) :: arg :: Nil) => Average(nodeToExpr(arg)) + case Token("TOK_FUNCTION", Token(COUNT(), Nil) :: arg :: Nil) => Count(nodeToExpr(arg)) + 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) => + 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) + case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), DecimalType) + + /* Arithmetic */ + case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) + case Token("+", left :: right:: Nil) => Add(nodeToExpr(left), nodeToExpr(right)) + 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)) + case Token("%", left :: right:: Nil) => Remainder(nodeToExpr(left), nodeToExpr(right)) + + /* 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)) + 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)) + + /* 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)) + + /* Complex datatype manipulation */ + case Token("[", child :: ordinal :: Nil) => + GetItem(nodeToExpr(child), nodeToExpr(ordinal)) + + /* 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)) + case Token("TOK_FUNCTIONSTAR", Token(name, Nil) :: args) => + UnresolvedFunction(name, Star(None) :: Nil) + + /* Literals */ + 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) => + Literal(strings.map(s => BaseSemanticAnalyzer.unescapeSQLString(s.getText)).mkString) + + // 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")) { + // Literal bigint. + 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")) { + // Literal tinyint. + 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) + BigDecimal(strVal) + } else { + 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) { + sys.error(s"Failed to parse number ${ast.getText}") + } 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} : + |${dumpTree(a).toString}" + + """.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, 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 { + 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(_, builder, indent + 1)) + builder + } +} 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 new file mode 100644 index 0000000000000..92d84208ab3f9 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala @@ -0,0 +1,164 @@ +/* + * 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 + +import catalyst.expressions._ +import catalyst.planning._ +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. + self: SQLContext#SparkPlanner => + + 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))(hiveContext) :: Nil + case _ => Nil + } + } + + 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)(hiveContext) :: Nil + case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => + InsertIntoParquetTable(table, planLater(child))(hiveContext.sparkContext) :: 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)(hiveContext) :: Nil + case m: MetastoreRelation => + HiveTableScan(m.output, m, None)(hiveContext) :: Nil + case _ => Nil + } + } + + /** + * 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.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))(hiveContext) + + otherPredicates + .reduceLeftOption(And) + .map(Filter(_, scan)) + .getOrElse(scan) :: Nil + + case _ => + Nil + } + } + + /** + * 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[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) + 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.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 { + case scan: HiveTableScan => + scan.copy(attributes = prunedCols)(hiveContext) + }) + } else { + val scan = relation match { + case MetastoreRelation(_, _, _) => { + HiveTableScan( + prunedCols, + relation.asInstanceOf[MetastoreRelation], + None)(hiveContext) + } + case ParquetRelation(_, _) => { + ParquetTableScan( + relation.output, + relation.asInstanceOf[ParquetRelation], + None)(hiveContext.sparkContext) + .pruneColumns(prunedCols) + } + } + predicateOpt.map(execution.Filter(_, scan)).getOrElse(scan) :: Nil + } + + if (isSimpleProject(projectList) && prunedCols == projectRefs) { + filteredScans + } else { + filteredScans.view.map(execution.Project(projectList, _)) + } + + 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]) + } +} 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 new file mode 100644 index 0000000000000..f20e9d4de4cf2 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala @@ -0,0 +1,76 @@ +/* + * 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 + +import java.io.{InputStreamReader, BufferedReader} + +import catalyst.expressions._ +import org.apache.spark.sql.execution._ + +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: SparkPlan)(@transient sc: HiveContext) + 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").asInstanceOf[Array[Any]]) + curLine = reader.readLine() + } + } + } + readerThread.start() + val outputProjection = new Projection(input) + iter + .map(outputProjection) + // TODO: Use SerDe + .map(_.mkString("", "\t", "\n").getBytes).foreach(outputStream.write) + outputStream.close() + readerThread.join() + outputLines.toIterator + } + } +} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala new file mode 100644 index 0000000000000..71d751cbc424d --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -0,0 +1,243 @@ +/* + * 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 + +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.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. + */ +private[hive] 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. + */ +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 + // it is smaller than what Spark suggests. + private val _minSplitsPerRDD = math.max( + sc.hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinSplits) + + + // TODO: set aws s3 credentials. + + private val _broadcastedHiveConf = + sc.sparkContext.broadcast(new SerializableWritable(sc.hiveconf)) + + 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() + deserializer.initialize(hconf, tableDesc.getProperties) + + // Deserialize each Writable to get the row value. + iter.map { + case v: Writable => deserializer.deserialize(v) + case value => + sys.error(s"Unable to deserialize non-Writable: $value of ${value.getClass.getName}") + } + } + 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 + + 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]'. + 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) + 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](sc.sparkContext) + } 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(sc.hiveconf) + 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( + sc.sparkContext, + _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) + } + +} + +private[hive] 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/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala new file mode 100644 index 0000000000000..17ae4ef63c054 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -0,0 +1,341 @@ +/* + * 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 + +import java.io.File +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 +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 + +import org.apache.spark.{SparkContext, SparkConf} + +import catalyst.analysis._ +import catalyst.plans.logical.{LogicalPlan, NativeCommand} +import catalyst.util._ + +object TestHive + extends TestHiveContext(new SparkContext("local", "TestSQLContext", new SparkConf())) + +/** + * 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. + * + * 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 TestHive must be serialized. + */ +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 + // without restarting the JVM. + System.clearProperty("spark.driver.port") + System.clearProperty("spark.hostPort") + + 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") + /** The location of the hive source code. */ + lazy val hiveDevHome = envVarToFile("HIVE_DEV_HOME") + + // 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 + */ + private def envVarToFile(envVar: String): Option[File] = { + Option(System.getenv(envVar)).map(new File(_)) + } + + /** + * 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 contains "LOAD DATA") { + val testDataLocation = + hiveDevHome.map(_.getCanonicalPath).getOrElse(inRepoTests.getCanonicalPath) + cmd.replaceAll("\\.\\.", testDataLocation) + } else { + cmd + } + + val hiveFilesTemp = File.createTempFile("catalystHiveFiles", "") + hiveFilesTemp.delete() + hiveFilesTemp.mkdir() + + val inRepoTests = new File("src/test/resources/") + 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 + + 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 QueryExecution with special debug workflow. + */ + abstract class QueryExecution extends super.QueryExecution { + override lazy val analyzed = { + val describedTables = logical match { + case NativeCommand(describedTable(tbl)) => tbl :: Nil + case _ => Nil + } + + // Make sure any test tables referenced are loaded. + val referencedTables = + describedTables ++ + 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. + analyzer(logical) + } + } + + case class TestTable(name: String, commands: (()=>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 + * demand when a query are run against it. + */ + 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. + // /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 '${getHiveFile("data/files/kv1.txt")}' INTO TABLE src".cmd), + 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("dest1", + "CREATE TABLE IF NOT EXISTS dest1 (key INT, value STRING)".cmd), + TestTable("dest2", + "CREATE TABLE IF NOT EXISTS dest2 (key INT, value STRING)".cmd), + 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)") + for (ds <- Seq("2008-04-08", "2008-04-09"); hr <- Seq("11", "12")) { + runSqlHive( + s"""LOAD DATA LOCAL INPATH '${getHiveFile("data/files/kv1.txt")}' + |OVERWRITE INTO TABLE srcpart PARTITION (ds='$ds',hr='$hr') + """.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 '${getHiveFile("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 + import org.apache.hadoop.hive.serde2.thrift.ThriftDeserializer + import org.apache.hadoop.mapred.SequenceFileInputFormat + import org.apache.hadoop.mapred.SequenceFileOutputFormat + + 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) + + + 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) + |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 '${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}' + |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 '${getHiveFile("data/files/episodes.avro")}' INTO TABLE episodes".cmd + ) + ) + + hiveQTestUtilTables.foreach(registerTestTable) + + private val loadedTables = new collection.mutable.HashSet[String] + + 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(_()) + } + } + + /** + * Records the UDFs present when the server starts, so we can delete ones that are created by + * tests. + */ + protected val originalUdfs: JavaSet[String] = FunctionRegistry.getFunctionNames + + /** + * Resets the test instance by deleting any tables that have been created. + * TODO: also clear out UDFs, views, etc. + */ + def reset() { + try { + // HACK: Hive is too noisy by default. + 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. + 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") + // Lots of tests fail if we do not change the partition whitelist from the default. + runSqlHive("set hive.metastore.partition.name.whitelist.pattern=.*") + + loadedTables.clear() + catalog.client.getAllTables("default").foreach { t => + logger.debug(s"Deleting table $t") + val table = catalog.client.getTable("default", t) + + catalog.client.getIndexes("default", t, 255).foreach { index => + catalog.client.dropIndex("default", t, index.getIndexName, true) + } + + if (!table.isIndexTable) { + catalog.client.dropTable("default", t) + } + } + + catalog.client.getAllDatabases.filterNot(_ == "default").foreach { db => + logger.debug(s"Dropping Database: $db") + catalog.client.dropDatabase(db, true, false, true) + } + + FunctionRegistry.getFunctionNames.filterNot(originalUdfs.contains(_)).foreach { udfName => + FunctionRegistry.unregisterTemporaryUDF(udfName) + } + + 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") + loadTestTable("srcpart") + } 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. + Thread.sleep(100000) + } + } +} 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 new file mode 100644 index 0000000000000..d20fd87f34f48 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala @@ -0,0 +1,356 @@ +/* + * 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 + +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} +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.fs.Path +import org.apache.hadoop.io.Writable +import org.apache.hadoop.mapred._ + +import catalyst.expressions._ +import catalyst.types.{BooleanType, DataType} +import org.apache.spark.{TaskContext, SparkException} +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._ + +/** + * 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])( + @transient val sc: HiveContext) + extends LeafNode + with HiveInspectors { + + 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 => + require( + pred.dataType == BooleanType, + s"Data type of predicate $pred must be BooleanType rather than ${pred.dataType}.") + + BindReferences.bindReference(pred, relation.partitionKeys) + } + + @transient + val hadoopReader = new HadoopTableReader(relation.tableDesc, sc) + + /** + * The hive object inspector for this table, which can be used to extract values from the + * serialized row representation. + */ + @transient + lazy val objectInspector = + relation.tableDesc.getDeserializer.getObjectInspector.asInstanceOf[StructObjectInspector] + + /** + * 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] = { + attributes.map { a => + val ordinal = relation.partitionKeys.indexOf(a) + if (ordinal >= 0) { + (_: 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: Any, _: Array[String]) => { + val data = objectInspector.getStructFieldData(row, ref) + unwrapData(data, ref.getFieldObjectInspector) + } + } + } + } + + private def castFromString(value: String, dataType: DataType) = { + Cast(Literal(value), dataType).apply(null) + } + + @transient + def inputRdd = if (!relation.hiveQlTable.isPartitioned) { + hadoopReader.makeRDDForTable(relation.hiveQlTable) + } else { + 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[hive] 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) + } + + // 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.apply(row).asInstanceOf[Boolean] + } + } + } + + def execute() = { + inputRdd.map { row => + val values = row match { + case Array(deserializedRow: AnyRef, partitionKeys: Array[String]) => + attributeFunctions.map(_(deserializedRow, partitionKeys)) + case deserializedRow: AnyRef => + attributeFunctions.map(_(deserializedRow, Array.empty)) + } + 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 + }) + } + } + + def output = attributes +} + +case class InsertIntoHiveTable( + table: MetastoreRelation, + partition: Map[String, Option[String]], + child: SparkPlan, + overwrite: Boolean) + (@transient sc: HiveContext) + extends UnaryNode { + + val outputClass = newSerializer(table.tableDesc).getSerializedClass + @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] + serializer.initialize(null, tableDesc.getProperties) + serializer + } + + override def otherCopyArgs = sc :: Nil + + 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 (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) => + val wrappedSeq = s.map(wrap(_, oi.getListElementObjectInspector)) + seqAsJavaList(wrappedSeq) + case (obj, _) => obj + } + + def saveAsHiveFile( + rdd: RDD[Writable], + valueClass: Class[_], + fileSinkConf: FileSinkDesc, + conf: JobConf, + isCompressed: Boolean) { + 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) + 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") + fileSinkConf.setCompressed(true) + fileSinkConf.setCompressCodec(conf.get("mapred.output.compression.codec")) + fileSinkConf.setCompressType(conf.get("mapred.output.compression.type")) + } + conf.setOutputCommitter(classOf[FileOutputCommitter]) + FileOutputFormat.setOutputPath( + conf, + SparkHiveHadoopWriter.createPathFromString(fileSinkConf.getDirName, conf)) + + logger.debug("Saving as hadoop file of type " + valueClass.getSimpleName) + + val writer = new SparkHiveHadoopWriter(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.sparkContext.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 + * `org.apache.hadoop.mapred.OutputFormat` provided by the table definition. + */ + def execute() = { + val childRdd = child.execute() + assert(childRdd != null) + + // 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 + 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( + fileSinkConf.getTableInfo.getDeserializer.getObjectInspector, + ObjectInspectorCopyOption.JAVA) + .asInstanceOf[StructObjectInspector] + + iter.map { row => + // Casts Strings to HiveVarchars when necessary. + val fieldOIs = standardOI.getAllStructFieldRefs.map(_.getFieldObjectInspector) + val mappedRow = row.zip(fieldOIs).map(wrap) + + serializer.serialize(mappedRow.toArray, standardOI) + } + } + + // 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, + jobConf, + sc.hiveconf.getBoolean("hive.exec.compress.output", 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}" + // 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 + if (partition.nonEmpty) { + 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) + // 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, + isSkewedStoreAsSubdir) + } else { + db.loadTable( + outputPath, + qualifiedTableName, + overwrite, + holdDDLTime) + } + + // 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) + } +} 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 new file mode 100644 index 0000000000000..5e775d6a048de --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala @@ -0,0 +1,467 @@ +/* + * 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 + +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._ +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 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. + val functionInfo: FunctionInfo = Option(FunctionRegistry.getFunctionInfo(name)).getOrElse( + sys.error(s"Couldn't find function $name")) + + if (classOf[UDF].isAssignableFrom(functionInfo.getFunctionClass)) { + val function = createFunction[UDF](name) + 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, children) + } 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}") + } + } + + def javaClassToDataType(clz: Class[_]): DataType = clz match { + 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[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 => 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] => 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 + 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)) + } +} + +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] + + /** Converts hive types to native catalyst types. */ + def unwrap(a: Any): Any = a match { + case null => null + 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 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 + 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 + } +} + +abstract class HiveUdf + extends Expression with Logging with HiveFunctionFactory { + self: Product => + + type UDFType + type EvaluatedType = Any + + 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 + + @transient + protected lazy val method = + function.getResolver.getEvalMethod(children.map(_.dataType.toTypeInfo)) + + @transient + lazy val dataType = javaClassToDataType(method.getReturnType) + + 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], + 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) + } + + 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 bd: BigDecimal => new HiveDecimal(bd.underlying()) + case other: AnyRef => other + }).asInstanceOf[AnyRef] + } + } + } + + // TODO: Finish input output types. + 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) + } + + // Invoke the udf and unwrap the result. + unwrap(method.invoke(function, args: _*)) + } +} + +case class HiveGenericUdf( + name: String, + children: Seq[Expression]) extends HiveUdf with HiveInspectors { + import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ + type UDFType = GenericUDF + + @transient + protected lazy val argumentInspectors = children.map(_.dataType).map(toInspector) + + @transient + protected lazy val returnInspector = function.initialize(argumentInspectors.toArray) + + val dataType: DataType = inspectorToDataType(returnInspector) + + override def apply(input: Row): Any = { + returnInspector // Make sure initialized. + 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)) + } +} + +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)).toArray) + } + + /** Converts native catalyst types to the types expected by Hive */ + 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 + 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 m: Map[_,_] => + mapAsJavaMap(m.map { case (k, v) => wrap(k) -> wrap(v) }) + case null => null + } + + 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 + case BinaryType => PrimitiveObjectInspectorFactory.javaByteArrayObjectInspector + } + + def inspectorToDataType(inspector: ObjectInspector): DataType = inspector match { + case s: StructObjectInspector => + StructType(s.getAllStructFieldRefs.map(f => { + types.StructField( + f.getFieldName, inspectorToDataType(f.getFieldObjectInspector), nullable = true) + })) + case l: ListObjectInspector => ArrayType(inspectorToDataType(l.getListElementObjectInspector)) + case m: MapObjectInspector => + MapType( + inspectorToDataType(m.getMapKeyObjectInspector), + inspectorToDataType(m.getMapValueObjectInspector)) + 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 + } + + implicit class typeInfoConversions(dt: DataType) { + import org.apache.hadoop.hive.serde2.typeinfo._ + import TypeInfoFactory._ + + def toTypeInfo: TypeInfo = dt match { + case BinaryType => binaryTypeInfo + 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( + name: String, + children: Seq[Expression]) extends AggregateExpression + with HiveInspectors + with HiveFunctionFactory { + + type UDFType = AbstractGenericUDAFResolver + + protected lazy val resolver: AbstractGenericUDAFResolver = createFunction(name) + + protected lazy val objectInspector = { + resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) + .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) + } + + protected lazy val inspectors = 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(",")})" + + def newInstance = new HiveUdafFunction(name, children, this) +} + +/** + * 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 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 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. + */ +case class HiveGenericUdtf( + name: String, + aliasNames: Seq[String], + children: Seq[Expression]) + extends Generator with HiveInspectors with HiveFunctionFactory { + + override def references = children.flatMap(_.references).toSet + + @transient + protected lazy val function: GenericUDTF = createFunction(name) + + protected lazy val inputInspectors = children.map(_.dataType).map(toInspector) + + protected lazy val outputInspectors = { + val structInspector = function.initialize(inputInspectors.toArray) + structInspector.getAllStructFieldRefs.map(_.getFieldObjectInspector) + } + + protected lazy val outputDataTypes = outputInspectors.map(inspectorToDataType) + + 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 { + case (attrName, attrDataType) => + AttributeReference(attrName, attrDataType, nullable = true)() + } + } else { + outputDataTypes.zipWithIndex.map { + case (attrDataType, i) => + AttributeReference(s"c_$i", attrDataType, nullable = true)() + } + } + } + + 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 = inputProjection(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(",")})" +} + +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 = 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] + + override def apply(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) + + @transient + val inputProjection = new Projection(exprs) + + def update(input: Row): Unit = { + val inputs = inputProjection(input).asInstanceOf[Seq[AnyRef]].toArray + function.iterate(buffer, inputs) + } +} diff --git a/sql/hive/src/test/resources/log4j.properties b/sql/hive/src/test/resources/log4j.properties new file mode 100644 index 0000000000000..5e17e3b596ba1 --- /dev/null +++ b/sql/hive/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/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala new file mode 100644 index 0000000000000..4b45e698601eb --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -0,0 +1,126 @@ +/* + * 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 + +import java.io.File + +/** + * A set of test cases based on the big-data-benchmark. + * https://amplab.cs.berkeley.edu/benchmark/ + */ +class BigDataBenchmarkSuite extends HiveComparisonTest { + import TestHive._ + + 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) + + 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("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) + } +} \ No newline at end of file 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..a12ab23946ed1 --- /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/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 new file mode 100644 index 0000000000000..8a5b97b7a05ef --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala @@ -0,0 +1,379 @@ +/* + * 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 + +import java.io._ +import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} + +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 + * and catalyst, comparing the results. + * + * The "golden" results from Hive are cached in an retrieved both from the classpath and + * [[answerCache]] to speed up testing. + * + * 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("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. -Dspark.hive.testshard=0:4. + */ + val shardInfo = Option(System.getProperty("spark.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 `-Dspark.hive.skiptests=passed,hiveFailed` is specified and test cases listed + * in [[passedDirectory]] or [[hiveFailedDirectory]] will be skipped. + */ + val skipDirectories = + Option(System.getProperty("spark.hive.skiptests")) + .toSeq + .flatMap(_.split(",")) + .map(name => new File(targetDir, s"$suiteName.$name")) + + val runOnlyDirectories = + 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("src/test/resources/golden") + 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()) { + 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()) { + 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()) { + 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()) { + 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 = { + val digest = java.security.MessageDigest.getInstance("MD5") + digest.update(str.getBytes) + new java.math.BigInteger(1, digest.digest).toString(16) + } + + protected def prepareAnswer( + hiveQuery: TestHive.type#SqlQueryExecution, + answer: Seq[String]): Seq[String] = { + 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 = 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. + if (!isOrdered) answer.sorted else answer + } + 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", + "Owner:", + // 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) = + nonDeterministicLineIndicators.map(line contains _).reduceLeft(_||_) + + /** + * 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) { + // 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) { + logger.debug(s"=== HIVE TEST: $testCaseName ===") + + // 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 { + // MINOR HACK: You must run a query before calling reset the first time. + TestHive.sql("SHOW TABLES") + TestHive.reset() + + val hiveCacheFiles = queryList.zipWithIndex.map { + case (queryString, i) => + val cachedAnswerName = s"$testCaseName-$i-${getMd5(queryString)}" + new File(answerCache, cachedAnswerName) + } + + val hiveCachedResults = hiveCacheFiles.flatMap { cachedAnswerFile => + logger.debug(s"Looking for cached answer file $cachedAnswerFile.") + if (cachedAnswerFile.exists) { + Some(fileToString(cachedAnswerFile)) + } else { + logger.debug(s"File $cachedAnswerFile not found") + None + } + }.map { + case "" => Nil + case "\n" => Seq("") + case other => other.split("\n").toSeq + } + + val hiveResults: Seq[Seq[String]] = + if (hiveCachedResults.size == queryList.size) { + logger.info(s"Using answer cache for test: $testCaseName") + hiveCachedResults + } else { + + val hiveQueries = queryList.map(new TestHive.SqlQueryExecution(_)) + // Make sure we can at least parse everything before attempting hive execution. + 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. + 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.") + // Analyze the query with catalyst to ensure test tables are loaded. + val answer = hiveQuery.analyzed match { + case _: ExplainCommand => Nil // No need to execute EXPLAIN queries as we don't check the output. + case _ => TestHive.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) + } + }.toSeq + TestHive.reset() + + computedResults + } + + // Run w/ catalyst + val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => + val query = new TestHive.SqlQueryExecution(queryString) + try { (query, prepareAnswer(query, query.stringResult())) } catch { + case e: Exception => + val errorMessage = + s""" + |Failed to execute query using catalyst: + |Error: ${e.getMessage} + |${stackTraceToString(e)} + |$query + |== HIVE - ${hive.size} row(s) == + |${hive.mkString("\n")} + | + |$consoleTestCase + """.stripMargin + stringToFile(new File(failedDirectory, testCaseName), errorMessage + consoleTestCase) + fail(errorMessage) + } + }.toSeq + + (queryList, hiveResults, catalystResults).zipped.foreach { + case (query, hive, (hiveQuery, catalyst)) => + // Check that the results match unless its an EXPLAIN query. + val preparedHive = prepareAnswer(hiveQuery,hive) + + 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 + + val resultComparison = sideBySide(hivePrintOut, catalystPrintOut).mkString("\n") + + if (recomputeCache) { + logger.warn(s"Clearing cache files for failed test $testCaseName") + hiveCacheFiles.foreach(_.delete()) + } + + val errorMessage = + s""" + |Results do not match for $testCaseName: + |$hiveQuery\n${hiveQuery.analyzed.output.map(_.name).mkString("\t")} + |$resultComparison + """.stripMargin + + stringToFile(new File(wrongDirectory, testCaseName), errorMessage + consoleTestCase) + fail(errorMessage) + } + } + + // Touch passed file. + new FileOutputStream(new File(passedDirectory, testCaseName)).close() + } catch { + case tf: org.scalatest.exceptions.TestFailedException => throw tf + case originalException: Exception => + 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 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.") + // 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) + } + } + + // If the canary query didn't fail then the environment is still okay, so just throw the original exception. + throw originalException + } + } + } +} \ No newline at end of file diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala new file mode 100644 index 0000000000000..d010023f789d4 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -0,0 +1,708 @@ +/* + * 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 + + +import java.io._ + +import util._ + +/** + * Runs the test cases that are included in the hive distribution. + */ +class HiveCompatibilitySuite extends HiveQueryFileTest { + // TODO: bundle in jar files... get from classpath + 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. */ + override def blackList = Seq( + // These tests use hooks that are not on the classpath and thus break all subsequent execution. + "hook_order", + "hook_context", + "mapjoin_hook", + "multi_sahooks", + "overridden_confs", + "query_properties", + "sample10", + "updateAccessTime", + "index_compact_binary_search", + "bucket_num_reducers", + "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/machine specific test answers, breaks the caching mechanism. + "authorization_3", + "authorization_5", + "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", + "protectmode2", + "describe_table", + "describe_comment_nonascii", + "udf5", + "udf_java_method", + + // 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", + + // Cant run without local map/reduce. + "index_auto_update", + "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", + "index_auto_empty", + "index_auto_partitioned", + "index_auto_unused", + "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. + // http://stackoverflow.com/a/1573715 + "ops_comparison", + + // Tests that seems to never complete on hive... + "skewjoin", + "database", + + // 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", + + // Avro tests seem to change the output format permanently thus breaking the answer cache, until + // 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). + "uniquejoin", + + // Hive seems to get the wrong answer on some outer joins. MySQL agrees with catalyst. + "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", + + // No support for unpublished test udfs. + "autogen_colalias", + + // Hive does not support buckets. + ".*bucket.*", + + // No window support yet + ".*window.*", + + // Fails in hive with authorization errors. + "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 on whiteList or + * blacklist are implicitly marked as ignored. + */ + override def whiteList = Seq( + "add_part_exist", + "add_partition_no_whitelist", + "add_partition_with_whitelist", + "alias_casted_column", + "alter2", + "alter4", + "alter5", + "alter_index", + "alter_merge_2", + "alter_partition_format_loc", + "alter_partition_protect_mode", + "alter_partition_with_whitelist", + "alter_table_serde", + "alter_varchar2", + "alter_view_as_select", + "ambiguous_col", + "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_join27", + "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", + "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", + "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", + "correlationoptimizer2", + "correlationoptimizer3", + "correlationoptimizer4", + "correlationoptimizer6", + "correlationoptimizer7", + "correlationoptimizer8", + "count", + "create_like_view", + "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_database_json", + "describe_pretty", + "describe_syntax", + "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_partitions_ignore_protection", + "drop_table", + "drop_table2", + "drop_view", + "escape_clusterby1", + "escape_distributeby1", + "escape_orderby1", + "escape_sortby1", + "fetch_aggregation", + "filter_join_breaktask", + "filter_join_breaktask2", + "groupby1", + "groupby11", + "groupby1_map", + "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", + "groupby7_map_skew", + "groupby7_noskew", + "groupby8_map", + "groupby8_map_skew", + "groupby8_noskew", + "groupby_distinct_samekey", + "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", + "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", + "input6", + "input7", + "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", + "insert_compressed", + "join0", + "join1", + "join10", + "join11", + "join12", + "join13", + "join14", + "join14_hadoop20", + "join15", + "join16", + "join17", + "join18", + "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_1to1", + "join_array", + "join_casesensitive", + "join_empty", + "join_filters", + "join_hive_626", + "join_nulls", + "join_reorder2", + "join_reorder3", + "join_reorder4", + "join_star", + "join_view", + "lateral_view_cp", + "lateral_view_ppd", + "lineage1", + "literal_double", + "literal_ints", + "literal_string", + "load_dyn_part7", + "load_file_with_space_in_the_name", + "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", + "nomore_ambiguous_table_col", + "nonblock_op_deduplicate", + "notable_alias1", + "notable_alias2", + "nullgroup", + "nullgroup2", + "nullgroup3", + "nullgroup4", + "nullgroup4_multi_distinct", + "nullgroup5", + "nullinput", + "nullinput2", + "nullscript", + "optional_outer", + "order", + "order2", + "outer_join_ppr", + "part_inherit_tbl_props", + "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", + "ppd_outer_join4", + "ppd_outer_join5", + "ppd_random", + "ppd_repeated_alias", + "ppd_udf_col", + "ppd_union", + "ppr_allchildsarenull", + "ppr_pushdown", + "ppr_pushdown2", + "ppr_pushdown3", + "progress_1", + "protectmode", + "push_or", + "query_with_semi", + "quote1", + "quote2", + "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", + "show_functions", + "show_partitions", + "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", + "stats_empty_partition", + "subq2", + "tablename_with_select", + "touch", + "type_widening", + "udaf_collect_set", + "udaf_corr", + "udaf_covar_pop", + "udaf_covar_samp", + "udf2", + "udf6", + "udf9", + "udf_10_trims", + "udf_E", + "udf_PI", + "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", + "udf_date_add", + "udf_date_sub", + "udf_datediff", + "udf_day", + "udf_dayofmonth", + "udf_degrees", + "udf_div", + "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_if", + "udf_index", + "udf_int", + "udf_isnotnull", + "udf_isnull", + "udf_java_method", + "udf_lcase", + "udf_length", + "udf_lessthan", + "udf_lessthanorequal", + "udf_like", + "udf_ln", + "udf_log", + "udf_log10", + "udf_log2", + "udf_lower", + "udf_lpad", + "udf_ltrim", + "udf_map", + "udf_minute", + "udf_modulo", + "udf_month", + "udf_negative", + "udf_not", + "udf_notequal", + "udf_notop", + "udf_nvl", + "udf_or", + "udf_parse_url", + "udf_positive", + "udf_pow", + "udf_power", + "udf_radians", + "udf_rand", + "udf_regexp", + "udf_regexp_extract", + "udf_regexp_replace", + "udf_repeat", + "udf_rlike", + "udf_round", + "udf_round_3", + "udf_rpad", + "udf_rtrim", + "udf_second", + "udf_sign", + "udf_sin", + "udf_smallint", + "udf_space", + "udf_sqrt", + "udf_std", + "udf_stddev", + "udf_stddev_pop", + "udf_stddev_samp", + "udf_string", + "udf_substring", + "udf_subtract", + "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", + "udf_upper", + "udf_var_pop", + "udf_var_samp", + "udf_variance", + "udf_weekofyear", + "udf_when", + "udf_xpath", + "udf_xpath_boolean", + "udf_xpath_double", + "udf_xpath_float", + "udf_xpath_int", + "udf_xpath_long", + "udf_xpath_short", + "udf_xpath_string", + "unicode_notation", + "union10", + "union11", + "union13", + "union14", + "union15", + "union16", + "union17", + "union18", + "union19", + "union2", + "union20", + "union22", + "union23", + "union24", + "union26", + "union27", + "union28", + "union29", + "union30", + "union31", + "union34", + "union4", + "union5", + "union6", + "union7", + "union8", + "union9", + "union_lateralview", + "union_ppr", + "union_remove_3", + "union_remove_6", + "union_script", + "varchar_2", + "varchar_join1", + "varchar_union1" + ) +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala new file mode 100644 index 0000000000000..f0a4ec3c02fa4 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -0,0 +1,70 @@ +/* + * 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 + +import java.io._ + +import catalyst.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 HiveComparisonTest { + /** 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: Seq[(String, File)] + + val runAll = + !(System.getProperty("spark.hive.alltests") == null) || + runOnlyDirectories.nonEmpty || + skipDirectories.nonEmpty + + 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) + + // Go through all the test cases and add them to scala test. + testCases.sorted.foreach { + case (testCaseName, testCaseFile) => + if (blackList.map(_.r.pattern.matcher(testCaseName).matches()).reduceLeft(_||_)) { + 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) + createQueryTest(testCaseName, queriesString) + } 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 && !runAll) + ignore(testCaseName) {} + } + } +} \ No newline at end of file diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala new file mode 100644 index 0000000000000..28a5d260b3185 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -0,0 +1,144 @@ +/* + * 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 test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. + */ +class HiveQuerySuite extends HiveComparisonTest { + import TestHive._ + + createQueryTest("Simple Average", + "SELECT AVG(key) FROM src") + + createQueryTest("Simple Average + 1", + "SELECT AVG(key) + 1.0 FROM src") + + createQueryTest("Simple Average + 1 with group", + "SELECT AVG(key) + 1.0, value FROM src group by value") + + createQueryTest("string literal", + "SELECT 'test' FROM src") + + createQueryTest("Escape sequences", + """SELECT key, '\\\t\\' FROM src WHERE key = 86""") + + createQueryTest("IgnoreExplain", + """EXPLAIN SELECT key FROM src""") + + createQueryTest("trivial join where clause", + "SELECT * FROM src a JOIN src b WHERE a.key = b.key") + + createQueryTest("trivial join ON clause", + "SELECT * FROM src a JOIN src b ON a.key = b.key") + + 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") + + ignore("partitioned table scan") { + 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; + |SELECT * FROM createdtable + """.stripMargin) + + createQueryTest("create table as with db name", + """ + |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("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") + + 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") + } + + 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))") + + 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 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala new file mode 100644 index 0000000000000..0dd79faa15396 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -0,0 +1,65 @@ +/* + * 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 test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. + */ +class HiveResolutionSuite extends HiveComparisonTest { + import TestHive._ + + createQueryTest("table.attr", + "SELECT src.key FROM src ORDER BY key LIMIT 1") + + createQueryTest("database.table", + "SELECT key FROM default.src ORDER BY key LIMIT 1") + + createQueryTest("database.table table.attr", + "SELECT src.key FROM default.src ORDER BY key LIMIT 1") + + createQueryTest("alias.attr", + "SELECT a.key FROM src a ORDER BY key LIMIT 1") + + createQueryTest("subquery-alias.attr", + "SELECT a.key FROM (SELECT * FROM src ORDER BY key LIMIT 1) a") + + createQueryTest("quoted alias.attr", + "SELECT `a`.`key` FROM src a ORDER BY key LIMIT 1") + + createQueryTest("attr", + "SELECT key FROM src a ORDER BY key LIMIT 1") + + createQueryTest("alias.*", + "SELECT a.* FROM src a ORDER BY key LIMIT 1") + + /** + * Negative examples. Currently only left here for documentation purposes. + * TODO(marmbrus): Test that catalyst fails on these queries. + */ + + /* SemanticException [Error 10009]: Line 1:7 Invalid table alias 'src' + createQueryTest("table.*", + "SELECT src.* FROM src a ORDER BY key LIMIT 1") */ + + /* Invalid table alias or column reference 'src': (possible column names are: key, value) + createQueryTest("tableName.attr from aliased subquery", + "SELECT src.key FROM (SELECT * FROM src ORDER BY key LIMIT 1) a") */ + +} \ No newline at end of file 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/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala new file mode 100644 index 0000000000000..8542f42aa9f43 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -0,0 +1,164 @@ +/* + * 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 + +import scala.collection.JavaConversions._ + +import org.apache.spark.sql.hive.TestHive + +/** + * A set of test cases that validate partition and column pruning. + */ +class PruningSuite extends HiveComparisonTest { + // 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("Partiton pruning: non-partitioned table", + "SELECT value FROM src WHERE key IS NOT NULL", + Seq("value"), + Seq("value", "key"), + Seq.empty) + + 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"), + Seq( + Seq("2008-04-08", "11"), + Seq("2008-04-08", "12"))) + + createPruningTest("Partition pruning: with filter 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("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("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("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"), + 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(s"$testCaseName - pruning test") { + val plan = new TestHive.SqlQueryExecution(sql).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, "Output columns mismatch") + assert(actualScannedColumns sameElements expectedScannedColumns, "Scanned columns mismatch") + + assert( + actualPartValues.length === expectedPartValues.length, + "Partition value count mismatches") + + for ((actual, expected) <- actualPartValues.zip(expectedPartValues)) { + assert(actual sameElements expected, "Partition values mismatch") + } + } + + // Creates a query test to compare query results generated by Hive and Catalyst. + createQueryTest(s"$testCaseName - query test", sql) + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala new file mode 100644 index 0000000000000..ee90061c7cd0e --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -0,0 +1,161 @@ +/* + * 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.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.analysis.UnresolvedRelation +import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.hive.TestHive + + +class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { + + 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] = { + // call to resolve references in order to get CREATE TABLE AS to work + val query = TestHive + .parseSql(querystr) + val finalQuery = + if (tableName.nonEmpty && filename.nonEmpty) + resolveParquetTable(tableName.get, filename.get, query) + else + query + TestHive.executePlan(finalQuery) + .toRdd + .collect() + } + + // stores a query output to a Parquet file + def storeQuery(querystr: String, filename: String): Unit = { + val query = WriteToFile( + filename, + TestHive.parseSql(querystr)) + TestHive + .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 = { + TestHive.loadTestTable("src") // may not be loaded now + 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 + val relation = ParquetRelation.create( + filename, + TestHive.analyzer(child), + TestHive.sparkContext.hadoopConfiguration, + Some(tableName)) + InsertIntoTable( + relation.asInstanceOf[BaseRelation], + Map.empty, + child, + overwrite = false) + } else + op + } + } + + override def beforeAll() { + // write test data + ParquetTestData.writeFile + // Override initial Parquet test table + TestHive.catalog.registerTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) + } + + override def afterAll() { + 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 * FROM parquet.testsource") + assert(rdd != null) + assert(rdd.forall(_.size == 6)) + } + + test("Simple column projection + filter on Parquet table") { + val rdd = runQuery("SELECT myboolean, mylong FROM parquet.testsource WHERE myboolean=true") + 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)) + compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String")) + } + + 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") + 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[Int](_.apply(0) match { + case x: Int => x + case _ => 0 + }) + val rddOrig = runQuery("SELECT * FROM src").sortBy(_.getInt(0)) + 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: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") + } + counter = counter + 1 + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 5847b95e3f5d1..062b888e80b48 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -201,7 +201,7 @@ class StreamingContext private[streaming] ( /** * Create an input stream with any arbitrary user implemented network receiver. - * Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param receiver Custom implementation of NetworkReceiver */ def networkStream[T: ClassTag]( @@ -211,7 +211,7 @@ class StreamingContext private[streaming] ( /** * Create an input stream with any arbitrary user implemented actor receiver. - * Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * @param props Props object defining creation of the actor * @param name Name of the actor * @param storageLevel RDD storage level. Defaults to memory-only. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index 9c5b177c163c8..bd78bae8a5c51 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -46,7 +46,7 @@ object ReceiverSupervisorStrategy { * A receiver trait to be mixed in with your Actor to gain access to * the API for pushing received data into Spark Streaming for being processed. * - * Find more details at: http://spark-project.org/docs/latest/streaming-custom-receivers.html + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * * @example {{{ * class MyActor extends Actor with Receiver{ From e09139d9ca529a8f983a8b3e2a8158c3f3caa523 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 20 Mar 2014 22:31:11 -0700 Subject: [PATCH 116/397] Fix maven jenkins: Add explicit init for required tables in SQLQuerySuite Sorry! I added this test at the last minute and failed to run it in maven as well. Note that, this will probably not be sufficient to actually fix the maven jenkins build, as that does not use the dev/run-tests scripts. We will need to configure it to also run dev/download-hive-tests.sh. The other option would be to check in the tests as I suggested in the original PR. (I can do this if we agree its the right thing to do). Long term it would probably be a good idea to also have maven run some sort of test env setup script so that we can decouple the test environment from the jenkins configuration. Author: Michael Armbrust Closes #191 from marmbrus/fixMaven and squashes the following commits: 3366e37 [Michael Armbrust] Fix maven jenkins: Add explicit init for required tables in SQLQuerySuite --- .../src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 3 +++ 1 file changed, 3 insertions(+) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 5728313d6dd3d..656d89c644b70 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -31,6 +31,9 @@ import TestSQLContext._ import TestData._ class SQLQuerySuite extends QueryTest { + // Make sure the tables are loaded. + TestData + test("agg") { checkAnswer( sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"), From 7e17fe69f9c3dc4cac024ea483f5d5f34ee06203 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 21 Mar 2014 15:05:45 -0700 Subject: [PATCH 117/397] Add hive test files to repository. Remove download script. This PR removes our test dependence on files hosted at Berkeley by checking the test queries and answers into the repository. This should also fix the maven Jenkins build. I realize this is a *giant* commit. But size wise its actually pretty small. We are only looking at ~1.2Mb compressed (~30Mb uncompressed). Given that we already have a ~80Mb file permanently added to the spark code lineage, I do not think that this will change the developer experience significantly. Furthermore, I think it is good engineering practice to consider such test support files as "code", since changes to them would indicate a change in functionality. These files were only excluded from the initial PR as I wanted the diff to be readable. Author: Michael Armbrust Closes #199 from marmbrus/hiveTestFiles and squashes the following commits: b9b9b17 [Michael Armbrust] Add hive test files to repository. Remove download script. --- dev/download-hive-tests.sh | 4 - dev/run-tests | 3 - .../resources/data/conf/hive-log4j.properties | 78 + .../test/resources/data/conf/hive-site.xml | 197 + .../resources/data/files/SortCol1Col2.txt | 4 + .../resources/data/files/SortCol2Col1.txt | 5 + .../resources/data/files/SortDescCol1Col2.txt | 4 + .../resources/data/files/SortDescCol2Col1.txt | 4 + sql/hive/src/test/resources/data/files/T1.txt | 6 + sql/hive/src/test/resources/data/files/T2.txt | 6 + sql/hive/src/test/resources/data/files/T3.txt | 4 + .../test/resources/data/files/TestSerDe.jar | Bin 0 -> 3838 bytes .../test/resources/data/files/UserVisits.dat | 55 + .../resources/data/files/apache.access.2.log | 1 + .../resources/data/files/apache.access.log | 1 + .../resources/data/files/archive_corrupt.rc | Bin 0 -> 1374 bytes .../test/resources/data/files/array_table.txt | 2 + .../src/test/resources/data/files/binary.txt | 10 + .../src/test/resources/data/files/bool.txt | 33 + .../src/test/resources/data/files/complex.seq | Bin 0 -> 1606 bytes .../test/resources/data/files/covar_tab.txt | 6 + .../data/files/create_nested_type.txt | 4 + .../src/test/resources/data/files/csv.txt | 18 + .../test/resources/data/files/datatypes.txt | 3 + .../test/resources/data/files/dim-data.txt | 4 + .../test/resources/data/files/doctors.avro | Bin 0 -> 521 bytes .../src/test/resources/data/files/docurl.txt | 8 + .../src/test/resources/data/files/double.txt | 16 + .../test/resources/data/files/employee.dat | 13 + .../test/resources/data/files/employee2.dat | 7 + .../resources/data/files/employee_part.txt | 9 + .../src/test/resources/data/files/empty1.txt | 0 .../src/test/resources/data/files/empty2.txt | 0 .../test/resources/data/files/episodes.avro | Bin 0 -> 597 bytes .../test/resources/data/files/escapetest.txt | Bin 0 -> 254 bytes .../resources/data/files/ext_test/test.dat | 6 + .../test/resources/data/files/fact-data.txt | 10 + .../resources/data/files/flights_join.txt | 20 + .../resources/data/files/flights_tiny.txt | 137 + .../resources/data/files/flights_tiny.txt.1 | 137 + .../data/files/groupby_groupingid.txt | 6 + .../resources/data/files/grouping_sets.txt | 6 + .../resources/data/files/grouping_sets1.txt | 6 + .../resources/data/files/grouping_sets2.txt | 6 + .../resources/data/files/hive_626_bar.txt | 1 + .../resources/data/files/hive_626_count.txt | 1 + .../resources/data/files/hive_626_foo.txt | 1 + .../src/test/resources/data/files/in1.txt | 3 + .../src/test/resources/data/files/in2.txt | 3 + .../src/test/resources/data/files/in3.txt | 4 + .../src/test/resources/data/files/in4.txt | 7 + .../src/test/resources/data/files/in5.txt | 19 + .../src/test/resources/data/files/in6.txt | 19 + .../src/test/resources/data/files/in7.txt | 3 + .../src/test/resources/data/files/in8.txt | 6 + .../src/test/resources/data/files/in9.txt | 6 + .../resources/data/files/infer_const_type.txt | 3 + .../src/test/resources/data/files/int.txt | 12 + .../src/test/resources/data/files/json.txt | 1 + .../src/test/resources/data/files/kv1.seq | Bin 0 -> 10508 bytes .../data/files/kv1.string-sorted.txt | 500 + .../src/test/resources/data/files/kv1.txt | 500 + .../resources/data/files/kv1.val.sorted.txt | 500 + .../test/resources/data/files/kv1_broken.seq | Bin 0 -> 216 bytes .../src/test/resources/data/files/kv1_cb.txt | 500 + .../src/test/resources/data/files/kv1_cc.txt | 500 + .../resources/data/files/kv1kv2.cogroup.txt | 1000 ++ .../src/test/resources/data/files/kv2.txt | 500 + .../src/test/resources/data/files/kv3.txt | 25 + .../src/test/resources/data/files/kv4.txt | 1 + .../src/test/resources/data/files/kv5.txt | 24 + .../src/test/resources/data/files/kv6.txt | 100 + .../src/test/resources/data/files/kv7.txt | 38 + .../src/test/resources/data/files/kv8.txt | 75 + .../data/files/leftsemijoin_mr_t1.txt | 2 + .../data/files/leftsemijoin_mr_t2.txt | 20 + .../test/resources/data/files/lineitem.txt | 100 + .../resources/data/files/lt100.sorted.txt | 84 + .../src/test/resources/data/files/lt100.txt | 84 + .../resources/data/files/lt100.txt.deflate | 3 + .../test/resources/data/files/map_table.txt | 2 + .../resources/data/files/nested_complex.txt | 2 + .../src/test/resources/data/files/null.txt | 10 + .../test/resources/data/files/nullfile.txt | 0 .../src/test/resources/data/files/nulls.txt | 1002 ++ .../test/resources/data/files/orc_create.txt | 3 + .../data/files/orc_create_people.txt | 100 + .../src/test/resources/data/files/over10k | 9999 +++++++++++++++++ sql/hive/src/test/resources/data/files/over1k | 1049 ++ .../src/test/resources/data/files/part.rc | Bin 0 -> 3214 bytes .../src/test/resources/data/files/part.seq | Bin 0 -> 3598 bytes .../test/resources/data/files/part_tiny.txt | 26 + .../test/resources/data/files/person age.txt | 5 + .../data/files/primitive_type_arrays.txt | 1 + .../src/test/resources/data/files/pw17.txt | 4 + .../src/test/resources/data/files/sales.txt | 2 + .../files/sample-queryplan-in-history.txt | 1 + .../resources/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 + .../resources/data/files/smb_bucket_input.rc | Bin 0 -> 253 bytes .../resources/data/files/smb_bucket_input.txt | 7 + .../test/resources/data/files/smbbucket_1.rc | Bin 0 -> 208 bytes .../test/resources/data/files/smbbucket_1.txt | 5 + .../test/resources/data/files/smbbucket_2.rc | Bin 0 -> 206 bytes .../test/resources/data/files/smbbucket_2.txt | 4 + .../test/resources/data/files/smbbucket_3.rc | Bin 0 -> 222 bytes .../test/resources/data/files/smbbucket_3.txt | 6 + .../src/test/resources/data/files/source.txt | 4 + .../test/resources/data/files/srcbucket0.txt | 493 + .../test/resources/data/files/srcbucket1.txt | 507 + .../test/resources/data/files/srcbucket20.txt | 118 + .../test/resources/data/files/srcbucket21.txt | 120 + .../test/resources/data/files/srcbucket22.txt | 124 + .../test/resources/data/files/srcbucket23.txt | 138 + .../data/files/srcsortbucket1outof4.txt | 118 + .../data/files/srcsortbucket2outof4.txt | 120 + .../data/files/srcsortbucket3outof4.txt | 124 + .../data/files/srcsortbucket4outof4.txt | 138 + .../src/test/resources/data/files/string.txt | Bin 0 -> 93 bytes .../test/resources/data/files/symlink1.txt | 2 + .../test/resources/data/files/symlink2.txt | 1 + .../src/test/resources/data/files/tbl.txt | 10 + .../src/test/resources/data/files/test.dat | 6 + .../src/test/resources/data/files/test2.dat | 6 + .../src/test/resources/data/files/text-en.txt | 95 + .../src/test/resources/data/files/things.txt | 1 + .../src/test/resources/data/files/things2.txt | 1 + .../src/test/resources/data/files/tiny_a.txt | 1 + .../src/test/resources/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 + .../test/resources/data/files/union_input.txt | 8 + sql/hive/src/test/resources/data/files/v1.txt | 4 + sql/hive/src/test/resources/data/files/v2.txt | 9 + .../src/test/resources/data/files/vc1.txt | 3 + sql/hive/src/test/resources/data/files/x.txt | 2 + sql/hive/src/test/resources/data/files/y.txt | 1 + sql/hive/src/test/resources/data/files/z.txt | 1 + .../src/test/resources/data/metadb/.gitignore | 1 + .../src/test/resources/data/scripts/cat.py | 29 + .../test/resources/data/scripts/cat_error.py | 24 + .../data/scripts/doubleescapedtab.py | 24 + .../resources/data/scripts/dumpdata_script.py | 27 + .../test/resources/data/scripts/error_script | 26 + .../data/scripts/escapedcarriagereturn.py | 23 + .../resources/data/scripts/escapednewline.py | 23 + .../test/resources/data/scripts/escapedtab.py | 23 + .../resources/data/scripts/input20_script | 20 + .../test/resources/data/scripts/newline.py | 24 + .../resources/data/scripts/q_test_cleanup.sql | 10 + .../resources/data/scripts/q_test_init.sql | 132 + .../resources/data/scripts/test_init_file.sql | 1 + ...' + '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 + ...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 + ...ISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a | 309 + ...quences-0-2f25c33d97c43f3276171624d988a286 | 1 + ...Explain-0-85d398864d4aa1d0e10ffd668fdf1a59 | 0 .../LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 | 175 + ...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 + ...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 + ...erwrite-0-2e2de36571343c0ebaf02c0c0c61ba8c | 0 ...erwrite-1-548d5a45ef20e066df145944dce3415a | 0 ...erwrite-2-77baf70da8adf8b4b82d076fee937fe1 | 0 ...erwrite-3-b9002c1d71895be765575b62656d1928 | 525 + ...erwrite-4-3b8e0155870264d89d275a1d5bd345d0 | 0 ...erwrite-5-b9002c1d71895be765575b62656d1928 | 1 + ...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 .../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 + .../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 + .../ql/src/test/queries/positive/join1.q | 3 + .../ql/src/test/queries/positive/join2.q | 3 + .../ql/src/test/queries/positive/join3.q | 4 + .../ql/src/test/queries/positive/join4.q | 14 + .../ql/src/test/queries/positive/join5.q | 15 + .../ql/src/test/queries/positive/join6.q | 16 + .../ql/src/test/queries/positive/join7.q | 21 + .../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 + .../ql/src/test/queries/positive/subq.q | 4 + .../ql/src/test/queries/positive/udf1.q | 5 + .../ql/src/test/queries/positive/udf4.q | 1 + .../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 + .../ql/src/test/queries/positive/union.q | 6 + 11284 files changed, 230618 insertions(+), 7 deletions(-) delete mode 100755 dev/download-hive-tests.sh create mode 100644 sql/hive/src/test/resources/data/conf/hive-log4j.properties create mode 100644 sql/hive/src/test/resources/data/conf/hive-site.xml create mode 100644 sql/hive/src/test/resources/data/files/SortCol1Col2.txt create mode 100644 sql/hive/src/test/resources/data/files/SortCol2Col1.txt create mode 100644 sql/hive/src/test/resources/data/files/SortDescCol1Col2.txt create mode 100644 sql/hive/src/test/resources/data/files/SortDescCol2Col1.txt create mode 100644 sql/hive/src/test/resources/data/files/T1.txt create mode 100644 sql/hive/src/test/resources/data/files/T2.txt create mode 100644 sql/hive/src/test/resources/data/files/T3.txt create mode 100644 sql/hive/src/test/resources/data/files/TestSerDe.jar create mode 100644 sql/hive/src/test/resources/data/files/UserVisits.dat create mode 100644 sql/hive/src/test/resources/data/files/apache.access.2.log create mode 100644 sql/hive/src/test/resources/data/files/apache.access.log create mode 100644 sql/hive/src/test/resources/data/files/archive_corrupt.rc create mode 100644 sql/hive/src/test/resources/data/files/array_table.txt create mode 100644 sql/hive/src/test/resources/data/files/binary.txt create mode 100644 sql/hive/src/test/resources/data/files/bool.txt create mode 100644 sql/hive/src/test/resources/data/files/complex.seq create mode 100644 sql/hive/src/test/resources/data/files/covar_tab.txt create mode 100644 sql/hive/src/test/resources/data/files/create_nested_type.txt create mode 100644 sql/hive/src/test/resources/data/files/csv.txt create mode 100644 sql/hive/src/test/resources/data/files/datatypes.txt create mode 100644 sql/hive/src/test/resources/data/files/dim-data.txt create mode 100644 sql/hive/src/test/resources/data/files/doctors.avro create mode 100644 sql/hive/src/test/resources/data/files/docurl.txt create mode 100644 sql/hive/src/test/resources/data/files/double.txt create mode 100644 sql/hive/src/test/resources/data/files/employee.dat create mode 100644 sql/hive/src/test/resources/data/files/employee2.dat create mode 100644 sql/hive/src/test/resources/data/files/employee_part.txt create mode 100644 sql/hive/src/test/resources/data/files/empty1.txt create mode 100644 sql/hive/src/test/resources/data/files/empty2.txt create mode 100644 sql/hive/src/test/resources/data/files/episodes.avro create mode 100644 sql/hive/src/test/resources/data/files/escapetest.txt create mode 100644 sql/hive/src/test/resources/data/files/ext_test/test.dat create mode 100644 sql/hive/src/test/resources/data/files/fact-data.txt create mode 100644 sql/hive/src/test/resources/data/files/flights_join.txt create mode 100644 sql/hive/src/test/resources/data/files/flights_tiny.txt create mode 100644 sql/hive/src/test/resources/data/files/flights_tiny.txt.1 create mode 100644 sql/hive/src/test/resources/data/files/groupby_groupingid.txt create mode 100644 sql/hive/src/test/resources/data/files/grouping_sets.txt create mode 100644 sql/hive/src/test/resources/data/files/grouping_sets1.txt create mode 100644 sql/hive/src/test/resources/data/files/grouping_sets2.txt create mode 100644 sql/hive/src/test/resources/data/files/hive_626_bar.txt create mode 100644 sql/hive/src/test/resources/data/files/hive_626_count.txt create mode 100644 sql/hive/src/test/resources/data/files/hive_626_foo.txt create mode 100644 sql/hive/src/test/resources/data/files/in1.txt create mode 100644 sql/hive/src/test/resources/data/files/in2.txt create mode 100644 sql/hive/src/test/resources/data/files/in3.txt create mode 100644 sql/hive/src/test/resources/data/files/in4.txt create mode 100644 sql/hive/src/test/resources/data/files/in5.txt create mode 100644 sql/hive/src/test/resources/data/files/in6.txt create mode 100644 sql/hive/src/test/resources/data/files/in7.txt create mode 100644 sql/hive/src/test/resources/data/files/in8.txt create mode 100644 sql/hive/src/test/resources/data/files/in9.txt create mode 100644 sql/hive/src/test/resources/data/files/infer_const_type.txt create mode 100644 sql/hive/src/test/resources/data/files/int.txt create mode 100644 sql/hive/src/test/resources/data/files/json.txt create mode 100644 sql/hive/src/test/resources/data/files/kv1.seq create mode 100644 sql/hive/src/test/resources/data/files/kv1.string-sorted.txt create mode 100644 sql/hive/src/test/resources/data/files/kv1.txt create mode 100644 sql/hive/src/test/resources/data/files/kv1.val.sorted.txt create mode 100644 sql/hive/src/test/resources/data/files/kv1_broken.seq create mode 100644 sql/hive/src/test/resources/data/files/kv1_cb.txt create mode 100644 sql/hive/src/test/resources/data/files/kv1_cc.txt create mode 100644 sql/hive/src/test/resources/data/files/kv1kv2.cogroup.txt create mode 100644 sql/hive/src/test/resources/data/files/kv2.txt create mode 100644 sql/hive/src/test/resources/data/files/kv3.txt create mode 100644 sql/hive/src/test/resources/data/files/kv4.txt create mode 100644 sql/hive/src/test/resources/data/files/kv5.txt create mode 100644 sql/hive/src/test/resources/data/files/kv6.txt create mode 100644 sql/hive/src/test/resources/data/files/kv7.txt create mode 100644 sql/hive/src/test/resources/data/files/kv8.txt create mode 100644 sql/hive/src/test/resources/data/files/leftsemijoin_mr_t1.txt create mode 100644 sql/hive/src/test/resources/data/files/leftsemijoin_mr_t2.txt create mode 100644 sql/hive/src/test/resources/data/files/lineitem.txt create mode 100644 sql/hive/src/test/resources/data/files/lt100.sorted.txt create mode 100644 sql/hive/src/test/resources/data/files/lt100.txt create mode 100644 sql/hive/src/test/resources/data/files/lt100.txt.deflate create mode 100644 sql/hive/src/test/resources/data/files/map_table.txt create mode 100644 sql/hive/src/test/resources/data/files/nested_complex.txt create mode 100644 sql/hive/src/test/resources/data/files/null.txt create mode 100644 sql/hive/src/test/resources/data/files/nullfile.txt create mode 100644 sql/hive/src/test/resources/data/files/nulls.txt create mode 100644 sql/hive/src/test/resources/data/files/orc_create.txt create mode 100644 sql/hive/src/test/resources/data/files/orc_create_people.txt create mode 100644 sql/hive/src/test/resources/data/files/over10k create mode 100644 sql/hive/src/test/resources/data/files/over1k create mode 100644 sql/hive/src/test/resources/data/files/part.rc create mode 100644 sql/hive/src/test/resources/data/files/part.seq create mode 100644 sql/hive/src/test/resources/data/files/part_tiny.txt create mode 100644 sql/hive/src/test/resources/data/files/person age.txt create mode 100644 sql/hive/src/test/resources/data/files/primitive_type_arrays.txt create mode 100644 sql/hive/src/test/resources/data/files/pw17.txt create mode 100644 sql/hive/src/test/resources/data/files/sales.txt create mode 100644 sql/hive/src/test/resources/data/files/sample-queryplan-in-history.txt create mode 100644 sql/hive/src/test/resources/data/files/sample-queryplan.txt create mode 100644 sql/hive/src/test/resources/data/files/smallsrcsortbucket1outof4.txt create mode 100644 sql/hive/src/test/resources/data/files/smallsrcsortbucket2outof4.txt create mode 100644 sql/hive/src/test/resources/data/files/smallsrcsortbucket3outof4.txt create mode 100644 sql/hive/src/test/resources/data/files/smallsrcsortbucket4outof4.txt create mode 100644 sql/hive/src/test/resources/data/files/smb_bucket_input.rc create mode 100644 sql/hive/src/test/resources/data/files/smb_bucket_input.txt create mode 100644 sql/hive/src/test/resources/data/files/smbbucket_1.rc create mode 100644 sql/hive/src/test/resources/data/files/smbbucket_1.txt create mode 100644 sql/hive/src/test/resources/data/files/smbbucket_2.rc create mode 100644 sql/hive/src/test/resources/data/files/smbbucket_2.txt create mode 100644 sql/hive/src/test/resources/data/files/smbbucket_3.rc create mode 100644 sql/hive/src/test/resources/data/files/smbbucket_3.txt create mode 100644 sql/hive/src/test/resources/data/files/source.txt create mode 100644 sql/hive/src/test/resources/data/files/srcbucket0.txt create mode 100644 sql/hive/src/test/resources/data/files/srcbucket1.txt create mode 100644 sql/hive/src/test/resources/data/files/srcbucket20.txt create mode 100644 sql/hive/src/test/resources/data/files/srcbucket21.txt create mode 100644 sql/hive/src/test/resources/data/files/srcbucket22.txt create mode 100644 sql/hive/src/test/resources/data/files/srcbucket23.txt create mode 100644 sql/hive/src/test/resources/data/files/srcsortbucket1outof4.txt create mode 100644 sql/hive/src/test/resources/data/files/srcsortbucket2outof4.txt create mode 100644 sql/hive/src/test/resources/data/files/srcsortbucket3outof4.txt create mode 100644 sql/hive/src/test/resources/data/files/srcsortbucket4outof4.txt create mode 100644 sql/hive/src/test/resources/data/files/string.txt create mode 100644 sql/hive/src/test/resources/data/files/symlink1.txt create mode 100644 sql/hive/src/test/resources/data/files/symlink2.txt create mode 100644 sql/hive/src/test/resources/data/files/tbl.txt create mode 100644 sql/hive/src/test/resources/data/files/test.dat create mode 100644 sql/hive/src/test/resources/data/files/test2.dat create mode 100644 sql/hive/src/test/resources/data/files/text-en.txt create mode 100644 sql/hive/src/test/resources/data/files/things.txt create mode 100644 sql/hive/src/test/resources/data/files/things2.txt create mode 100644 sql/hive/src/test/resources/data/files/tiny_a.txt create mode 100644 sql/hive/src/test/resources/data/files/tiny_b.txt create mode 100644 sql/hive/src/test/resources/data/files/types/primitives/090101.txt create mode 100644 sql/hive/src/test/resources/data/files/types/primitives/090201.txt create mode 100644 sql/hive/src/test/resources/data/files/types/primitives/090301.txt create mode 100644 sql/hive/src/test/resources/data/files/types/primitives/090401.txt create mode 100644 sql/hive/src/test/resources/data/files/union_input.txt create mode 100644 sql/hive/src/test/resources/data/files/v1.txt create mode 100644 sql/hive/src/test/resources/data/files/v2.txt create mode 100644 sql/hive/src/test/resources/data/files/vc1.txt create mode 100644 sql/hive/src/test/resources/data/files/x.txt create mode 100644 sql/hive/src/test/resources/data/files/y.txt create mode 100644 sql/hive/src/test/resources/data/files/z.txt create mode 100644 sql/hive/src/test/resources/data/metadb/.gitignore create mode 100644 sql/hive/src/test/resources/data/scripts/cat.py create mode 100644 sql/hive/src/test/resources/data/scripts/cat_error.py create mode 100644 sql/hive/src/test/resources/data/scripts/doubleescapedtab.py create mode 100644 sql/hive/src/test/resources/data/scripts/dumpdata_script.py create mode 100755 sql/hive/src/test/resources/data/scripts/error_script create mode 100644 sql/hive/src/test/resources/data/scripts/escapedcarriagereturn.py create mode 100644 sql/hive/src/test/resources/data/scripts/escapednewline.py create mode 100644 sql/hive/src/test/resources/data/scripts/escapedtab.py create mode 100755 sql/hive/src/test/resources/data/scripts/input20_script create mode 100644 sql/hive/src/test/resources/data/scripts/newline.py create mode 100644 sql/hive/src/test/resources/data/scripts/q_test_cleanup.sql create mode 100644 sql/hive/src/test/resources/data/scripts/q_test_init.sql create mode 100644 sql/hive/src/test/resources/data/scripts/test_init_file.sql create mode 100644 sql/hive/src/test/resources/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb create mode 100644 sql/hive/src/test/resources/golden/'1' + 1-0-130514c6116c311d808590a075b187b create mode 100644 sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e create mode 100644 sql/hive/src/test/resources/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 create mode 100644 sql/hive/src/test/resources/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 create mode 100644 sql/hive/src/test/resources/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a create mode 100644 sql/hive/src/test/resources/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a create mode 100644 sql/hive/src/test/resources/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 create mode 100644 sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 create mode 100644 sql/hive/src/test/resources/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e create mode 100644 sql/hive/src/test/resources/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e create mode 100644 sql/hive/src/test/resources/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f create mode 100644 sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 create mode 100644 sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 create mode 100644 sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d create mode 100644 sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 create mode 100644 sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 create mode 100644 sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 create mode 100644 sql/hive/src/test/resources/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a create mode 100644 sql/hive/src/test/resources/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d create mode 100644 sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 create mode 100644 sql/hive/src/test/resources/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f create mode 100644 sql/hive/src/test/resources/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 create mode 100644 sql/hive/src/test/resources/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 create mode 100644 sql/hive/src/test/resources/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 create mode 100644 sql/hive/src/test/resources/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 create mode 100644 sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e create mode 100644 sql/hive/src/test/resources/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 create mode 100644 sql/hive/src/test/resources/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e create mode 100644 sql/hive/src/test/resources/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b create mode 100644 sql/hive/src/test/resources/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb create mode 100644 sql/hive/src/test/resources/golden/1Y + 1-0-a4541db51882b19503649138fbb295f create mode 100644 sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d create mode 100644 sql/hive/src/test/resources/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 create mode 100644 sql/hive/src/test/resources/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 create mode 100644 sql/hive/src/test/resources/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 create mode 100644 sql/hive/src/test/resources/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be create mode 100644 sql/hive/src/test/resources/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 create mode 100644 sql/hive/src/test/resources/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa create mode 100644 sql/hive/src/test/resources/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 create mode 100644 sql/hive/src/test/resources/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 create mode 100644 sql/hive/src/test/resources/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee create mode 100644 sql/hive/src/test/resources/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 create mode 100644 sql/hive/src/test/resources/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 create mode 100644 sql/hive/src/test/resources/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a create mode 100644 sql/hive/src/test/resources/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 create mode 100644 sql/hive/src/test/resources/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 create mode 100644 sql/hive/src/test/resources/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 create mode 100644 sql/hive/src/test/resources/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 create mode 100644 sql/hive/src/test/resources/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 create mode 100644 sql/hive/src/test/resources/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb create mode 100644 sql/hive/src/test/resources/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 create mode 100644 sql/hive/src/test/resources/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 create mode 100644 sql/hive/src/test/resources/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 create mode 100644 sql/hive/src/test/resources/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 create mode 100644 sql/hive/src/test/resources/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 create mode 100644 sql/hive/src/test/resources/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 create mode 100644 sql/hive/src/test/resources/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa create mode 100644 sql/hive/src/test/resources/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 create mode 100644 sql/hive/src/test/resources/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 create mode 100644 sql/hive/src/test/resources/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb create mode 100644 sql/hive/src/test/resources/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b create mode 100644 sql/hive/src/test/resources/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 create mode 100644 sql/hive/src/test/resources/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 create mode 100644 sql/hive/src/test/resources/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 create mode 100644 sql/hive/src/test/resources/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef create mode 100644 sql/hive/src/test/resources/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c create mode 100644 sql/hive/src/test/resources/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 create mode 100644 sql/hive/src/test/resources/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af create mode 100644 sql/hive/src/test/resources/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 create mode 100644 sql/hive/src/test/resources/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f create mode 100644 sql/hive/src/test/resources/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 create mode 100644 sql/hive/src/test/resources/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 create mode 100644 sql/hive/src/test/resources/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 create mode 100644 sql/hive/src/test/resources/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 create mode 100644 sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d create mode 100644 sql/hive/src/test/resources/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 create mode 100644 sql/hive/src/test/resources/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 create mode 100644 sql/hive/src/test/resources/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 create mode 100644 sql/hive/src/test/resources/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 create mode 100644 sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 sql/hive/src/test/resources/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da create mode 100644 sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 sql/hive/src/test/resources/golden/alter2-12-75a213649242c2410ea6846f08c91d75 create mode 100644 sql/hive/src/test/resources/golden/alter2-13-ca51e03a3de391983429b6ad877e573c create mode 100644 sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75 create mode 100644 sql/hive/src/test/resources/golden/alter2-16-77500c83ffcece95511a4b21d67382dd create mode 100644 sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75 create mode 100644 sql/hive/src/test/resources/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 create mode 100644 sql/hive/src/test/resources/golden/alter2-2-75a213649242c2410ea6846f08c91d75 create mode 100644 sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 create mode 100644 sql/hive/src/test/resources/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 create mode 100644 sql/hive/src/test/resources/golden/alter2-23-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter2-24-775d69742a1c07df8da87e8a017d955 create mode 100644 sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5 create mode 100644 sql/hive/src/test/resources/golden/alter2-26-63f13c364546ddce5d2176c6604a948f create mode 100644 sql/hive/src/test/resources/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f create mode 100644 sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5 create mode 100644 sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f create mode 100644 sql/hive/src/test/resources/golden/alter2-3-ca51e03a3de391983429b6ad877e573c create mode 100644 sql/hive/src/test/resources/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae create mode 100644 sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5 create mode 100644 sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f create mode 100644 sql/hive/src/test/resources/golden/alter2-33-1934026d0228967097280eed35551f74 create mode 100644 sql/hive/src/test/resources/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b create mode 100644 sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5 create mode 100644 sql/hive/src/test/resources/golden/alter2-36-63f13c364546ddce5d2176c6604a948f create mode 100644 sql/hive/src/test/resources/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f create mode 100644 sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5 create mode 100644 sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f create mode 100644 sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 sql/hive/src/test/resources/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae create mode 100644 sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5 create mode 100644 sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f create mode 100644 sql/hive/src/test/resources/golden/alter2-43-1934026d0228967097280eed35551f74 create mode 100644 sql/hive/src/test/resources/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 sql/hive/src/test/resources/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c create mode 100644 sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75 create mode 100644 sql/hive/src/test/resources/golden/alter2-6-77500c83ffcece95511a4b21d67382dd create mode 100644 sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 create mode 100644 sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75 create mode 100644 sql/hive/src/test/resources/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e create mode 100644 sql/hive/src/test/resources/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade create mode 100644 sql/hive/src/test/resources/golden/alter3-1-75be487df30e301e156a22eee075633d create mode 100644 sql/hive/src/test/resources/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 create mode 100644 sql/hive/src/test/resources/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 create mode 100644 sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 create mode 100644 sql/hive/src/test/resources/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b create mode 100644 sql/hive/src/test/resources/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f create mode 100644 sql/hive/src/test/resources/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 create mode 100644 sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b create mode 100644 sql/hive/src/test/resources/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d create mode 100644 sql/hive/src/test/resources/golden/alter3-19-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f create mode 100644 sql/hive/src/test/resources/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 create mode 100644 sql/hive/src/test/resources/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 create mode 100644 sql/hive/src/test/resources/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 create mode 100644 sql/hive/src/test/resources/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 create mode 100644 sql/hive/src/test/resources/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 create mode 100644 sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face create mode 100644 sql/hive/src/test/resources/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e create mode 100644 sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 create mode 100644 sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 create mode 100644 sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c create mode 100644 sql/hive/src/test/resources/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 create mode 100644 sql/hive/src/test/resources/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 create mode 100644 sql/hive/src/test/resources/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 create mode 100644 sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 create mode 100644 sql/hive/src/test/resources/golden/alter3-4-399fc26b344c98ababa104522601c0cc create mode 100644 sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 create mode 100644 sql/hive/src/test/resources/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 create mode 100644 sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 create mode 100644 sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 create mode 100644 sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 create mode 100644 sql/hive/src/test/resources/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 create mode 100644 sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 create mode 100644 sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 create mode 100644 sql/hive/src/test/resources/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d create mode 100644 sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 create mode 100644 sql/hive/src/test/resources/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f create mode 100644 sql/hive/src/test/resources/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 sql/hive/src/test/resources/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 create mode 100644 sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 create mode 100644 sql/hive/src/test/resources/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d create mode 100644 sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 create mode 100644 sql/hive/src/test/resources/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 create mode 100644 sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter4-6-acd58e84952d310aeddf78579c36286 create mode 100644 sql/hive/src/test/resources/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 create mode 100644 sql/hive/src/test/resources/golden/alter4-8-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 create mode 100644 sql/hive/src/test/resources/golden/alter5-0-953553e14d835682fa47338dcfffe227 create mode 100644 sql/hive/src/test/resources/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b create mode 100644 sql/hive/src/test/resources/golden/alter5-10-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 create mode 100644 sql/hive/src/test/resources/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 create mode 100644 sql/hive/src/test/resources/golden/alter5-13-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 create mode 100644 sql/hive/src/test/resources/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b create mode 100644 sql/hive/src/test/resources/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d create mode 100644 sql/hive/src/test/resources/golden/alter5-17-9176dc5fb5206209fa907a289db1263d create mode 100644 sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf create mode 100644 sql/hive/src/test/resources/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 create mode 100644 sql/hive/src/test/resources/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d create mode 100644 sql/hive/src/test/resources/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 create mode 100644 sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf create mode 100644 sql/hive/src/test/resources/golden/alter5-3-2fc59e32c07186869811705c89aafadc create mode 100644 sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf create mode 100644 sql/hive/src/test/resources/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 create mode 100644 sql/hive/src/test/resources/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 create mode 100644 sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf create mode 100644 sql/hive/src/test/resources/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 create mode 100644 sql/hive/src/test/resources/golden/alter5-9-1e085f9741197e659413828c42386733 create mode 100644 sql/hive/src/test/resources/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 create mode 100644 sql/hive/src/test/resources/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e create mode 100644 sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 create mode 100644 sql/hive/src/test/resources/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb create mode 100644 sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 create mode 100644 sql/hive/src/test/resources/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 create mode 100644 sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc create mode 100644 sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed create mode 100644 sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d create mode 100644 sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 create mode 100644 sql/hive/src/test/resources/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 create mode 100644 sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 create mode 100644 sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c create mode 100644 sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c create mode 100644 sql/hive/src/test/resources/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 create mode 100644 sql/hive/src/test/resources/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad create mode 100644 sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b create mode 100644 sql/hive/src/test/resources/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 create mode 100644 sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b create mode 100644 sql/hive/src/test/resources/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 create mode 100644 sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b create mode 100644 sql/hive/src/test/resources/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 create mode 100644 sql/hive/src/test/resources/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d create mode 100644 sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb create mode 100644 sql/hive/src/test/resources/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a create mode 100644 sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 create mode 100644 sql/hive/src/test/resources/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d create mode 100644 sql/hive/src/test/resources/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab create mode 100644 sql/hive/src/test/resources/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b create mode 100644 sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 create mode 100644 sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 create mode 100644 sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 sql/hive/src/test/resources/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 create mode 100644 sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 create mode 100644 sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 create mode 100644 sql/hive/src/test/resources/golden/archive-16-892147913578bcf60620b7dd73893dd0 create mode 100644 sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 sql/hive/src/test/resources/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a create mode 100644 sql/hive/src/test/resources/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f create mode 100644 sql/hive/src/test/resources/golden/archive-2-713efc113418b01f76ffd589840193c8 create mode 100644 sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 create mode 100644 sql/hive/src/test/resources/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 create mode 100644 sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 create mode 100644 sql/hive/src/test/resources/golden/archive-23-892147913578bcf60620b7dd73893dd0 create mode 100644 sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 create mode 100644 sql/hive/src/test/resources/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f create mode 100644 sql/hive/src/test/resources/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a create mode 100644 sql/hive/src/test/resources/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 create mode 100644 sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 create mode 100644 sql/hive/src/test/resources/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 create mode 100644 sql/hive/src/test/resources/golden/archive-3-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a create mode 100644 sql/hive/src/test/resources/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 create mode 100644 sql/hive/src/test/resources/golden/archive-32-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/archive-4-3e95421993ab28d18245ec2340f580a3 create mode 100644 sql/hive/src/test/resources/golden/archive-5-c0c18ac884677231a41eea8d980d0451 create mode 100644 sql/hive/src/test/resources/golden/archive-6-528ab9750a558af7f1a43b3108e793dd create mode 100644 sql/hive/src/test/resources/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca create mode 100644 sql/hive/src/test/resources/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c create mode 100644 sql/hive/src/test/resources/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c create mode 100644 sql/hive/src/test/resources/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf create mode 100644 sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e create mode 100644 sql/hive/src/test/resources/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d create mode 100644 sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb create mode 100644 sql/hive/src/test/resources/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 create mode 100644 sql/hive/src/test/resources/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 sql/hive/src/test/resources/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca create mode 100644 sql/hive/src/test/resources/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c create mode 100644 sql/hive/src/test/resources/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf create mode 100644 sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba create mode 100644 sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 create mode 100644 sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c create mode 100644 sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a create mode 100644 sql/hive/src/test/resources/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e create mode 100644 sql/hive/src/test/resources/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 create mode 100644 sql/hive/src/test/resources/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 create mode 100644 sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e create mode 100644 sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 create mode 100644 sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 create mode 100644 sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 create mode 100644 sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 create mode 100644 sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 create mode 100644 sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 create mode 100644 sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 create mode 100644 sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 create mode 100644 sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 create mode 100644 sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e create mode 100644 sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 create mode 100644 sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 create mode 100644 sql/hive/src/test/resources/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 create mode 100644 sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 create mode 100644 sql/hive/src/test/resources/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 create mode 100644 sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f create mode 100644 sql/hive/src/test/resources/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 create mode 100644 sql/hive/src/test/resources/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 create mode 100644 sql/hive/src/test/resources/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 create mode 100644 sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 create mode 100644 sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 create mode 100644 sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf create mode 100644 sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 create mode 100644 sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 create mode 100644 sql/hive/src/test/resources/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 create mode 100644 sql/hive/src/test/resources/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 create mode 100644 sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c create mode 100644 sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b create mode 100644 sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f create mode 100644 sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 create mode 100644 sql/hive/src/test/resources/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 create mode 100644 sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f create mode 100644 sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca create mode 100644 sql/hive/src/test/resources/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 create mode 100644 sql/hive/src/test/resources/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a create mode 100644 sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 create mode 100644 sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd create mode 100644 sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 create mode 100644 sql/hive/src/test/resources/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee create mode 100644 sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 create mode 100644 sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 create mode 100644 sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f create mode 100644 sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 create mode 100644 sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb create mode 100644 sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join23-1-b31437533a2b890788938455cb32f679 create mode 100644 sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 create mode 100644 sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b create mode 100644 sql/hive/src/test/resources/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d create mode 100644 sql/hive/src/test/resources/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 create mode 100644 sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 create mode 100644 sql/hive/src/test/resources/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 create mode 100644 sql/hive/src/test/resources/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e create mode 100644 sql/hive/src/test/resources/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 create mode 100644 sql/hive/src/test/resources/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e create mode 100644 sql/hive/src/test/resources/golden/auto_join25-2-8180638a57b64557e02815c863031755 create mode 100644 sql/hive/src/test/resources/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e create mode 100644 sql/hive/src/test/resources/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 create mode 100644 sql/hive/src/test/resources/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 create mode 100644 sql/hive/src/test/resources/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f create mode 100644 sql/hive/src/test/resources/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca create mode 100644 sql/hive/src/test/resources/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a create mode 100644 sql/hive/src/test/resources/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 create mode 100644 sql/hive/src/test/resources/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 create mode 100644 sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 create mode 100644 sql/hive/src/test/resources/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 create mode 100644 sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 create mode 100644 sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e create mode 100644 sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c create mode 100644 sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 create mode 100644 sql/hive/src/test/resources/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 create mode 100644 sql/hive/src/test/resources/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 create mode 100644 sql/hive/src/test/resources/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e create mode 100644 sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d create mode 100644 sql/hive/src/test/resources/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 create mode 100644 sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f create mode 100644 sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a create mode 100644 sql/hive/src/test/resources/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d create mode 100644 sql/hive/src/test/resources/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba create mode 100644 sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e create mode 100644 sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 create mode 100644 sql/hive/src/test/resources/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 create mode 100644 sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 create mode 100644 sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e create mode 100644 sql/hive/src/test/resources/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f create mode 100644 sql/hive/src/test/resources/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f create mode 100644 sql/hive/src/test/resources/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d create mode 100644 sql/hive/src/test/resources/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b create mode 100644 sql/hive/src/test/resources/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c create mode 100644 sql/hive/src/test/resources/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b create mode 100644 sql/hive/src/test/resources/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a create mode 100644 sql/hive/src/test/resources/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc create mode 100644 sql/hive/src/test/resources/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 create mode 100644 sql/hive/src/test/resources/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd create mode 100644 sql/hive/src/test/resources/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf create mode 100644 sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef create mode 100644 sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee create mode 100644 sql/hive/src/test/resources/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a create mode 100644 sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd create mode 100644 sql/hive/src/test/resources/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a create mode 100644 sql/hive/src/test/resources/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 create mode 100644 sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 create mode 100644 sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd create mode 100644 sql/hive/src/test/resources/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 create mode 100644 sql/hive/src/test/resources/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 create mode 100644 sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 create mode 100644 sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd create mode 100644 sql/hive/src/test/resources/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 create mode 100644 sql/hive/src/test/resources/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 create mode 100644 sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 create mode 100644 sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 create mode 100644 sql/hive/src/test/resources/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e create mode 100644 sql/hive/src/test/resources/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 create mode 100644 sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 create mode 100644 sql/hive/src/test/resources/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd create mode 100644 sql/hive/src/test/resources/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab create mode 100644 sql/hive/src/test/resources/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca create mode 100644 sql/hive/src/test/resources/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 create mode 100644 sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 create mode 100644 sql/hive/src/test/resources/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 create mode 100644 sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b create mode 100644 sql/hive/src/test/resources/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 create mode 100644 sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed create mode 100644 sql/hive/src/test/resources/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 sql/hive/src/test/resources/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 create mode 100644 sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 sql/hive/src/test/resources/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 sql/hive/src/test/resources/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a create mode 100644 sql/hive/src/test/resources/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 create mode 100644 sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 sql/hive/src/test/resources/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 sql/hive/src/test/resources/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d create mode 100644 sql/hive/src/test/resources/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 sql/hive/src/test/resources/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b create mode 100644 sql/hive/src/test/resources/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 sql/hive/src/test/resources/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 create mode 100644 sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 create mode 100644 sql/hive/src/test/resources/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 create mode 100644 sql/hive/src/test/resources/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef create mode 100644 sql/hive/src/test/resources/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 create mode 100644 sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 create mode 100644 sql/hive/src/test/resources/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 sql/hive/src/test/resources/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 create mode 100644 sql/hive/src/test/resources/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e create mode 100644 sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 sql/hive/src/test/resources/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 create mode 100644 sql/hive/src/test/resources/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 create mode 100644 sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 create mode 100644 sql/hive/src/test/resources/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 create mode 100644 sql/hive/src/test/resources/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 create mode 100644 sql/hive/src/test/resources/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 create mode 100644 sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 create mode 100644 sql/hive/src/test/resources/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 create mode 100644 sql/hive/src/test/resources/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 create mode 100644 sql/hive/src/test/resources/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c create mode 100644 sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a create mode 100644 sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 create mode 100644 sql/hive/src/test/resources/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 create mode 100644 sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 create mode 100644 sql/hive/src/test/resources/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 create mode 100644 sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e create mode 100644 sql/hive/src/test/resources/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 create mode 100644 sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 create mode 100644 sql/hive/src/test/resources/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 create mode 100644 sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 create mode 100644 sql/hive/src/test/resources/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 create mode 100644 sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 create mode 100644 sql/hive/src/test/resources/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 create mode 100644 sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 create mode 100644 sql/hive/src/test/resources/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 create mode 100644 sql/hive/src/test/resources/golden/cluster-20-294891c1d956245540a80aa800ba393d create mode 100644 sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f create mode 100644 sql/hive/src/test/resources/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 create mode 100644 sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e create mode 100644 sql/hive/src/test/resources/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae create mode 100644 sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a create mode 100644 sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f create mode 100644 sql/hive/src/test/resources/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc create mode 100644 sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 create mode 100644 sql/hive/src/test/resources/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 create mode 100644 sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 create mode 100644 sql/hive/src/test/resources/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d create mode 100644 sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 create mode 100644 sql/hive/src/test/resources/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d create mode 100644 sql/hive/src/test/resources/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c create mode 100644 sql/hive/src/test/resources/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 create mode 100644 sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce create mode 100644 sql/hive/src/test/resources/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe create mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 create mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c create mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 create mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 create mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 create mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c create mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 create mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d create mode 100644 sql/hive/src/test/resources/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f create mode 100644 sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 create mode 100644 sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 create mode 100644 sql/hive/src/test/resources/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 create mode 100644 sql/hive/src/test/resources/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c create mode 100644 sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 create mode 100644 sql/hive/src/test/resources/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 sql/hive/src/test/resources/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a create mode 100644 sql/hive/src/test/resources/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 create mode 100644 sql/hive/src/test/resources/golden/combine2-12-cd15ffd140539cf86090814729ec4748 create mode 100644 sql/hive/src/test/resources/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 create mode 100644 sql/hive/src/test/resources/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c create mode 100644 sql/hive/src/test/resources/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 create mode 100644 sql/hive/src/test/resources/golden/combine2-16-557997716a68312e8cae75428e3ce31 create mode 100644 sql/hive/src/test/resources/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 create mode 100644 sql/hive/src/test/resources/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 create mode 100644 sql/hive/src/test/resources/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 sql/hive/src/test/resources/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 sql/hive/src/test/resources/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 sql/hive/src/test/resources/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 sql/hive/src/test/resources/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/combine2-7-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 create mode 100644 sql/hive/src/test/resources/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 create mode 100644 sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 sql/hive/src/test/resources/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c create mode 100644 sql/hive/src/test/resources/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 create mode 100644 sql/hive/src/test/resources/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af create mode 100644 sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 create mode 100644 sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 create mode 100644 sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 create mode 100644 sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b create mode 100644 sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc create mode 100644 sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 create mode 100644 sql/hive/src/test/resources/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a create mode 100644 sql/hive/src/test/resources/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b create mode 100644 sql/hive/src/test/resources/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f create mode 100644 sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 sql/hive/src/test/resources/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd create mode 100644 sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 create mode 100644 sql/hive/src/test/resources/golden/combine3-22-11025483569617a9f014b5defd71e933 create mode 100644 sql/hive/src/test/resources/golden/combine3-23-4725c48df09565618cbffd05953a5f62 create mode 100644 sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 sql/hive/src/test/resources/golden/combine3-6-4725c48df09565618cbffd05953a5f62 create mode 100644 sql/hive/src/test/resources/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 create mode 100644 sql/hive/src/test/resources/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b create mode 100644 sql/hive/src/test/resources/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c create mode 100644 sql/hive/src/test/resources/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e create mode 100644 sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d create mode 100644 sql/hive/src/test/resources/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce create mode 100644 sql/hive/src/test/resources/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d create mode 100644 sql/hive/src/test/resources/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f create mode 100644 sql/hive/src/test/resources/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b create mode 100644 sql/hive/src/test/resources/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf create mode 100644 sql/hive/src/test/resources/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 create mode 100644 sql/hive/src/test/resources/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d create mode 100644 sql/hive/src/test/resources/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b create mode 100644 sql/hive/src/test/resources/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f create mode 100644 sql/hive/src/test/resources/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a create mode 100644 sql/hive/src/test/resources/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc create mode 100644 sql/hive/src/test/resources/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 create mode 100644 sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 create mode 100644 sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f create mode 100644 sql/hive/src/test/resources/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b create mode 100644 sql/hive/src/test/resources/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 create mode 100644 sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 create mode 100644 sql/hive/src/test/resources/golden/count-11-29aa086fe75b55482a91316c38498565 create mode 100644 sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 create mode 100644 sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 create mode 100644 sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/count-4-590bf60b8d4dfa135f73dbb52180136f create mode 100644 sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 create mode 100644 sql/hive/src/test/resources/golden/count-6-29aa086fe75b55482a91316c38498565 create mode 100644 sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 create mode 100644 sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/count-9-590bf60b8d4dfa135f73dbb52180136f create mode 100644 sql/hive/src/test/resources/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 create mode 100644 sql/hive/src/test/resources/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 create mode 100644 sql/hive/src/test/resources/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e create mode 100644 sql/hive/src/test/resources/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 create mode 100644 sql/hive/src/test/resources/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 create mode 100644 sql/hive/src/test/resources/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 create mode 100644 sql/hive/src/test/resources/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac create mode 100644 sql/hive/src/test/resources/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 create mode 100644 sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f create mode 100644 sql/hive/src/test/resources/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 create mode 100644 sql/hive/src/test/resources/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 create mode 100644 sql/hive/src/test/resources/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c create mode 100644 sql/hive/src/test/resources/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd create mode 100644 sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 create mode 100644 sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d create mode 100644 sql/hive/src/test/resources/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 create mode 100644 sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba create mode 100644 sql/hive/src/test/resources/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 create mode 100644 sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 create mode 100644 sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a create mode 100644 sql/hive/src/test/resources/golden/create_1-14-437bd1acbae61f48e851addb769d3379 create mode 100644 sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 create mode 100644 sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 create mode 100644 sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 create mode 100644 sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 create mode 100644 sql/hive/src/test/resources/golden/create_1-4-610b82bf7b0080d293977927e5ef780c create mode 100644 sql/hive/src/test/resources/golden/create_1-5-c77b018276b1558c1d9462e0625e152e create mode 100644 sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a create mode 100644 sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 create mode 100644 sql/hive/src/test/resources/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 create mode 100644 sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 create mode 100644 sql/hive/src/test/resources/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d create mode 100644 sql/hive/src/test/resources/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee create mode 100644 sql/hive/src/test/resources/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e create mode 100644 sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 create mode 100644 sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 create mode 100644 sql/hive/src/test/resources/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 create mode 100644 sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 create mode 100644 sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 create mode 100644 sql/hive/src/test/resources/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b create mode 100644 sql/hive/src/test/resources/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 create mode 100644 sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 create mode 100644 sql/hive/src/test/resources/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 create mode 100644 sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 create mode 100644 sql/hive/src/test/resources/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a create mode 100644 sql/hive/src/test/resources/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 create mode 100644 sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc create mode 100644 sql/hive/src/test/resources/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 create mode 100644 sql/hive/src/test/resources/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 create mode 100644 sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c create mode 100644 sql/hive/src/test/resources/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 create mode 100644 sql/hive/src/test/resources/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e create mode 100644 sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 create mode 100644 sql/hive/src/test/resources/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 create mode 100644 sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af create mode 100644 sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 create mode 100644 sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 create mode 100644 sql/hive/src/test/resources/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a create mode 100644 sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 create mode 100644 sql/hive/src/test/resources/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 create mode 100644 sql/hive/src/test/resources/golden/create_like-5-dc9de26002604e9e436135bd4b40636d create mode 100644 sql/hive/src/test/resources/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 create mode 100644 sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 create mode 100644 sql/hive/src/test/resources/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb create mode 100644 sql/hive/src/test/resources/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb create mode 100644 sql/hive/src/test/resources/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b create mode 100644 sql/hive/src/test/resources/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a create mode 100644 sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb create mode 100644 sql/hive/src/test/resources/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb create mode 100644 sql/hive/src/test/resources/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c create mode 100644 sql/hive/src/test/resources/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f create mode 100644 sql/hive/src/test/resources/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b create mode 100644 sql/hive/src/test/resources/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b create mode 100644 sql/hive/src/test/resources/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d create mode 100644 sql/hive/src/test/resources/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f create mode 100644 sql/hive/src/test/resources/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af create mode 100644 sql/hive/src/test/resources/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 create mode 100644 sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 create mode 100644 sql/hive/src/test/resources/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 create mode 100644 sql/hive/src/test/resources/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 create mode 100644 sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 create mode 100644 sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 create mode 100644 sql/hive/src/test/resources/golden/create_nested_type-3-ac452c9279877935983c37113898e53c create mode 100644 sql/hive/src/test/resources/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff create mode 100644 sql/hive/src/test/resources/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d create mode 100644 sql/hive/src/test/resources/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea create mode 100644 sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 create mode 100644 sql/hive/src/test/resources/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d create mode 100644 sql/hive/src/test/resources/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de create mode 100644 sql/hive/src/test/resources/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd create mode 100644 sql/hive/src/test/resources/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e create mode 100644 sql/hive/src/test/resources/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 create mode 100644 sql/hive/src/test/resources/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 create mode 100644 sql/hive/src/test/resources/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd create mode 100644 sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e create mode 100644 sql/hive/src/test/resources/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 create mode 100644 sql/hive/src/test/resources/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 create mode 100644 sql/hive/src/test/resources/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd create mode 100644 sql/hive/src/test/resources/golden/create_view-11-2021c047203276dd2db51a56e672fcea create mode 100644 sql/hive/src/test/resources/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc create mode 100644 sql/hive/src/test/resources/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a create mode 100644 sql/hive/src/test/resources/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 create mode 100644 sql/hive/src/test/resources/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 create mode 100644 sql/hive/src/test/resources/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 create mode 100644 sql/hive/src/test/resources/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a create mode 100644 sql/hive/src/test/resources/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 create mode 100644 sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b create mode 100644 sql/hive/src/test/resources/golden/create_view-2-d80dcd1271ab264292e9938f3162427c create mode 100644 sql/hive/src/test/resources/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 create mode 100644 sql/hive/src/test/resources/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 create mode 100644 sql/hive/src/test/resources/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 create mode 100644 sql/hive/src/test/resources/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 create mode 100644 sql/hive/src/test/resources/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 create mode 100644 sql/hive/src/test/resources/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 create mode 100644 sql/hive/src/test/resources/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 create mode 100644 sql/hive/src/test/resources/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 create mode 100644 sql/hive/src/test/resources/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 create mode 100644 sql/hive/src/test/resources/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 create mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc create mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 create mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 create mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 create mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 create mode 100644 sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc create mode 100644 sql/hive/src/test/resources/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 create mode 100644 sql/hive/src/test/resources/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 create mode 100644 sql/hive/src/test/resources/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 create mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e create mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 create mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 create mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b create mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 create mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f create mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 create mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec create mode 100644 sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 create mode 100644 sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 create mode 100644 sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd create mode 100644 sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e create mode 100644 sql/hive/src/test/resources/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f create mode 100644 sql/hive/src/test/resources/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 create mode 100644 sql/hive/src/test/resources/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 create mode 100644 sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 create mode 100644 sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941 create mode 100644 sql/hive/src/test/resources/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 create mode 100644 sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 create mode 100644 sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 create mode 100644 sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 create mode 100644 sql/hive/src/test/resources/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 create mode 100644 sql/hive/src/test/resources/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba create mode 100644 sql/hive/src/test/resources/golden/database_drop-8-97101266791d2b2c662bcde549422318 create mode 100644 sql/hive/src/test/resources/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 create mode 100644 sql/hive/src/test/resources/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a create mode 100644 sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 create mode 100644 sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 create mode 100644 sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 create mode 100644 sql/hive/src/test/resources/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 create mode 100644 sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb create mode 100644 sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a create mode 100644 sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 create mode 100644 sql/hive/src/test/resources/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b create mode 100644 sql/hive/src/test/resources/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 create mode 100644 sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 create mode 100644 sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 create mode 100644 sql/hive/src/test/resources/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 create mode 100644 sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 create mode 100644 sql/hive/src/test/resources/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 create mode 100644 sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a create mode 100644 sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 create mode 100644 sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 create mode 100644 sql/hive/src/test/resources/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a create mode 100644 sql/hive/src/test/resources/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd create mode 100644 sql/hive/src/test/resources/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 create mode 100644 sql/hive/src/test/resources/golden/date_2-2-cab14d992c53c106ab257fae52001e04 create mode 100644 sql/hive/src/test/resources/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 create mode 100644 sql/hive/src/test/resources/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 create mode 100644 sql/hive/src/test/resources/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 create mode 100644 sql/hive/src/test/resources/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c create mode 100644 sql/hive/src/test/resources/golden/date_join1-1-3a68de2112a212a07a3068916c608fb create mode 100644 sql/hive/src/test/resources/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 create mode 100644 sql/hive/src/test/resources/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 create mode 100644 sql/hive/src/test/resources/golden/date_serde-1-36e6041f53433482631018410bb62a99 create mode 100644 sql/hive/src/test/resources/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 create mode 100644 sql/hive/src/test/resources/golden/date_serde-3-e6167e27465514356c557a77d956ea46 create mode 100644 sql/hive/src/test/resources/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 create mode 100644 sql/hive/src/test/resources/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c create mode 100644 sql/hive/src/test/resources/golden/date_serde-6-e00facec2986bc55074868eff87ba22a create mode 100644 sql/hive/src/test/resources/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d create mode 100644 sql/hive/src/test/resources/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 create mode 100644 sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd create mode 100644 sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 create mode 100644 sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b create mode 100644 sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a create mode 100644 sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d create mode 100644 sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c create mode 100644 sql/hive/src/test/resources/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee create mode 100644 sql/hive/src/test/resources/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b create mode 100644 sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d create mode 100644 sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e create mode 100644 sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff create mode 100644 sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e create mode 100644 sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e create mode 100644 sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a create mode 100644 sql/hive/src/test/resources/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee create mode 100644 sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 create mode 100644 sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d create mode 100644 sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e create mode 100644 sql/hive/src/test/resources/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef create mode 100644 sql/hive/src/test/resources/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 create mode 100644 sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 create mode 100644 sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 create mode 100644 sql/hive/src/test/resources/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 create mode 100644 sql/hive/src/test/resources/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 create mode 100644 sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b create mode 100644 sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 create mode 100644 sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b create mode 100644 sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 create mode 100644 sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 create mode 100644 sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 create mode 100644 sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 create mode 100644 sql/hive/src/test/resources/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f create mode 100644 sql/hive/src/test/resources/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 create mode 100644 sql/hive/src/test/resources/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d create mode 100644 sql/hive/src/test/resources/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e create mode 100644 sql/hive/src/test/resources/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c create mode 100644 sql/hive/src/test/resources/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c create mode 100644 sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e create mode 100644 sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 create mode 100644 sql/hive/src/test/resources/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 create mode 100644 sql/hive/src/test/resources/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 create mode 100644 sql/hive/src/test/resources/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 create mode 100644 sql/hive/src/test/resources/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e create mode 100644 sql/hive/src/test/resources/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e create mode 100644 sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab create mode 100644 sql/hive/src/test/resources/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 create mode 100644 sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c create mode 100644 sql/hive/src/test/resources/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 create mode 100644 sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be create mode 100644 sql/hive/src/test/resources/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba create mode 100644 sql/hive/src/test/resources/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 create mode 100644 sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 create mode 100644 sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 create mode 100644 sql/hive/src/test/resources/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 create mode 100644 sql/hive/src/test/resources/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb create mode 100644 sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 create mode 100644 sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a create mode 100644 sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 create mode 100644 sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 create mode 100644 sql/hive/src/test/resources/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 create mode 100644 sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 create mode 100644 sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 create mode 100644 sql/hive/src/test/resources/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad create mode 100644 sql/hive/src/test/resources/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa create mode 100644 sql/hive/src/test/resources/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce create mode 100644 sql/hive/src/test/resources/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 create mode 100644 sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b create mode 100644 sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c create mode 100644 sql/hive/src/test/resources/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e create mode 100644 sql/hive/src/test/resources/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 create mode 100644 sql/hive/src/test/resources/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f create mode 100644 sql/hive/src/test/resources/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb create mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a create mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b create mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 create mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 create mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c create mode 100644 sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 create mode 100644 sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab create mode 100644 sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 create mode 100644 sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 create mode 100644 sql/hive/src/test/resources/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a create mode 100644 sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 create mode 100644 sql/hive/src/test/resources/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 create mode 100644 sql/hive/src/test/resources/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 create mode 100644 sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 create mode 100644 sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 create mode 100644 sql/hive/src/test/resources/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 create mode 100644 sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 create mode 100644 sql/hive/src/test/resources/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa create mode 100644 sql/hive/src/test/resources/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 create mode 100644 sql/hive/src/test/resources/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f create mode 100644 sql/hive/src/test/resources/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 create mode 100644 sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 create mode 100644 sql/hive/src/test/resources/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 create mode 100644 sql/hive/src/test/resources/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa create mode 100644 sql/hive/src/test/resources/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 create mode 100644 sql/hive/src/test/resources/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 create mode 100644 sql/hive/src/test/resources/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 create mode 100644 sql/hive/src/test/resources/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 create mode 100644 sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 create mode 100644 sql/hive/src/test/resources/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 create mode 100644 sql/hive/src/test/resources/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d create mode 100644 sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f create mode 100644 sql/hive/src/test/resources/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 create mode 100644 sql/hive/src/test/resources/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 create mode 100644 sql/hive/src/test/resources/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 create mode 100644 sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf create mode 100644 sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 create mode 100644 sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 create mode 100644 sql/hive/src/test/resources/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 create mode 100644 sql/hive/src/test/resources/golden/escape1-3-4267651148da591da38737028fdbd80 create mode 100644 sql/hive/src/test/resources/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 create mode 100644 sql/hive/src/test/resources/golden/escape1-5-70729c3d79ded87e884c176138174645 create mode 100644 sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 create mode 100644 sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 create mode 100644 sql/hive/src/test/resources/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 create mode 100644 sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 create mode 100644 sql/hive/src/test/resources/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e create mode 100644 sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 create mode 100644 sql/hive/src/test/resources/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe create mode 100644 sql/hive/src/test/resources/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 create mode 100644 sql/hive/src/test/resources/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 create mode 100644 sql/hive/src/test/resources/golden/escape2-7-70729c3d79ded87e884c176138174645 create mode 100644 sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 create mode 100644 sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 create mode 100644 sql/hive/src/test/resources/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf create mode 100644 sql/hive/src/test/resources/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 create mode 100644 sql/hive/src/test/resources/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb create mode 100644 sql/hive/src/test/resources/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 create mode 100644 sql/hive/src/test/resources/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 create mode 100644 sql/hive/src/test/resources/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 create mode 100644 sql/hive/src/test/resources/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 create mode 100644 sql/hive/src/test/resources/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 create mode 100644 sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 sql/hive/src/test/resources/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 create mode 100644 sql/hive/src/test/resources/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 create mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d create mode 100644 sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 create mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 create mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 create mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 create mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e create mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 create mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 create mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 create mode 100644 sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 create mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc create mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 create mode 100644 sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 create mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a create mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 create mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 create mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd create mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d create mode 100644 sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 create mode 100644 sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 create mode 100644 sql/hive/src/test/resources/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 create mode 100644 sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 create mode 100644 sql/hive/src/test/resources/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 create mode 100644 sql/hive/src/test/resources/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b create mode 100644 sql/hive/src/test/resources/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e create mode 100644 sql/hive/src/test/resources/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 create mode 100644 sql/hive/src/test/resources/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 create mode 100644 sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 create mode 100644 sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e create mode 100644 sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 create mode 100644 sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby1-2-f90acd59ace31e16059bae52583188cc create mode 100644 sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d create mode 100644 sql/hive/src/test/resources/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 create mode 100644 sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 create mode 100644 sql/hive/src/test/resources/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea create mode 100644 sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 create mode 100644 sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 create mode 100644 sql/hive/src/test/resources/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 create mode 100644 sql/hive/src/test/resources/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 create mode 100644 sql/hive/src/test/resources/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f create mode 100644 sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd create mode 100644 sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d create mode 100644 sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 create mode 100644 sql/hive/src/test/resources/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 create mode 100644 sql/hive/src/test/resources/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 create mode 100644 sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe create mode 100644 sql/hive/src/test/resources/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 create mode 100644 sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc create mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 create mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea create mode 100644 sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 create mode 100644 sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b create mode 100644 sql/hive/src/test/resources/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 create mode 100644 sql/hive/src/test/resources/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 create mode 100644 sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c create mode 100644 sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 create mode 100644 sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 create mode 100644 sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 create mode 100644 sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c create mode 100644 sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 create mode 100644 sql/hive/src/test/resources/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 create mode 100644 sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 create mode 100644 sql/hive/src/test/resources/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b create mode 100644 sql/hive/src/test/resources/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 create mode 100644 sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 create mode 100644 sql/hive/src/test/resources/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b create mode 100644 sql/hive/src/test/resources/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 create mode 100644 sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 create mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 create mode 100644 sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 create mode 100644 sql/hive/src/test/resources/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e create mode 100644 sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 create mode 100644 sql/hive/src/test/resources/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f create mode 100644 sql/hive/src/test/resources/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 create mode 100644 sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 create mode 100644 sql/hive/src/test/resources/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f create mode 100644 sql/hive/src/test/resources/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 create mode 100644 sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 create mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e create mode 100644 sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac create mode 100644 sql/hive/src/test/resources/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 create mode 100644 sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac create mode 100644 sql/hive/src/test/resources/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 create mode 100644 sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac create mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 create mode 100644 sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac create mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 create mode 100644 sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 create mode 100644 sql/hive/src/test/resources/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 create mode 100644 sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd create mode 100644 sql/hive/src/test/resources/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 create mode 100644 sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a create mode 100644 sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 create mode 100644 sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 sql/hive/src/test/resources/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e create mode 100644 sql/hive/src/test/resources/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 create mode 100644 sql/hive/src/test/resources/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 create mode 100644 sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e create mode 100644 sql/hive/src/test/resources/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f create mode 100644 sql/hive/src/test/resources/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 create mode 100644 sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed create mode 100644 sql/hive/src/test/resources/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b create mode 100644 sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e create mode 100644 sql/hive/src/test/resources/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f create mode 100644 sql/hive/src/test/resources/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 create mode 100644 sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 create mode 100644 sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e create mode 100644 sql/hive/src/test/resources/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 create mode 100644 sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e create mode 100644 sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 create mode 100644 sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 create mode 100644 sql/hive/src/test/resources/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 create mode 100644 sql/hive/src/test/resources/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 create mode 100644 sql/hive/src/test/resources/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a create mode 100644 sql/hive/src/test/resources/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 create mode 100644 sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 create mode 100644 sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 create mode 100644 sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 create mode 100644 sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 create mode 100644 sql/hive/src/test/resources/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 create mode 100644 sql/hive/src/test/resources/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 create mode 100644 sql/hive/src/test/resources/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 create mode 100644 sql/hive/src/test/resources/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 create mode 100644 sql/hive/src/test/resources/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 create mode 100644 sql/hive/src/test/resources/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad create mode 100644 sql/hive/src/test/resources/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 create mode 100644 sql/hive/src/test/resources/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 create mode 100644 sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 create mode 100644 sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb create mode 100644 sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 create mode 100644 sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 create mode 100644 sql/hive/src/test/resources/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 create mode 100644 sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 create mode 100644 sql/hive/src/test/resources/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 create mode 100644 sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f create mode 100644 sql/hive/src/test/resources/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 create mode 100644 sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb create mode 100644 sql/hive/src/test/resources/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 create mode 100644 sql/hive/src/test/resources/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac create mode 100644 sql/hive/src/test/resources/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c create mode 100644 sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d create mode 100644 sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 create mode 100644 sql/hive/src/test/resources/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 create mode 100644 sql/hive/src/test/resources/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c create mode 100644 sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d create mode 100644 sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 create mode 100644 sql/hive/src/test/resources/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 create mode 100644 sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f create mode 100644 sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf create mode 100644 sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 create mode 100644 sql/hive/src/test/resources/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 create mode 100644 sql/hive/src/test/resources/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a create mode 100644 sql/hive/src/test/resources/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 create mode 100644 sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 create mode 100644 sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 create mode 100644 sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 create mode 100644 sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 create mode 100644 sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 create mode 100644 sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 create mode 100644 sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 create mode 100644 sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 create mode 100644 sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d create mode 100644 sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad create mode 100644 sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 create mode 100644 sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f create mode 100644 sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 create mode 100644 sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 create mode 100644 sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e create mode 100644 sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 create mode 100644 sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 create mode 100644 sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf create mode 100644 sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca create mode 100644 sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 create mode 100644 sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 create mode 100644 sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe create mode 100644 sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f create mode 100644 sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 create mode 100644 sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 create mode 100644 sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 create mode 100644 sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 create mode 100644 sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 create mode 100644 sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 create mode 100644 sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 create mode 100644 sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 create mode 100644 sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b create mode 100644 sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 create mode 100644 sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d create mode 100644 sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 create mode 100644 sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 create mode 100644 sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 create mode 100644 sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b create mode 100644 sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 create mode 100644 sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c create mode 100644 sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 create mode 100644 sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 create mode 100644 sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b create mode 100644 sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 create mode 100644 sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 create mode 100644 sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c create mode 100644 sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 create mode 100644 sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 create mode 100644 sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 create mode 100644 sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae create mode 100644 sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 create mode 100644 sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 create mode 100644 sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 create mode 100644 sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e create mode 100644 sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 create mode 100644 sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd create mode 100644 sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e create mode 100644 sql/hive/src/test/resources/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 create mode 100644 sql/hive/src/test/resources/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 create mode 100644 sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 create mode 100644 sql/hive/src/test/resources/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a create mode 100644 sql/hive/src/test/resources/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce create mode 100644 sql/hive/src/test/resources/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 create mode 100644 sql/hive/src/test/resources/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 create mode 100644 sql/hive/src/test/resources/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b create mode 100644 sql/hive/src/test/resources/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 create mode 100644 sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 create mode 100644 sql/hive/src/test/resources/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 create mode 100644 sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 create mode 100644 sql/hive/src/test/resources/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 create mode 100644 sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f create mode 100644 sql/hive/src/test/resources/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 create mode 100644 sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 create mode 100644 sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 create mode 100644 sql/hive/src/test/resources/golden/input10-0-acdd72fda64c5463ce29f31020012d42 create mode 100644 sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 create mode 100644 sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 create mode 100644 sql/hive/src/test/resources/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb create mode 100644 sql/hive/src/test/resources/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 create mode 100644 sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 create mode 100644 sql/hive/src/test/resources/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 create mode 100644 sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 create mode 100644 sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e create mode 100644 sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 create mode 100644 sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c create mode 100644 sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 create mode 100644 sql/hive/src/test/resources/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 create mode 100644 sql/hive/src/test/resources/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf create mode 100644 sql/hive/src/test/resources/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 create mode 100644 sql/hive/src/test/resources/golden/input12-7-16da1c1293626b943343443368679e9c create mode 100644 sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd create mode 100644 sql/hive/src/test/resources/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c create mode 100644 sql/hive/src/test/resources/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a create mode 100644 sql/hive/src/test/resources/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 create mode 100644 sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf create mode 100644 sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 create mode 100644 sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 create mode 100644 sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 create mode 100644 sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 create mode 100644 sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 create mode 100644 sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a create mode 100644 sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc create mode 100644 sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 create mode 100644 sql/hive/src/test/resources/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 create mode 100644 sql/hive/src/test/resources/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 create mode 100644 sql/hive/src/test/resources/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e create mode 100644 sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 create mode 100644 sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 create mode 100644 sql/hive/src/test/resources/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c create mode 100644 sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 create mode 100644 sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd create mode 100644 sql/hive/src/test/resources/golden/input2-3-45b9f786f64afa5e039d1856fe926649 create mode 100644 sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b create mode 100644 sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/input2-6-4a6b9674c126337f71834f11613f996d create mode 100644 sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 create mode 100644 sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 create mode 100644 sql/hive/src/test/resources/golden/input21-0-7f40c525398953892ffd6283f54ba427 create mode 100644 sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 create mode 100644 sql/hive/src/test/resources/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 create mode 100644 sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe create mode 100644 sql/hive/src/test/resources/golden/input22-0-b9cd4041ca510639838a97376747b498 create mode 100644 sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 create mode 100644 sql/hive/src/test/resources/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 create mode 100644 sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 create mode 100644 sql/hive/src/test/resources/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf create mode 100644 sql/hive/src/test/resources/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb create mode 100644 sql/hive/src/test/resources/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 create mode 100644 sql/hive/src/test/resources/golden/input24-1-f57485de80ad64f9931bf92514fa1 create mode 100644 sql/hive/src/test/resources/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 create mode 100644 sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 create mode 100644 sql/hive/src/test/resources/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 create mode 100644 sql/hive/src/test/resources/golden/input25-1-f57485de80ad64f9931bf92514fa1 create mode 100644 sql/hive/src/test/resources/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 create mode 100644 sql/hive/src/test/resources/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 create mode 100644 sql/hive/src/test/resources/golden/input25-4-72c74c55dd7d85d194448e9c58488938 create mode 100644 sql/hive/src/test/resources/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 create mode 100644 sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 create mode 100644 sql/hive/src/test/resources/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a create mode 100644 sql/hive/src/test/resources/golden/input28-1-f57485de80ad64f9931bf92514fa1 create mode 100644 sql/hive/src/test/resources/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c create mode 100644 sql/hive/src/test/resources/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 create mode 100644 sql/hive/src/test/resources/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f create mode 100644 sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 create mode 100644 sql/hive/src/test/resources/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff create mode 100644 sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d create mode 100644 sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 create mode 100644 sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 create mode 100644 sql/hive/src/test/resources/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a create mode 100644 sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 create mode 100644 sql/hive/src/test/resources/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 create mode 100644 sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 create mode 100644 sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 create mode 100644 sql/hive/src/test/resources/golden/input3-6-ba8c440158c2519353d02471bfb05694 create mode 100644 sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 create mode 100644 sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be create mode 100644 sql/hive/src/test/resources/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e create mode 100644 sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c create mode 100644 sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab create mode 100644 sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 create mode 100644 sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 create mode 100644 sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 create mode 100644 sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 create mode 100644 sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f create mode 100644 sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79 create mode 100644 sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe create mode 100644 sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f create mode 100644 sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 create mode 100644 sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79 create mode 100644 sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 create mode 100644 sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d create mode 100644 sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 create mode 100644 sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e create mode 100644 sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 create mode 100644 sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 create mode 100644 sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 create mode 100644 sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 create mode 100644 sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 create mode 100644 sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 create mode 100644 sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 create mode 100644 sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 create mode 100644 sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d create mode 100644 sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 create mode 100644 sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e create mode 100644 sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 create mode 100644 sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 create mode 100644 sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e create mode 100644 sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 create mode 100644 sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 create mode 100644 sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 create mode 100644 sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 create mode 100644 sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d create mode 100644 sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece create mode 100644 sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d create mode 100644 sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 create mode 100644 sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 create mode 100644 sql/hive/src/test/resources/golden/input4-0-b9cd4041ca510639838a97376747b498 create mode 100644 sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 create mode 100644 sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 create mode 100644 sql/hive/src/test/resources/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 create mode 100644 sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 create mode 100644 sql/hive/src/test/resources/golden/input40-0-37e74908fd43254141ae3664f13a48d create mode 100644 sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 create mode 100644 sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 create mode 100644 sql/hive/src/test/resources/golden/input40-3-12f634800ffae17942ddd789a01af227 create mode 100644 sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e create mode 100644 sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 create mode 100644 sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 create mode 100644 sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 create mode 100644 sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 create mode 100644 sql/hive/src/test/resources/golden/input41-1-8112b3a278e8337dbd6f017ae178658b create mode 100644 sql/hive/src/test/resources/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 create mode 100644 sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee create mode 100644 sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a create mode 100644 sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 create mode 100644 sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f create mode 100644 sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca create mode 100644 sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 create mode 100644 sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 create mode 100644 sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab create mode 100644 sql/hive/src/test/resources/golden/input49-0-2bd546beeb607da5b925ffbea20a741c create mode 100644 sql/hive/src/test/resources/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d create mode 100644 sql/hive/src/test/resources/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 create mode 100644 sql/hive/src/test/resources/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 create mode 100644 sql/hive/src/test/resources/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 create mode 100644 sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 create mode 100644 sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 create mode 100644 sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d create mode 100644 sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af create mode 100644 sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea create mode 100644 sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 create mode 100644 sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad create mode 100644 sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input6-0-659e06570690cceeb3f37e10e855d2ea create mode 100644 sql/hive/src/test/resources/golden/input6-1-21149162906e31964a328b6cb2d5286e create mode 100644 sql/hive/src/test/resources/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c create mode 100644 sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 create mode 100644 sql/hive/src/test/resources/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 create mode 100644 sql/hive/src/test/resources/golden/input7-2-c740098a289f9a4267d26fc10347a0dc create mode 100644 sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa create mode 100644 sql/hive/src/test/resources/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa create mode 100644 sql/hive/src/test/resources/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba create mode 100644 sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 create mode 100644 sql/hive/src/test/resources/golden/input9-1-b30105391cb53915997a84ca3137bd0a create mode 100644 sql/hive/src/test/resources/golden/input9-2-171ea277f05a642699a0f91bacffccd8 create mode 100644 sql/hive/src/test/resources/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e create mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 create mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be create mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 create mode 100644 sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 create mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d create mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc create mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 create mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c create mode 100644 sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a create mode 100644 sql/hive/src/test/resources/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 create mode 100644 sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 create mode 100644 sql/hive/src/test/resources/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 create mode 100644 sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb create mode 100644 sql/hive/src/test/resources/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 create mode 100644 sql/hive/src/test/resources/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 create mode 100644 sql/hive/src/test/resources/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc create mode 100644 sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 create mode 100644 sql/hive/src/test/resources/golden/input_part10-1-35f67c0112026170d015b5f80a254bde create mode 100644 sql/hive/src/test/resources/golden/input_part10-2-40069c199502c2724ac2a2733f964248 create mode 100644 sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 create mode 100644 sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 create mode 100644 sql/hive/src/test/resources/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf create mode 100644 sql/hive/src/test/resources/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde create mode 100644 sql/hive/src/test/resources/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 create mode 100644 sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 create mode 100644 sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 create mode 100644 sql/hive/src/test/resources/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 create mode 100644 sql/hive/src/test/resources/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e create mode 100644 sql/hive/src/test/resources/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd create mode 100644 sql/hive/src/test/resources/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 create mode 100644 sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 create mode 100644 sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 create mode 100644 sql/hive/src/test/resources/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff create mode 100644 sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 create mode 100644 sql/hive/src/test/resources/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef create mode 100644 sql/hive/src/test/resources/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 create mode 100644 sql/hive/src/test/resources/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b create mode 100644 sql/hive/src/test/resources/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c create mode 100644 sql/hive/src/test/resources/golden/input_part5-2-d678533566fba4b922f01284538ca484 create mode 100644 sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 create mode 100644 sql/hive/src/test/resources/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d create mode 100644 sql/hive/src/test/resources/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 create mode 100644 sql/hive/src/test/resources/golden/input_part7-0-14448a12627b8bccf6828032f938f39e create mode 100644 sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 create mode 100644 sql/hive/src/test/resources/golden/input_part8-0-f60652796af548135f9e104b456840e3 create mode 100644 sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 create mode 100644 sql/hive/src/test/resources/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 create mode 100644 sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 create mode 100644 sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 create mode 100644 sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e create mode 100644 sql/hive/src/test/resources/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf create mode 100644 sql/hive/src/test/resources/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 create mode 100644 sql/hive/src/test/resources/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 create mode 100644 sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c create mode 100644 sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f create mode 100644 sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd create mode 100644 sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac create mode 100644 sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 create mode 100644 sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 create mode 100644 sql/hive/src/test/resources/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 create mode 100644 sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 create mode 100644 sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 create mode 100644 sql/hive/src/test/resources/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 create mode 100644 sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf create mode 100644 sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c create mode 100644 sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c create mode 100644 sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 create mode 100644 sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 create mode 100644 sql/hive/src/test/resources/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 create mode 100644 sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 create mode 100644 sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 create mode 100644 sql/hive/src/test/resources/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f create mode 100644 sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d create mode 100644 sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a create mode 100644 sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe create mode 100644 sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a create mode 100644 sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 create mode 100644 sql/hive/src/test/resources/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac create mode 100644 sql/hive/src/test/resources/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 create mode 100644 sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 create mode 100644 sql/hive/src/test/resources/golden/insert into and insert overwrite-0-2e2de36571343c0ebaf02c0c0c61ba8c create mode 100644 sql/hive/src/test/resources/golden/insert into and insert overwrite-1-548d5a45ef20e066df145944dce3415a create mode 100644 sql/hive/src/test/resources/golden/insert into and insert overwrite-2-77baf70da8adf8b4b82d076fee937fe1 create mode 100644 sql/hive/src/test/resources/golden/insert into and insert overwrite-3-b9002c1d71895be765575b62656d1928 create mode 100644 sql/hive/src/test/resources/golden/insert into and insert overwrite-4-3b8e0155870264d89d275a1d5bd345d0 create mode 100644 sql/hive/src/test/resources/golden/insert into and insert overwrite-5-b9002c1d71895be765575b62656d1928 create mode 100644 sql/hive/src/test/resources/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac create mode 100644 sql/hive/src/test/resources/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 create mode 100644 sql/hive/src/test/resources/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 create mode 100644 sql/hive/src/test/resources/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 create mode 100644 sql/hive/src/test/resources/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef create mode 100644 sql/hive/src/test/resources/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 create mode 100644 sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 create mode 100644 sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b create mode 100644 sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 create mode 100644 sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb create mode 100644 sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 create mode 100644 sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 create mode 100644 sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 create mode 100644 sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb create mode 100644 sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 create mode 100644 sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 create mode 100644 sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb create mode 100644 sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d create mode 100644 sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b create mode 100644 sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa create mode 100644 sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 create mode 100644 sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d create mode 100644 sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd create mode 100644 sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 create mode 100644 sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa create mode 100644 sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b create mode 100644 sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b create mode 100644 sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a create mode 100644 sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 create mode 100644 sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c create mode 100644 sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d create mode 100644 sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c create mode 100644 sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f create mode 100644 sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 create mode 100644 sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba create mode 100644 sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d create mode 100644 sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d create mode 100644 sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f create mode 100644 sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 create mode 100644 sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab create mode 100644 sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 create mode 100644 sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a create mode 100644 sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b create mode 100644 sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 create mode 100644 sql/hive/src/test/resources/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 create mode 100644 sql/hive/src/test/resources/golden/join0-1-83f5e512d6058a47f92b5218781f5070 create mode 100644 sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 create mode 100644 sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e create mode 100644 sql/hive/src/test/resources/golden/join1-2-7b07671864bbfdc627ee794932e32b1e create mode 100644 sql/hive/src/test/resources/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 create mode 100644 sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 create mode 100644 sql/hive/src/test/resources/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 create mode 100644 sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d create mode 100644 sql/hive/src/test/resources/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 create mode 100644 sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 create mode 100644 sql/hive/src/test/resources/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 create mode 100644 sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f create mode 100644 sql/hive/src/test/resources/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 create mode 100644 sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 create mode 100644 sql/hive/src/test/resources/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 create mode 100644 sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e create mode 100644 sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 create mode 100644 sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 create mode 100644 sql/hive/src/test/resources/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 create mode 100644 sql/hive/src/test/resources/golden/join14-6-de39302191b63d7aa8f92885b089fe2 create mode 100644 sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 create mode 100644 sql/hive/src/test/resources/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 create mode 100644 sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f create mode 100644 sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 create mode 100644 sql/hive/src/test/resources/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 create mode 100644 sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 create mode 100644 sql/hive/src/test/resources/golden/join15-0-930db2823512f7f8f34cac104f2162e0 create mode 100644 sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 create mode 100644 sql/hive/src/test/resources/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c create mode 100644 sql/hive/src/test/resources/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 create mode 100644 sql/hive/src/test/resources/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b create mode 100644 sql/hive/src/test/resources/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 create mode 100644 sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/join18-0-269843197b598569f4b50cbe911960a5 create mode 100644 sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 create mode 100644 sql/hive/src/test/resources/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 create mode 100644 sql/hive/src/test/resources/golden/join19-1-7e7d1f07c34dd4be5425264196201766 create mode 100644 sql/hive/src/test/resources/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca create mode 100644 sql/hive/src/test/resources/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c create mode 100644 sql/hive/src/test/resources/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a create mode 100644 sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 create mode 100644 sql/hive/src/test/resources/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e create mode 100644 sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 create mode 100644 sql/hive/src/test/resources/golden/join20-2-222655137ca9242f063c7717acbe4c65 create mode 100644 sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 create mode 100644 sql/hive/src/test/resources/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 create mode 100644 sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f create mode 100644 sql/hive/src/test/resources/golden/join22-0-97081b5f73052a612fb391a57ad9b7b create mode 100644 sql/hive/src/test/resources/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d create mode 100644 sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 create mode 100644 sql/hive/src/test/resources/golden/join24-0-721dfa03bfea05e55506c571b6c3585b create mode 100644 sql/hive/src/test/resources/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d create mode 100644 sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 create mode 100644 sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 create mode 100644 sql/hive/src/test/resources/golden/join25-1-360b3676605ccb3d89cf555342db87af create mode 100644 sql/hive/src/test/resources/golden/join25-2-ef930bed933341636c71622e6f89e9e9 create mode 100644 sql/hive/src/test/resources/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 create mode 100644 sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join26-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 sql/hive/src/test/resources/golden/join26-1-10fbe6c99e93cfeafcd454261744728e create mode 100644 sql/hive/src/test/resources/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d create mode 100644 sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join27-0-360b3676605ccb3d89cf555342db87af create mode 100644 sql/hive/src/test/resources/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 create mode 100644 sql/hive/src/test/resources/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 create mode 100644 sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 create mode 100644 sql/hive/src/test/resources/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 create mode 100644 sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 create mode 100644 sql/hive/src/test/resources/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 create mode 100644 sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join29-0-e78402f7585b17c76c32790571d749eb create mode 100644 sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/join29-4-b0524de7985cdb7b57fd995721654856 create mode 100644 sql/hive/src/test/resources/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 create mode 100644 sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 create mode 100644 sql/hive/src/test/resources/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 create mode 100644 sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 create mode 100644 sql/hive/src/test/resources/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d create mode 100644 sql/hive/src/test/resources/golden/join30-2-194edec0592cf441617ca7caf9756baa create mode 100644 sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join31-0-619db2f9dd69aa236aa804cced726c9a create mode 100644 sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 create mode 100644 sql/hive/src/test/resources/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 create mode 100644 sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join32-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb create mode 100644 sql/hive/src/test/resources/golden/join32-5-425c5c35b23c42254dabec49aa6613bf create mode 100644 sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b create mode 100644 sql/hive/src/test/resources/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 create mode 100644 sql/hive/src/test/resources/golden/join33-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb create mode 100644 sql/hive/src/test/resources/golden/join33-5-425c5c35b23c42254dabec49aa6613bf create mode 100644 sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join34-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/join34-4-51cc5d253befd784a9d466c0b402f47a create mode 100644 sql/hive/src/test/resources/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 create mode 100644 sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 create mode 100644 sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/join35-4-99cb704ccc2813400908a62571891dc7 create mode 100644 sql/hive/src/test/resources/golden/join35-5-60956f3307651237f52ffbe41c827c1c create mode 100644 sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 create mode 100644 sql/hive/src/test/resources/golden/join36-1-3aba153d62042206c0f001ea74980da create mode 100644 sql/hive/src/test/resources/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 create mode 100644 sql/hive/src/test/resources/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 create mode 100644 sql/hive/src/test/resources/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 create mode 100644 sql/hive/src/test/resources/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 create mode 100644 sql/hive/src/test/resources/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 create mode 100644 sql/hive/src/test/resources/golden/join36-7-171ede21302050a33468d95e329a38a3 create mode 100644 sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 create mode 100644 sql/hive/src/test/resources/golden/join37-1-360b3676605ccb3d89cf555342db87af create mode 100644 sql/hive/src/test/resources/golden/join37-2-2310f2b8726137811543e04db98dd1db create mode 100644 sql/hive/src/test/resources/golden/join37-3-c4b3a85965108ad7013ac3931598af09 create mode 100644 sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join38-0-42e6ad674a5142d05258a2d29578e046 create mode 100644 sql/hive/src/test/resources/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f create mode 100644 sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 create mode 100644 sql/hive/src/test/resources/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 create mode 100644 sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef create mode 100644 sql/hive/src/test/resources/golden/join39-0-54e9f7924abb40d1c601de908de6f25b create mode 100644 sql/hive/src/test/resources/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 create mode 100644 sql/hive/src/test/resources/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 create mode 100644 sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join4-0-531e91e11b3891627c2675935fda14cd create mode 100644 sql/hive/src/test/resources/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 create mode 100644 sql/hive/src/test/resources/golden/join4-2-dc967001beb776f3a859e9360823c361 create mode 100644 sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac create mode 100644 sql/hive/src/test/resources/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 create mode 100644 sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa create mode 100644 sql/hive/src/test/resources/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 create mode 100644 sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda create mode 100644 sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 create mode 100644 sql/hive/src/test/resources/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 create mode 100644 sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 create mode 100644 sql/hive/src/test/resources/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e create mode 100644 sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 create mode 100644 sql/hive/src/test/resources/golden/join40-7-222655137ca9242f063c7717acbe4c65 create mode 100644 sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 create mode 100644 sql/hive/src/test/resources/golden/join40-9-6b7404af014eeb4de196577f14ef18cc create mode 100644 sql/hive/src/test/resources/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 create mode 100644 sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 create mode 100644 sql/hive/src/test/resources/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a create mode 100644 sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 create mode 100644 sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e create mode 100644 sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 create mode 100644 sql/hive/src/test/resources/golden/join5-0-531e91e11b3891627c2675935fda14cd create mode 100644 sql/hive/src/test/resources/golden/join5-1-76feff1282895f38b673f52acfd7429e create mode 100644 sql/hive/src/test/resources/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 create mode 100644 sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/join6-0-531e91e11b3891627c2675935fda14cd create mode 100644 sql/hive/src/test/resources/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 create mode 100644 sql/hive/src/test/resources/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 create mode 100644 sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 create mode 100644 sql/hive/src/test/resources/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 create mode 100644 sql/hive/src/test/resources/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 create mode 100644 sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/join8-0-531e91e11b3891627c2675935fda14cd create mode 100644 sql/hive/src/test/resources/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 create mode 100644 sql/hive/src/test/resources/golden/join8-2-f1196bca86a749375da35f134206a8ca create mode 100644 sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a create mode 100644 sql/hive/src/test/resources/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 create mode 100644 sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a create mode 100644 sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e create mode 100644 sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b create mode 100644 sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c create mode 100644 sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b create mode 100644 sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c create mode 100644 sql/hive/src/test/resources/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b create mode 100644 sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c create mode 100644 sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e create mode 100644 sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b create mode 100644 sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c create mode 100644 sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b create mode 100644 sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c create mode 100644 sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e create mode 100644 sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b create mode 100644 sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c create mode 100644 sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 create mode 100644 sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 create mode 100644 sql/hive/src/test/resources/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 create mode 100644 sql/hive/src/test/resources/golden/join_array-1-2c086fcf118fd2538bfa00724209091e create mode 100644 sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de create mode 100644 sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 create mode 100644 sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 create mode 100644 sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a create mode 100644 sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c create mode 100644 sql/hive/src/test/resources/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 create mode 100644 sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 create mode 100644 sql/hive/src/test/resources/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d create mode 100644 sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc create mode 100644 sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd create mode 100644 sql/hive/src/test/resources/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad create mode 100644 sql/hive/src/test/resources/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 create mode 100644 sql/hive/src/test/resources/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e create mode 100644 sql/hive/src/test/resources/golden/join_empty-3-3b2231f25886c152f222856256ac3473 create mode 100644 sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a create mode 100644 sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a create mode 100644 sql/hive/src/test/resources/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 create mode 100644 sql/hive/src/test/resources/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 create mode 100644 sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 create mode 100644 sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 create mode 100644 sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe create mode 100644 sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b create mode 100644 sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 create mode 100644 sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b create mode 100644 sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 create mode 100644 sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb create mode 100644 sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 create mode 100644 sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 create mode 100644 sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e create mode 100644 sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 create mode 100644 sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 create mode 100644 sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba create mode 100644 sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 create mode 100644 sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 create mode 100644 sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba create mode 100644 sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 create mode 100644 sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 create mode 100644 sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 create mode 100644 sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 create mode 100644 sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e create mode 100644 sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a create mode 100644 sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e create mode 100644 sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 create mode 100644 sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada create mode 100644 sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 create mode 100644 sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 create mode 100644 sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf create mode 100644 sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 create mode 100644 sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 create mode 100644 sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 create mode 100644 sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 create mode 100644 sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f create mode 100644 sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 create mode 100644 sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf create mode 100644 sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e create mode 100644 sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e create mode 100644 sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f create mode 100644 sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 create mode 100644 sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 create mode 100644 sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca create mode 100644 sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 create mode 100644 sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 create mode 100644 sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 create mode 100644 sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 create mode 100644 sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf create mode 100644 sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f create mode 100644 sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab create mode 100644 sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 create mode 100644 sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 create mode 100644 sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe create mode 100644 sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b create mode 100644 sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 create mode 100644 sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b create mode 100644 sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 create mode 100644 sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 create mode 100644 sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb create mode 100644 sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 create mode 100644 sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 create mode 100644 sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e create mode 100644 sql/hive/src/test/resources/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e create mode 100644 sql/hive/src/test/resources/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 create mode 100644 sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b create mode 100644 sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 create mode 100644 sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c create mode 100644 sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 create mode 100644 sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e create mode 100644 sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 create mode 100644 sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac create mode 100644 sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 create mode 100644 sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 create mode 100644 sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba create mode 100644 sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 create mode 100644 sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 create mode 100644 sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba create mode 100644 sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 create mode 100644 sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 create mode 100644 sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 create mode 100644 sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 create mode 100644 sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 create mode 100644 sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e create mode 100644 sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a create mode 100644 sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 create mode 100644 sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada create mode 100644 sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 create mode 100644 sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 create mode 100644 sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 create mode 100644 sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 create mode 100644 sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d create mode 100644 sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e create mode 100644 sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 create mode 100644 sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d create mode 100644 sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 create mode 100644 sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa create mode 100644 sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 create mode 100644 sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 create mode 100644 sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e create mode 100644 sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 create mode 100644 sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf create mode 100644 sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 create mode 100644 sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 create mode 100644 sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 create mode 100644 sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 create mode 100644 sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 create mode 100644 sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f create mode 100644 sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf create mode 100644 sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e create mode 100644 sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e create mode 100644 sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f create mode 100644 sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 create mode 100644 sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa create mode 100644 sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 create mode 100644 sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca create mode 100644 sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 create mode 100644 sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 create mode 100644 sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 create mode 100644 sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf create mode 100644 sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f create mode 100644 sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab create mode 100644 sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 create mode 100644 sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 create mode 100644 sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce create mode 100644 sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 create mode 100644 sql/hive/src/test/resources/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 create mode 100644 sql/hive/src/test/resources/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 create mode 100644 sql/hive/src/test/resources/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e create mode 100644 sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 create mode 100644 sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 create mode 100644 sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 create mode 100644 sql/hive/src/test/resources/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 create mode 100644 sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed create mode 100644 sql/hive/src/test/resources/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a create mode 100644 sql/hive/src/test/resources/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b create mode 100644 sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d create mode 100644 sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f create mode 100644 sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c create mode 100644 sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e create mode 100644 sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb create mode 100644 sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb create mode 100644 sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc create mode 100644 sql/hive/src/test/resources/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e create mode 100644 sql/hive/src/test/resources/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b create mode 100644 sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c create mode 100644 sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c create mode 100644 sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f create mode 100644 sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a create mode 100644 sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 create mode 100644 sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b create mode 100644 sql/hive/src/test/resources/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 create mode 100644 sql/hive/src/test/resources/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e create mode 100644 sql/hive/src/test/resources/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 create mode 100644 sql/hive/src/test/resources/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee create mode 100644 sql/hive/src/test/resources/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 create mode 100644 sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 create mode 100644 sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 create mode 100644 sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 create mode 100644 sql/hive/src/test/resources/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba create mode 100644 sql/hive/src/test/resources/golden/join_star-0-6001f83f7780835737e3262a34c2b832 create mode 100644 sql/hive/src/test/resources/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d create mode 100644 sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e create mode 100644 sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae create mode 100644 sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 create mode 100644 sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb create mode 100644 sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 create mode 100644 sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 create mode 100644 sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 create mode 100644 sql/hive/src/test/resources/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa create mode 100644 sql/hive/src/test/resources/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 create mode 100644 sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 create mode 100644 sql/hive/src/test/resources/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 create mode 100644 sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b create mode 100644 sql/hive/src/test/resources/golden/join_star-23-bada259628918dbfb4837bcb58258530 create mode 100644 sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 create mode 100644 sql/hive/src/test/resources/golden/join_star-25-be5af28b6bccf529a31ef68619de699e create mode 100644 sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 create mode 100644 sql/hive/src/test/resources/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 create mode 100644 sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 create mode 100644 sql/hive/src/test/resources/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da create mode 100644 sql/hive/src/test/resources/golden/join_star-4-aaba115af7543b214820a48a694f2463 create mode 100644 sql/hive/src/test/resources/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da create mode 100644 sql/hive/src/test/resources/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a create mode 100644 sql/hive/src/test/resources/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a create mode 100644 sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 create mode 100644 sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 create mode 100644 sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 create mode 100644 sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db create mode 100644 sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b create mode 100644 sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 create mode 100644 sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 create mode 100644 sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 create mode 100644 sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 create mode 100644 sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b create mode 100644 sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 create mode 100644 sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 create mode 100644 sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef create mode 100644 sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d create mode 100644 sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 create mode 100644 sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 create mode 100644 sql/hive/src/test/resources/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee create mode 100644 sql/hive/src/test/resources/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb create mode 100644 sql/hive/src/test/resources/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 create mode 100644 sql/hive/src/test/resources/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 create mode 100644 sql/hive/src/test/resources/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 create mode 100644 sql/hive/src/test/resources/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e create mode 100644 sql/hive/src/test/resources/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b create mode 100644 sql/hive/src/test/resources/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a create mode 100644 sql/hive/src/test/resources/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b create mode 100644 sql/hive/src/test/resources/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e create mode 100644 sql/hive/src/test/resources/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba create mode 100644 sql/hive/src/test/resources/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e create mode 100644 sql/hive/src/test/resources/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab create mode 100644 sql/hive/src/test/resources/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e create mode 100644 sql/hive/src/test/resources/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e create mode 100644 sql/hive/src/test/resources/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae create mode 100644 sql/hive/src/test/resources/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 create mode 100644 sql/hive/src/test/resources/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_cp-4-507876b610812506343e9d251645170b create mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a create mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe create mode 100644 sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa create mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d create mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e create mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b create mode 100644 sql/hive/src/test/resources/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 create mode 100644 sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 create mode 100644 sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a create mode 100644 sql/hive/src/test/resources/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 create mode 100644 sql/hive/src/test/resources/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 create mode 100644 sql/hive/src/test/resources/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c create mode 100644 sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f create mode 100644 sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 create mode 100644 sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 create mode 100644 sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 create mode 100644 sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 create mode 100644 sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 create mode 100644 sql/hive/src/test/resources/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb create mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 create mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 create mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e create mode 100644 sql/hive/src/test/resources/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 create mode 100644 sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 create mode 100644 sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f create mode 100644 sql/hive/src/test/resources/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 create mode 100644 sql/hive/src/test/resources/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 create mode 100644 sql/hive/src/test/resources/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 create mode 100644 sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc create mode 100644 sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 create mode 100644 sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 create mode 100644 sql/hive/src/test/resources/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 create mode 100644 sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e create mode 100644 sql/hive/src/test/resources/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 create mode 100644 sql/hive/src/test/resources/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 create mode 100644 sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e create mode 100644 sql/hive/src/test/resources/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 create mode 100644 sql/hive/src/test/resources/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 create mode 100644 sql/hive/src/test/resources/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c create mode 100644 sql/hive/src/test/resources/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 create mode 100644 sql/hive/src/test/resources/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 create mode 100644 sql/hive/src/test/resources/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 create mode 100644 sql/hive/src/test/resources/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc create mode 100644 sql/hive/src/test/resources/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 create mode 100644 sql/hive/src/test/resources/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 create mode 100644 sql/hive/src/test/resources/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 create mode 100644 sql/hive/src/test/resources/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 create mode 100644 sql/hive/src/test/resources/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 create mode 100644 sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 create mode 100644 sql/hive/src/test/resources/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e create mode 100644 sql/hive/src/test/resources/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 create mode 100644 sql/hive/src/test/resources/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 create mode 100644 sql/hive/src/test/resources/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 create mode 100644 sql/hive/src/test/resources/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 create mode 100644 sql/hive/src/test/resources/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 create mode 100644 sql/hive/src/test/resources/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 create mode 100644 sql/hive/src/test/resources/golden/lock2-3-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 sql/hive/src/test/resources/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca create mode 100644 sql/hive/src/test/resources/golden/lock3-0-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 sql/hive/src/test/resources/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 create mode 100644 sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac create mode 100644 sql/hive/src/test/resources/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe create mode 100644 sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 create mode 100644 sql/hive/src/test/resources/golden/lock4-1-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 sql/hive/src/test/resources/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 create mode 100644 sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac create mode 100644 sql/hive/src/test/resources/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe create mode 100644 sql/hive/src/test/resources/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd create mode 100644 sql/hive/src/test/resources/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 create mode 100644 sql/hive/src/test/resources/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 create mode 100644 sql/hive/src/test/resources/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef create mode 100644 sql/hive/src/test/resources/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 create mode 100644 sql/hive/src/test/resources/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 create mode 100644 sql/hive/src/test/resources/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 create mode 100644 sql/hive/src/test/resources/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 create mode 100644 sql/hive/src/test/resources/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e create mode 100644 sql/hive/src/test/resources/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a create mode 100644 sql/hive/src/test/resources/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 create mode 100644 sql/hive/src/test/resources/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c create mode 100644 sql/hive/src/test/resources/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe create mode 100644 sql/hive/src/test/resources/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e create mode 100644 sql/hive/src/test/resources/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c create mode 100644 sql/hive/src/test/resources/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 create mode 100644 sql/hive/src/test/resources/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b create mode 100644 sql/hive/src/test/resources/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc create mode 100644 sql/hive/src/test/resources/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c create mode 100644 sql/hive/src/test/resources/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b create mode 100644 sql/hive/src/test/resources/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a create mode 100644 sql/hive/src/test/resources/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c create mode 100644 sql/hive/src/test/resources/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f create mode 100644 sql/hive/src/test/resources/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab create mode 100644 sql/hive/src/test/resources/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c create mode 100644 sql/hive/src/test/resources/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 create mode 100644 sql/hive/src/test/resources/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca create mode 100644 sql/hive/src/test/resources/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c create mode 100644 sql/hive/src/test/resources/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb create mode 100644 sql/hive/src/test/resources/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 create mode 100644 sql/hive/src/test/resources/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 create mode 100644 sql/hive/src/test/resources/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a create mode 100644 sql/hive/src/test/resources/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 create mode 100644 sql/hive/src/test/resources/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 create mode 100644 sql/hive/src/test/resources/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b create mode 100644 sql/hive/src/test/resources/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 create mode 100644 sql/hive/src/test/resources/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/merge1-1-2c73c923962b91afdf0004a705432550 create mode 100644 sql/hive/src/test/resources/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 create mode 100644 sql/hive/src/test/resources/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 create mode 100644 sql/hive/src/test/resources/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 create mode 100644 sql/hive/src/test/resources/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 create mode 100644 sql/hive/src/test/resources/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 create mode 100644 sql/hive/src/test/resources/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 create mode 100644 sql/hive/src/test/resources/golden/merge1-3-1f560722f18ef618a0343313a0cac462 create mode 100644 sql/hive/src/test/resources/golden/merge1-4-3277fe538b66923cd879b45371838d2b create mode 100644 sql/hive/src/test/resources/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 create mode 100644 sql/hive/src/test/resources/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 create mode 100644 sql/hive/src/test/resources/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 create mode 100644 sql/hive/src/test/resources/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a create mode 100644 sql/hive/src/test/resources/golden/merge1-9-64678b0928c4fd054e1578458001c86 create mode 100644 sql/hive/src/test/resources/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 create mode 100644 sql/hive/src/test/resources/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 create mode 100644 sql/hive/src/test/resources/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 create mode 100644 sql/hive/src/test/resources/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a create mode 100644 sql/hive/src/test/resources/golden/merge2-14-64678b0928c4fd054e1578458001c86 create mode 100644 sql/hive/src/test/resources/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 create mode 100644 sql/hive/src/test/resources/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 create mode 100644 sql/hive/src/test/resources/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 create mode 100644 sql/hive/src/test/resources/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f create mode 100644 sql/hive/src/test/resources/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 create mode 100644 sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 sql/hive/src/test/resources/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 create mode 100644 sql/hive/src/test/resources/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 create mode 100644 sql/hive/src/test/resources/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 create mode 100644 sql/hive/src/test/resources/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd create mode 100644 sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32 create mode 100644 sql/hive/src/test/resources/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 create mode 100644 sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11 create mode 100644 sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 create mode 100644 sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 create mode 100644 sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a create mode 100644 sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d create mode 100644 sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af create mode 100644 sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d create mode 100644 sql/hive/src/test/resources/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 create mode 100644 sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c create mode 100644 sql/hive/src/test/resources/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 create mode 100644 sql/hive/src/test/resources/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 create mode 100644 sql/hive/src/test/resources/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b create mode 100644 sql/hive/src/test/resources/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c create mode 100644 sql/hive/src/test/resources/golden/mergejoins-3-c408f69470d652da283442a62b384e46 create mode 100644 sql/hive/src/test/resources/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 create mode 100644 sql/hive/src/test/resources/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e create mode 100644 sql/hive/src/test/resources/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 create mode 100644 sql/hive/src/test/resources/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 create mode 100644 sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 create mode 100644 sql/hive/src/test/resources/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/mi-1-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e create mode 100644 sql/hive/src/test/resources/golden/mi-3-b66a495f7bdf106a7886b72267b8659d create mode 100644 sql/hive/src/test/resources/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c create mode 100644 sql/hive/src/test/resources/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc create mode 100644 sql/hive/src/test/resources/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 create mode 100644 sql/hive/src/test/resources/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 create mode 100644 sql/hive/src/test/resources/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 create mode 100644 sql/hive/src/test/resources/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b create mode 100644 sql/hive/src/test/resources/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 create mode 100644 sql/hive/src/test/resources/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 create mode 100644 sql/hive/src/test/resources/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b create mode 100644 sql/hive/src/test/resources/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 create mode 100644 sql/hive/src/test/resources/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 create mode 100644 sql/hive/src/test/resources/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 create mode 100644 sql/hive/src/test/resources/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 create mode 100644 sql/hive/src/test/resources/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e create mode 100644 sql/hive/src/test/resources/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 create mode 100644 sql/hive/src/test/resources/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 create mode 100644 sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 create mode 100644 sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 create mode 100644 sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 create mode 100644 sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f create mode 100644 sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b create mode 100644 sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505 create mode 100644 sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d create mode 100644 sql/hive/src/test/resources/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 create mode 100644 sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 create mode 100644 sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b create mode 100644 sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 create mode 100644 sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 create mode 100644 sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 create mode 100644 sql/hive/src/test/resources/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 create mode 100644 sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 create mode 100644 sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 create mode 100644 sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 create mode 100644 sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad create mode 100644 sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 create mode 100644 sql/hive/src/test/resources/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d create mode 100644 sql/hive/src/test/resources/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd create mode 100644 sql/hive/src/test/resources/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e create mode 100644 sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d create mode 100644 sql/hive/src/test/resources/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 create mode 100644 sql/hive/src/test/resources/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 create mode 100644 sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 create mode 100644 sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c create mode 100644 sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 create mode 100644 sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 create mode 100644 sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 create mode 100644 sql/hive/src/test/resources/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 create mode 100644 sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b create mode 100644 sql/hive/src/test/resources/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 create mode 100644 sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c create mode 100644 sql/hive/src/test/resources/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e create mode 100644 sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c create mode 100644 sql/hive/src/test/resources/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 create mode 100644 sql/hive/src/test/resources/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 create mode 100644 sql/hive/src/test/resources/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf create mode 100644 sql/hive/src/test/resources/golden/nullinput2-0-40b73200f853b473855e85eba391f008 create mode 100644 sql/hive/src/test/resources/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 create mode 100644 sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 create mode 100644 sql/hive/src/test/resources/golden/nullscript-0-4477b1566208a6805bb20630755a9375 create mode 100644 sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 create mode 100644 sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 create mode 100644 sql/hive/src/test/resources/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 create mode 100644 sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 create mode 100644 sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc create mode 100644 sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 create mode 100644 sql/hive/src/test/resources/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a create mode 100644 sql/hive/src/test/resources/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 create mode 100644 sql/hive/src/test/resources/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f create mode 100644 sql/hive/src/test/resources/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a create mode 100644 sql/hive/src/test/resources/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 create mode 100644 sql/hive/src/test/resources/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e create mode 100644 sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 create mode 100644 sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 create mode 100644 sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d create mode 100644 sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 create mode 100644 sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 create mode 100644 sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 create mode 100644 sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 create mode 100644 sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 create mode 100644 sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d create mode 100644 sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb create mode 100644 sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 create mode 100644 sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 create mode 100644 sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 create mode 100644 sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 create mode 100644 sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 create mode 100644 sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 create mode 100644 sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 create mode 100644 sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 create mode 100644 sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead create mode 100644 sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c create mode 100644 sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 create mode 100644 sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 create mode 100644 sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 create mode 100644 sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae create mode 100644 sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d create mode 100644 sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 create mode 100644 sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 create mode 100644 sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 create mode 100644 sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 create mode 100644 sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a create mode 100644 sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 create mode 100644 sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 create mode 100644 sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 create mode 100644 sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 create mode 100644 sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae create mode 100644 sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 create mode 100644 sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 create mode 100644 sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 create mode 100644 sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 create mode 100644 sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 create mode 100644 sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 create mode 100644 sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 create mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 create mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 create mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 create mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e create mode 100644 sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 create mode 100644 sql/hive/src/test/resources/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c create mode 100644 sql/hive/src/test/resources/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 create mode 100644 sql/hive/src/test/resources/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 create mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 create mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf create mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 create mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 create mode 100644 sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e create mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 create mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e create mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 create mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 create mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc create mode 100644 sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f create mode 100644 sql/hive/src/test/resources/golden/order-0-5c2fda577771db3b316e0d2bd02d048a create mode 100644 sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 create mode 100644 sql/hive/src/test/resources/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf create mode 100644 sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 create mode 100644 sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf create mode 100644 sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f create mode 100644 sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c create mode 100644 sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 create mode 100644 sql/hive/src/test/resources/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba create mode 100644 sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 create mode 100644 sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 create mode 100644 sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 create mode 100644 sql/hive/src/test/resources/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 create mode 100644 sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a create mode 100644 sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 create mode 100644 sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a create mode 100644 sql/hive/src/test/resources/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 create mode 100644 sql/hive/src/test/resources/golden/parallel-5-e27b511ffc5828b09069121031d17a2f create mode 100644 sql/hive/src/test/resources/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 create mode 100644 sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a create mode 100644 sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 create mode 100644 sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 create mode 100644 sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 create mode 100644 sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc create mode 100644 sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd create mode 100644 sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec create mode 100644 sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 create mode 100644 sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 create mode 100644 sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea create mode 100644 sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 create mode 100644 sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b create mode 100644 sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 create mode 100644 sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 create mode 100644 sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 create mode 100644 sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 create mode 100644 sql/hive/src/test/resources/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a create mode 100644 sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 create mode 100644 sql/hive/src/test/resources/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 create mode 100644 sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 create mode 100644 sql/hive/src/test/resources/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c create mode 100644 sql/hive/src/test/resources/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 create mode 100644 sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 create mode 100644 sql/hive/src/test/resources/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb create mode 100644 sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 create mode 100644 sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 create mode 100644 sql/hive/src/test/resources/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d create mode 100644 sql/hive/src/test/resources/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 create mode 100644 sql/hive/src/test/resources/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf create mode 100644 sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 create mode 100644 sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d create mode 100644 sql/hive/src/test/resources/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b create mode 100644 sql/hive/src/test/resources/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 create mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 create mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 create mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf create mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 create mode 100644 sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b create mode 100644 sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd create mode 100644 sql/hive/src/test/resources/golden/plan_json-0-74146da55d57b22443140e7fbab3375c create mode 100644 sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 create mode 100644 sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 create mode 100644 sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 create mode 100644 sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 create mode 100644 sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 create mode 100644 sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b create mode 100644 sql/hive/src/test/resources/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 create mode 100644 sql/hive/src/test/resources/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 create mode 100644 sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 create mode 100644 sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 create mode 100644 sql/hive/src/test/resources/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 create mode 100644 sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e create mode 100644 sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d create mode 100644 sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e create mode 100644 sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d create mode 100644 sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f create mode 100644 sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f create mode 100644 sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 create mode 100644 sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 create mode 100644 sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 create mode 100644 sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 create mode 100644 sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 create mode 100644 sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d create mode 100644 sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d create mode 100644 sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 create mode 100644 sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b create mode 100644 sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 create mode 100644 sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 create mode 100644 sql/hive/src/test/resources/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c create mode 100644 sql/hive/src/test/resources/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b create mode 100644 sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 create mode 100644 sql/hive/src/test/resources/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 create mode 100644 sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 create mode 100644 sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 create mode 100644 sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 create mode 100644 sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed create mode 100644 sql/hive/src/test/resources/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae create mode 100644 sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 create mode 100644 sql/hive/src/test/resources/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 create mode 100644 sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e create mode 100644 sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 create mode 100644 sql/hive/src/test/resources/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 create mode 100644 sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 create mode 100644 sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c create mode 100644 sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 create mode 100644 sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 create mode 100644 sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 create mode 100644 sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 create mode 100644 sql/hive/src/test/resources/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d create mode 100644 sql/hive/src/test/resources/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 create mode 100644 sql/hive/src/test/resources/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 create mode 100644 sql/hive/src/test/resources/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d create mode 100644 sql/hive/src/test/resources/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 create mode 100644 sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f create mode 100644 sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 create mode 100644 sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a create mode 100644 sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 create mode 100644 sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb create mode 100644 sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 create mode 100644 sql/hive/src/test/resources/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 create mode 100644 sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f create mode 100644 sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 create mode 100644 sql/hive/src/test/resources/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 create mode 100644 sql/hive/src/test/resources/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 create mode 100644 sql/hive/src/test/resources/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 create mode 100644 sql/hive/src/test/resources/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 create mode 100644 sql/hive/src/test/resources/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a create mode 100644 sql/hive/src/test/resources/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c create mode 100644 sql/hive/src/test/resources/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b create mode 100644 sql/hive/src/test/resources/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 create mode 100644 sql/hive/src/test/resources/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b create mode 100644 sql/hive/src/test/resources/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 create mode 100644 sql/hive/src/test/resources/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 create mode 100644 sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 create mode 100644 sql/hive/src/test/resources/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea create mode 100644 sql/hive/src/test/resources/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b create mode 100644 sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d create mode 100644 sql/hive/src/test/resources/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 create mode 100644 sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d create mode 100644 sql/hive/src/test/resources/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 create mode 100644 sql/hive/src/test/resources/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 create mode 100644 sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d create mode 100644 sql/hive/src/test/resources/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 create mode 100644 sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d create mode 100644 sql/hive/src/test/resources/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 create mode 100644 sql/hive/src/test/resources/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc create mode 100644 sql/hive/src/test/resources/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 create mode 100644 sql/hive/src/test/resources/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 create mode 100644 sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 create mode 100644 sql/hive/src/test/resources/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b create mode 100644 sql/hive/src/test/resources/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 create mode 100644 sql/hive/src/test/resources/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 create mode 100644 sql/hive/src/test/resources/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 create mode 100644 sql/hive/src/test/resources/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e create mode 100644 sql/hive/src/test/resources/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 create mode 100644 sql/hive/src/test/resources/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 create mode 100644 sql/hive/src/test/resources/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 create mode 100644 sql/hive/src/test/resources/golden/protectmode-40-39024332d020339bd9396c9b15403a0c create mode 100644 sql/hive/src/test/resources/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 create mode 100644 sql/hive/src/test/resources/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb create mode 100644 sql/hive/src/test/resources/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 create mode 100644 sql/hive/src/test/resources/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c create mode 100644 sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 create mode 100644 sql/hive/src/test/resources/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 create mode 100644 sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 create mode 100644 sql/hive/src/test/resources/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc create mode 100644 sql/hive/src/test/resources/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 create mode 100644 sql/hive/src/test/resources/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e create mode 100644 sql/hive/src/test/resources/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 create mode 100644 sql/hive/src/test/resources/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 create mode 100644 sql/hive/src/test/resources/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 create mode 100644 sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a create mode 100644 sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 create mode 100644 sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb create mode 100644 sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 create mode 100644 sql/hive/src/test/resources/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 create mode 100644 sql/hive/src/test/resources/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 create mode 100644 sql/hive/src/test/resources/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 create mode 100644 sql/hive/src/test/resources/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 create mode 100644 sql/hive/src/test/resources/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 create mode 100644 sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 create mode 100644 sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 create mode 100644 sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e create mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d create mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 create mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 create mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 create mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 create mode 100644 sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 create mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e create mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 create mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 create mode 100644 sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a create mode 100644 sql/hive/src/test/resources/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 create mode 100644 sql/hive/src/test/resources/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 create mode 100644 sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b create mode 100644 sql/hive/src/test/resources/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 create mode 100644 sql/hive/src/test/resources/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 create mode 100644 sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 create mode 100644 sql/hive/src/test/resources/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 create mode 100644 sql/hive/src/test/resources/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d create mode 100644 sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee create mode 100644 sql/hive/src/test/resources/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a create mode 100644 sql/hive/src/test/resources/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 create mode 100644 sql/hive/src/test/resources/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed create mode 100644 sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 create mode 100644 sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 create mode 100644 sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b create mode 100644 sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 create mode 100644 sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d create mode 100644 sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 create mode 100644 sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a create mode 100644 sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 create mode 100644 sql/hive/src/test/resources/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 create mode 100644 sql/hive/src/test/resources/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b create mode 100644 sql/hive/src/test/resources/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 create mode 100644 sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 create mode 100644 sql/hive/src/test/resources/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 create mode 100644 sql/hive/src/test/resources/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb create mode 100644 sql/hive/src/test/resources/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 create mode 100644 sql/hive/src/test/resources/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 create mode 100644 sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 create mode 100644 sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db create mode 100644 sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 create mode 100644 sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 create mode 100644 sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 create mode 100644 sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 create mode 100644 sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa create mode 100644 sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 create mode 100644 sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d create mode 100644 sql/hive/src/test/resources/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 create mode 100644 sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 create mode 100644 sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 create mode 100644 sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 create mode 100644 sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-16-d032f4795c1186255acea241387adf93 create mode 100644 sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 create mode 100644 sql/hive/src/test/resources/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f create mode 100644 sql/hive/src/test/resources/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a create mode 100644 sql/hive/src/test/resources/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 create mode 100644 sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a create mode 100644 sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 create mode 100644 sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd create mode 100644 sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 create mode 100644 sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 create mode 100644 sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 create mode 100644 sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 create mode 100644 sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-36-d032f4795c1186255acea241387adf93 create mode 100644 sql/hive/src/test/resources/golden/rename_column-37-9c36cac1372650b703400c60dd29042c create mode 100644 sql/hive/src/test/resources/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 create mode 100644 sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd create mode 100644 sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 create mode 100644 sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b create mode 100644 sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 create mode 100644 sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 create mode 100644 sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 create mode 100644 sql/hive/src/test/resources/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc create mode 100644 sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 create mode 100644 sql/hive/src/test/resources/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 create mode 100644 sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f create mode 100644 sql/hive/src/test/resources/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 create mode 100644 sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e create mode 100644 sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 create mode 100644 sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 create mode 100644 sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef create mode 100644 sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 create mode 100644 sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 create mode 100644 sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 create mode 100644 sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 create mode 100644 sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 create mode 100644 sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f create mode 100644 sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 create mode 100644 sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 create mode 100644 sql/hive/src/test/resources/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a create mode 100644 sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e create mode 100644 sql/hive/src/test/resources/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a create mode 100644 sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e create mode 100644 sql/hive/src/test/resources/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde create mode 100644 sql/hive/src/test/resources/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e create mode 100644 sql/hive/src/test/resources/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 create mode 100644 sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f create mode 100644 sql/hive/src/test/resources/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 create mode 100644 sql/hive/src/test/resources/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 create mode 100644 sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 create mode 100644 sql/hive/src/test/resources/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d create mode 100644 sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d create mode 100644 sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 create mode 100644 sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 create mode 100644 sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d create mode 100644 sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 create mode 100644 sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 create mode 100644 sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 create mode 100644 sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 create mode 100644 sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c create mode 100644 sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 create mode 100644 sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 create mode 100644 sql/hive/src/test/resources/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c create mode 100644 sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e create mode 100644 sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 create mode 100644 sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a create mode 100644 sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 create mode 100644 sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca create mode 100644 sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 create mode 100644 sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 create mode 100644 sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce create mode 100644 sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 create mode 100644 sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 create mode 100644 sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c create mode 100644 sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 create mode 100644 sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c create mode 100644 sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe create mode 100644 sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 create mode 100644 sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab create mode 100644 sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 create mode 100644 sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 create mode 100644 sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 create mode 100644 sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 create mode 100644 sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 create mode 100644 sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c create mode 100644 sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 create mode 100644 sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d create mode 100644 sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 create mode 100644 sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c create mode 100644 sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 create mode 100644 sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 create mode 100644 sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 create mode 100644 sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 create mode 100644 sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 create mode 100644 sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d create mode 100644 sql/hive/src/test/resources/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 create mode 100644 sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a create mode 100644 sql/hive/src/test/resources/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 create mode 100644 sql/hive/src/test/resources/golden/sort-0-eee35bbc888a705ae527625447668032 create mode 100644 sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf create mode 100644 sql/hive/src/test/resources/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec create mode 100644 sql/hive/src/test/resources/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c create mode 100644 sql/hive/src/test/resources/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 create mode 100644 sql/hive/src/test/resources/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 create mode 100644 sql/hive/src/test/resources/golden/stats0-13-ca0569fd4200d549843deadf84044649 create mode 100644 sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 create mode 100644 sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c create mode 100644 sql/hive/src/test/resources/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 create mode 100644 sql/hive/src/test/resources/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed create mode 100644 sql/hive/src/test/resources/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/stats0-20-5f7439f97275da70f633e135205f2095 create mode 100644 sql/hive/src/test/resources/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a create mode 100644 sql/hive/src/test/resources/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 create mode 100644 sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c create mode 100644 sql/hive/src/test/resources/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 create mode 100644 sql/hive/src/test/resources/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b create mode 100644 sql/hive/src/test/resources/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c create mode 100644 sql/hive/src/test/resources/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 create mode 100644 sql/hive/src/test/resources/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 create mode 100644 sql/hive/src/test/resources/golden/stats0-29-ca0569fd4200d549843deadf84044649 create mode 100644 sql/hive/src/test/resources/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 create mode 100644 sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c create mode 100644 sql/hive/src/test/resources/golden/stats0-4-5f7439f97275da70f633e135205f2095 create mode 100644 sql/hive/src/test/resources/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a create mode 100644 sql/hive/src/test/resources/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 create mode 100644 sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c create mode 100644 sql/hive/src/test/resources/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 create mode 100644 sql/hive/src/test/resources/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b create mode 100644 sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e create mode 100644 sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b create mode 100644 sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 create mode 100644 sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 create mode 100644 sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 create mode 100644 sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 create mode 100644 sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e create mode 100644 sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 create mode 100644 sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b create mode 100644 sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 create mode 100644 sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a create mode 100644 sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 create mode 100644 sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 create mode 100644 sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa create mode 100644 sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 create mode 100644 sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa create mode 100644 sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc create mode 100644 sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 create mode 100644 sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa create mode 100644 sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 create mode 100644 sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 create mode 100644 sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 create mode 100644 sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b create mode 100644 sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 create mode 100644 sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 create mode 100644 sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 create mode 100644 sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 create mode 100644 sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 create mode 100644 sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 create mode 100644 sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c create mode 100644 sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d create mode 100644 sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 create mode 100644 sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 create mode 100644 sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 create mode 100644 sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 create mode 100644 sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 create mode 100644 sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 create mode 100644 sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 create mode 100644 sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c create mode 100644 sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 create mode 100644 sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 create mode 100644 sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 create mode 100644 sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 create mode 100644 sql/hive/src/test/resources/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 create mode 100644 sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 create mode 100644 sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd create mode 100644 sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c create mode 100644 sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd create mode 100644 sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c create mode 100644 sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d create mode 100644 sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 create mode 100644 sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f create mode 100644 sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 create mode 100644 sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df create mode 100644 sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 create mode 100644 sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 create mode 100644 sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 create mode 100644 sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c create mode 100644 sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 create mode 100644 sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 create mode 100644 sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc create mode 100644 sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce create mode 100644 sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 create mode 100644 sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 create mode 100644 sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d create mode 100644 sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 create mode 100644 sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 create mode 100644 sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc create mode 100644 sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 create mode 100644 sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 create mode 100644 sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 create mode 100644 sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d create mode 100644 sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 create mode 100644 sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 create mode 100644 sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d create mode 100644 sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d create mode 100644 sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 create mode 100644 sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 create mode 100644 sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 create mode 100644 sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 create mode 100644 sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c create mode 100644 sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 create mode 100644 sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 create mode 100644 sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 create mode 100644 sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d create mode 100644 sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 create mode 100644 sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c create mode 100644 sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 create mode 100644 sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 create mode 100644 sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc create mode 100644 sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 create mode 100644 sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce create mode 100644 sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 create mode 100644 sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 create mode 100644 sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 create mode 100644 sql/hive/src/test/resources/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 create mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 create mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 create mode 100644 sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 create mode 100644 sql/hive/src/test/resources/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 create mode 100644 sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 create mode 100644 sql/hive/src/test/resources/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 create mode 100644 sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae create mode 100644 sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca create mode 100644 sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b create mode 100644 sql/hive/src/test/resources/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c create mode 100644 sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 create mode 100644 sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c create mode 100644 sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 create mode 100644 sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a create mode 100644 sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e create mode 100644 sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 create mode 100644 sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec create mode 100644 sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 create mode 100644 sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 create mode 100644 sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda create mode 100644 sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf create mode 100644 sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad create mode 100644 sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 create mode 100644 sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd create mode 100644 sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 create mode 100644 sql/hive/src/test/resources/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 create mode 100644 sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca create mode 100644 sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 create mode 100644 sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e create mode 100644 sql/hive/src/test/resources/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 create mode 100644 sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 create mode 100644 sql/hive/src/test/resources/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d create mode 100644 sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 create mode 100644 sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 create mode 100644 sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c create mode 100644 sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 create mode 100644 sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 create mode 100644 sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b create mode 100644 sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 create mode 100644 sql/hive/src/test/resources/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 create mode 100644 sql/hive/src/test/resources/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc create mode 100644 sql/hive/src/test/resources/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 create mode 100644 sql/hive/src/test/resources/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb create mode 100644 sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 create mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af create mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d create mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 create mode 100644 sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f create mode 100644 sql/hive/src/test/resources/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 create mode 100644 sql/hive/src/test/resources/golden/touch-1-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/touch-10-27ad2962fed131f51ba802596ba37278 create mode 100644 sql/hive/src/test/resources/golden/touch-2-3e95421993ab28d18245ec2340f580a3 create mode 100644 sql/hive/src/test/resources/golden/touch-3-c0c18ac884677231a41eea8d980d0451 create mode 100644 sql/hive/src/test/resources/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a create mode 100644 sql/hive/src/test/resources/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c create mode 100644 sql/hive/src/test/resources/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 create mode 100644 sql/hive/src/test/resources/golden/touch-7-3752320b12abae0d138148d56a27c5b1 create mode 100644 sql/hive/src/test/resources/golden/touch-8-3562ab40242756452595cd7eae79b0ce create mode 100644 sql/hive/src/test/resources/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 create mode 100644 sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe create mode 100644 sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 create mode 100644 sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 create mode 100644 sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 create mode 100644 sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 create mode 100644 sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 create mode 100644 sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 create mode 100644 sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f create mode 100644 sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 create mode 100644 sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 create mode 100644 sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 create mode 100644 sql/hive/src/test/resources/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 create mode 100644 sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 create mode 100644 sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d create mode 100644 sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 create mode 100644 sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 create mode 100644 sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 create mode 100644 sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 create mode 100644 sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 create mode 100644 sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a create mode 100644 sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 create mode 100644 sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 create mode 100644 sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 create mode 100644 sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 create mode 100644 sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 create mode 100644 sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 create mode 100644 sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f create mode 100644 sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 create mode 100644 sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 create mode 100644 sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da create mode 100644 sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f create mode 100644 sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 create mode 100644 sql/hive/src/test/resources/golden/udf2-2-31c8af064bac42541558a95fad3bca97 create mode 100644 sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 create mode 100644 sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c create mode 100644 sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 create mode 100644 sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e create mode 100644 sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 create mode 100644 sql/hive/src/test/resources/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 create mode 100644 sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 create mode 100644 sql/hive/src/test/resources/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 create mode 100644 sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e create mode 100644 sql/hive/src/test/resources/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 create mode 100644 sql/hive/src/test/resources/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 create mode 100644 sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 create mode 100644 sql/hive/src/test/resources/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/udf8-1-63c38297946a2060c0cff4a426b0520c create mode 100644 sql/hive/src/test/resources/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 create mode 100644 sql/hive/src/test/resources/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb create mode 100644 sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b create mode 100644 sql/hive/src/test/resources/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 create mode 100644 sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c create mode 100644 sql/hive/src/test/resources/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e create mode 100644 sql/hive/src/test/resources/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 create mode 100644 sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f create mode 100644 sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c create mode 100644 sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a create mode 100644 sql/hive/src/test/resources/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 create mode 100644 sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f create mode 100644 sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c create mode 100644 sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a create mode 100644 sql/hive/src/test/resources/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 create mode 100644 sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f create mode 100644 sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 create mode 100644 sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 create mode 100644 sql/hive/src/test/resources/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 create mode 100644 sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f create mode 100644 sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 create mode 100644 sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 create mode 100644 sql/hive/src/test/resources/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 create mode 100644 sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 create mode 100644 sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 create mode 100644 sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 create mode 100644 sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 create mode 100644 sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c create mode 100644 sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 create mode 100644 sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 create mode 100644 sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f create mode 100644 sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b create mode 100644 sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 create mode 100644 sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a create mode 100644 sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e create mode 100644 sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 create mode 100644 sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 create mode 100644 sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee create mode 100644 sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a create mode 100644 sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 create mode 100644 sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122 create mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a create mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c create mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 create mode 100644 sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa create mode 100644 sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd create mode 100644 sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f create mode 100644 sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 create mode 100644 sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b create mode 100644 sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b create mode 100644 sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 create mode 100644 sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 create mode 100644 sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 create mode 100644 sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f create mode 100644 sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 create mode 100644 sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 create mode 100644 sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b create mode 100644 sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 create mode 100644 sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b create mode 100644 sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 create mode 100644 sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b create mode 100644 sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a create mode 100644 sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b create mode 100644 sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f create mode 100644 sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d create mode 100644 sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 create mode 100644 sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb create mode 100644 sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 create mode 100644 sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f create mode 100644 sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 create mode 100644 sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e create mode 100644 sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 create mode 100644 sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 create mode 100644 sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f create mode 100644 sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c create mode 100644 sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 create mode 100644 sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c create mode 100644 sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 create mode 100644 sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 create mode 100644 sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 create mode 100644 sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb create mode 100644 sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd create mode 100644 sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df create mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 create mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 create mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 create mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 create mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 create mode 100644 sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 create mode 100644 sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 create mode 100644 sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea create mode 100644 sql/hive/src/test/resources/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c create mode 100644 sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a create mode 100644 sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab create mode 100644 sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d create mode 100644 sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 create mode 100644 sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a create mode 100644 sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e create mode 100644 sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb create mode 100644 sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 create mode 100644 sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 create mode 100644 sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 create mode 100644 sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f create mode 100644 sql/hive/src/test/resources/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 create mode 100644 sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e create mode 100644 sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e create mode 100644 sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 create mode 100644 sql/hive/src/test/resources/golden/udf_count-2-3344645abe535426307b9327b381fc85 create mode 100644 sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 create mode 100644 sql/hive/src/test/resources/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 create mode 100644 sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 create mode 100644 sql/hive/src/test/resources/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 create mode 100644 sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 create mode 100644 sql/hive/src/test/resources/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 create mode 100644 sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 create mode 100644 sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 create mode 100644 sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 create mode 100644 sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 create mode 100644 sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 create mode 100644 sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 create mode 100644 sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 create mode 100644 sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 create mode 100644 sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 create mode 100644 sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 create mode 100644 sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 create mode 100644 sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae create mode 100644 sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 create mode 100644 sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b create mode 100644 sql/hive/src/test/resources/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 create mode 100644 sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae create mode 100644 sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 create mode 100644 sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b create mode 100644 sql/hive/src/test/resources/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 create mode 100644 sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db create mode 100644 sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 create mode 100644 sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b create mode 100644 sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 create mode 100644 sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 create mode 100644 sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 create mode 100644 sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 create mode 100644 sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 create mode 100644 sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f create mode 100644 sql/hive/src/test/resources/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 create mode 100644 sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a create mode 100644 sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 create mode 100644 sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a create mode 100644 sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 create mode 100644 sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 create mode 100644 sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f create mode 100644 sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 create mode 100644 sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b create mode 100644 sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 create mode 100644 sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 create mode 100644 sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 create mode 100644 sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 create mode 100644 sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef create mode 100644 sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 create mode 100644 sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 create mode 100644 sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 create mode 100644 sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 create mode 100644 sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b create mode 100644 sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 create mode 100644 sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e create mode 100644 sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 create mode 100644 sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f create mode 100644 sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 create mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f create mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 create mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 create mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 create mode 100644 sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 create mode 100644 sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 create mode 100644 sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 create mode 100644 sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a create mode 100644 sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 create mode 100644 sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae create mode 100644 sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 create mode 100644 sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c create mode 100644 sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 create mode 100644 sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf create mode 100644 sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea create mode 100644 sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce create mode 100644 sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 create mode 100644 sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a create mode 100644 sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 create mode 100644 sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c create mode 100644 sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a create mode 100644 sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 create mode 100644 sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f create mode 100644 sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca create mode 100644 sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc create mode 100644 sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 create mode 100644 sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 create mode 100644 sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 create mode 100644 sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 create mode 100644 sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 create mode 100644 sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 create mode 100644 sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 create mode 100644 sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 create mode 100644 sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 create mode 100644 sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 create mode 100644 sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba create mode 100644 sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 create mode 100644 sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 create mode 100644 sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 create mode 100644 sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 create mode 100644 sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd create mode 100644 sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad create mode 100644 sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e create mode 100644 sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b create mode 100644 sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 create mode 100644 sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 create mode 100644 sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 create mode 100644 sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d create mode 100644 sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 create mode 100644 sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f create mode 100644 sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a create mode 100644 sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f create mode 100644 sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf create mode 100644 sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b create mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 create mode 100644 sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c create mode 100644 sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 create mode 100644 sql/hive/src/test/resources/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a create mode 100644 sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 create mode 100644 sql/hive/src/test/resources/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc create mode 100644 sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 create mode 100644 sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 create mode 100644 sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 create mode 100644 sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e create mode 100644 sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 create mode 100644 sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c create mode 100644 sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 create mode 100644 sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a create mode 100644 sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 create mode 100644 sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f create mode 100644 sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 create mode 100644 sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 create mode 100644 sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 create mode 100644 sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 create mode 100644 sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 create mode 100644 sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c create mode 100644 sql/hive/src/test/resources/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e create mode 100644 sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 create mode 100644 sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae create mode 100644 sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f create mode 100644 sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 create mode 100644 sql/hive/src/test/resources/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e create mode 100644 sql/hive/src/test/resources/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf create mode 100644 sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 create mode 100644 sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 create mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa create mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f create mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e create mode 100644 sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a create mode 100644 sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 create mode 100644 sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 create mode 100644 sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 create mode 100644 sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e create mode 100644 sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f create mode 100644 sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 create mode 100644 sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 create mode 100644 sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 create mode 100644 sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 create mode 100644 sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 create mode 100644 sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 create mode 100644 sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 create mode 100644 sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 create mode 100644 sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 create mode 100644 sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 create mode 100644 sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 create mode 100644 sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 create mode 100644 sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee create mode 100644 sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 create mode 100644 sql/hive/src/test/resources/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 create mode 100644 sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c create mode 100644 sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 create mode 100644 sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 create mode 100644 sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee create mode 100644 sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 create mode 100644 sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 create mode 100644 sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf create mode 100644 sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c create mode 100644 sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 create mode 100644 sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c create mode 100644 sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e create mode 100644 sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 create mode 100644 sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b create mode 100644 sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 create mode 100644 sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa create mode 100644 sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc create mode 100644 sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 create mode 100644 sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 create mode 100644 sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 create mode 100644 sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd create mode 100644 sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def create mode 100644 sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 create mode 100644 sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 create mode 100644 sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 create mode 100644 sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 create mode 100644 sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab create mode 100644 sql/hive/src/test/resources/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d create mode 100644 sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 create mode 100644 sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 create mode 100644 sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a create mode 100644 sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 create mode 100644 sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd create mode 100644 sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 create mode 100644 sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 create mode 100644 sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c create mode 100644 sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 create mode 100644 sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 create mode 100644 sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 create mode 100644 sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 create mode 100644 sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 create mode 100644 sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 create mode 100644 sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 create mode 100644 sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 create mode 100644 sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e create mode 100644 sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 create mode 100644 sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee create mode 100644 sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 create mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 create mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 create mode 100644 sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca create mode 100644 sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 create mode 100644 sql/hive/src/test/resources/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 create mode 100644 sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 create mode 100644 sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 create mode 100644 sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb create mode 100644 sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e create mode 100644 sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a create mode 100644 sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 create mode 100644 sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 create mode 100644 sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca create mode 100644 sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 create mode 100644 sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 create mode 100644 sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 create mode 100644 sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c create mode 100644 sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 create mode 100644 sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a create mode 100644 sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b create mode 100644 sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 create mode 100644 sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce create mode 100644 sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 create mode 100644 sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 create mode 100644 sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 create mode 100644 sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 create mode 100644 sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 create mode 100644 sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 create mode 100644 sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a create mode 100644 sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd create mode 100644 sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 create mode 100644 sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 create mode 100644 sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f create mode 100644 sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 create mode 100644 sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 create mode 100644 sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e create mode 100644 sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 create mode 100644 sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa create mode 100644 sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a create mode 100644 sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef create mode 100644 sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d create mode 100644 sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa create mode 100644 sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a create mode 100644 sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e create mode 100644 sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 create mode 100644 sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef create mode 100644 sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d create mode 100644 sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee create mode 100644 sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 create mode 100644 sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab create mode 100644 sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 create mode 100644 sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 create mode 100644 sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 create mode 100644 sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 create mode 100644 sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd create mode 100644 sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 create mode 100644 sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 create mode 100644 sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a create mode 100644 sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 create mode 100644 sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 create mode 100644 sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 create mode 100644 sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 create mode 100644 sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 create mode 100644 sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 create mode 100644 sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 create mode 100644 sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 create mode 100644 sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 create mode 100644 sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 create mode 100644 sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 create mode 100644 sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b create mode 100644 sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 create mode 100644 sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 create mode 100644 sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d create mode 100644 sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 create mode 100644 sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 create mode 100644 sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 create mode 100644 sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 create mode 100644 sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 create mode 100644 sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 create mode 100644 sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 create mode 100644 sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d create mode 100644 sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd create mode 100644 sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 create mode 100644 sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 create mode 100644 sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 create mode 100644 sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a create mode 100644 sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 create mode 100644 sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 create mode 100644 sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 create mode 100644 sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece create mode 100644 sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 create mode 100644 sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 create mode 100644 sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 create mode 100644 sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 create mode 100644 sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b create mode 100644 sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 create mode 100644 sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b create mode 100644 sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 create mode 100644 sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 create mode 100644 sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad create mode 100644 sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 create mode 100644 sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d create mode 100644 sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a create mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 create mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 create mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc create mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b create mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 create mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 create mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 create mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 create mode 100644 sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 create mode 100644 sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 create mode 100644 sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 create mode 100644 sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 create mode 100644 sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c create mode 100644 sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a create mode 100644 sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 create mode 100644 sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 create mode 100644 sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 create mode 100644 sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 create mode 100644 sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d create mode 100644 sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 create mode 100644 sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e create mode 100644 sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 create mode 100644 sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a create mode 100644 sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b create mode 100644 sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 create mode 100644 sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce create mode 100644 sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 create mode 100644 sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 create mode 100644 sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 create mode 100644 sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d create mode 100644 sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea create mode 100644 sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b create mode 100644 sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b create mode 100644 sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a create mode 100644 sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba create mode 100644 sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b create mode 100644 sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 create mode 100644 sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 create mode 100644 sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f create mode 100644 sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 create mode 100644 sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d create mode 100644 sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f create mode 100644 sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 create mode 100644 sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 create mode 100644 sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 create mode 100644 sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 create mode 100644 sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c create mode 100644 sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 create mode 100644 sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 create mode 100644 sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 create mode 100644 sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead create mode 100644 sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d create mode 100644 sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de create mode 100644 sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 create mode 100644 sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 create mode 100644 sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b create mode 100644 sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 create mode 100644 sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b create mode 100644 sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 create mode 100644 sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a create mode 100644 sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de create mode 100644 sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 create mode 100644 sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 create mode 100644 sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a create mode 100644 sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de create mode 100644 sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 create mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 create mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd create mode 100644 sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 create mode 100644 sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 create mode 100644 sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 create mode 100644 sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 create mode 100644 sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc create mode 100644 sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 create mode 100644 sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a create mode 100644 sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 create mode 100644 sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 create mode 100644 sql/hive/src/test/resources/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 create mode 100644 sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 create mode 100644 sql/hive/src/test/resources/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 create mode 100644 sql/hive/src/test/resources/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e create mode 100644 sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 create mode 100644 sql/hive/src/test/resources/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 create mode 100644 sql/hive/src/test/resources/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 create mode 100644 sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 create mode 100644 sql/hive/src/test/resources/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 create mode 100644 sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af create mode 100644 sql/hive/src/test/resources/golden/union10-2-60ee9eae687170317ff91dafe6b799bf create mode 100644 sql/hive/src/test/resources/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 create mode 100644 sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 create mode 100644 sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 create mode 100644 sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f create mode 100644 sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 create mode 100644 sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 create mode 100644 sql/hive/src/test/resources/golden/union13-0-6970d1e2405f3769a28a8804887ac657 create mode 100644 sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 create mode 100644 sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/union14-1-11603fb6225423979af6c062cfa9389b create mode 100644 sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 create mode 100644 sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 create mode 100644 sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 create mode 100644 sql/hive/src/test/resources/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 create mode 100644 sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 create mode 100644 sql/hive/src/test/resources/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a create mode 100644 sql/hive/src/test/resources/golden/union17-1-9491c271b7e2c351ddcf126f73679892 create mode 100644 sql/hive/src/test/resources/golden/union17-2-e6fb104913b9da8193167ee263993dd1 create mode 100644 sql/hive/src/test/resources/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 create mode 100644 sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 create mode 100644 sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 create mode 100644 sql/hive/src/test/resources/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a create mode 100644 sql/hive/src/test/resources/golden/union18-1-9491c271b7e2c351ddcf126f73679892 create mode 100644 sql/hive/src/test/resources/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 create mode 100644 sql/hive/src/test/resources/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 create mode 100644 sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 create mode 100644 sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 create mode 100644 sql/hive/src/test/resources/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a create mode 100644 sql/hive/src/test/resources/golden/union19-1-9491c271b7e2c351ddcf126f73679892 create mode 100644 sql/hive/src/test/resources/golden/union19-2-b6161b64e57a2502c79662866782cbdf create mode 100644 sql/hive/src/test/resources/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 create mode 100644 sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 create mode 100644 sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 create mode 100644 sql/hive/src/test/resources/golden/union2-0-58813daf3d1af57b17518b606f8345b2 create mode 100644 sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 create mode 100644 sql/hive/src/test/resources/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d create mode 100644 sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c create mode 100644 sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c create mode 100644 sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 create mode 100644 sql/hive/src/test/resources/golden/union22-0-4bd6583b3635f1db95765565518bda8d create mode 100644 sql/hive/src/test/resources/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 create mode 100644 sql/hive/src/test/resources/golden/union22-10-90e9c4388340428a1f68868e9322c400 create mode 100644 sql/hive/src/test/resources/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 create mode 100644 sql/hive/src/test/resources/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 create mode 100644 sql/hive/src/test/resources/golden/union22-4-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union22-5-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 create mode 100644 sql/hive/src/test/resources/golden/union22-7-25fc734982956a164adde6bb1d4d8751 create mode 100644 sql/hive/src/test/resources/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 create mode 100644 sql/hive/src/test/resources/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 create mode 100644 sql/hive/src/test/resources/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 create mode 100644 sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd create mode 100644 sql/hive/src/test/resources/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 create mode 100644 sql/hive/src/test/resources/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c create mode 100644 sql/hive/src/test/resources/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 create mode 100644 sql/hive/src/test/resources/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a create mode 100644 sql/hive/src/test/resources/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 create mode 100644 sql/hive/src/test/resources/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f create mode 100644 sql/hive/src/test/resources/golden/union24-4-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 create mode 100644 sql/hive/src/test/resources/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 create mode 100644 sql/hive/src/test/resources/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb create mode 100644 sql/hive/src/test/resources/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 create mode 100644 sql/hive/src/test/resources/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d create mode 100644 sql/hive/src/test/resources/golden/union26-1-4702087f160230463e6114d6534d47e4 create mode 100644 sql/hive/src/test/resources/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 create mode 100644 sql/hive/src/test/resources/golden/union26-3-4702087f160230463e6114d6534d47e4 create mode 100644 sql/hive/src/test/resources/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 create mode 100644 sql/hive/src/test/resources/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f create mode 100644 sql/hive/src/test/resources/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 create mode 100644 sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 create mode 100644 sql/hive/src/test/resources/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 create mode 100644 sql/hive/src/test/resources/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 create mode 100644 sql/hive/src/test/resources/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f create mode 100644 sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 create mode 100644 sql/hive/src/test/resources/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 create mode 100644 sql/hive/src/test/resources/golden/union29-1-a40036f756a306f4226634b6ab67c72c create mode 100644 sql/hive/src/test/resources/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 create mode 100644 sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 create mode 100644 sql/hive/src/test/resources/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d create mode 100644 sql/hive/src/test/resources/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 create mode 100644 sql/hive/src/test/resources/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 create mode 100644 sql/hive/src/test/resources/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 create mode 100644 sql/hive/src/test/resources/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 create mode 100644 sql/hive/src/test/resources/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 create mode 100644 sql/hive/src/test/resources/golden/union30-2-252f12f7532ca67132bfc62063c58430 create mode 100644 sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 create mode 100644 sql/hive/src/test/resources/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 sql/hive/src/test/resources/golden/union31-1-b12aa87aaf64b573ce0be7013117651b create mode 100644 sql/hive/src/test/resources/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 create mode 100644 sql/hive/src/test/resources/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 create mode 100644 sql/hive/src/test/resources/golden/union31-12-de2477a134ab37bb22d879028becf2e5 create mode 100644 sql/hive/src/test/resources/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 create mode 100644 sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 create mode 100644 sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e create mode 100644 sql/hive/src/test/resources/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 create mode 100644 sql/hive/src/test/resources/golden/union31-17-b12aa87aaf64b573ce0be7013117651b create mode 100644 sql/hive/src/test/resources/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 create mode 100644 sql/hive/src/test/resources/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e create mode 100644 sql/hive/src/test/resources/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 create mode 100644 sql/hive/src/test/resources/golden/union31-20-dceb6d9a03096d68d090557692ed6899 create mode 100644 sql/hive/src/test/resources/golden/union31-21-bda2e62d0ecd30200b2359df420da574 create mode 100644 sql/hive/src/test/resources/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a create mode 100644 sql/hive/src/test/resources/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df create mode 100644 sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 create mode 100644 sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 create mode 100644 sql/hive/src/test/resources/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc create mode 100644 sql/hive/src/test/resources/golden/union31-4-77653483e5ce18437c9f85d3b297335d create mode 100644 sql/hive/src/test/resources/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 create mode 100644 sql/hive/src/test/resources/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 create mode 100644 sql/hive/src/test/resources/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 create mode 100644 sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb create mode 100644 sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 create mode 100644 sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e create mode 100644 sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 create mode 100644 sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b create mode 100644 sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 create mode 100644 sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 create mode 100644 sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 create mode 100644 sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e create mode 100644 sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 create mode 100644 sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b create mode 100644 sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 create mode 100644 sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb create mode 100644 sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db create mode 100644 sql/hive/src/test/resources/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f create mode 100644 sql/hive/src/test/resources/golden/union33-1-1df7c476abb48234c839878414720873 create mode 100644 sql/hive/src/test/resources/golden/union33-2-3987150ad66482f5186826efcfb117f7 create mode 100644 sql/hive/src/test/resources/golden/union33-3-bfe75625806b414a5098af487b91046a create mode 100644 sql/hive/src/test/resources/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 create mode 100644 sql/hive/src/test/resources/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 create mode 100644 sql/hive/src/test/resources/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 create mode 100644 sql/hive/src/test/resources/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 create mode 100644 sql/hive/src/test/resources/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 create mode 100644 sql/hive/src/test/resources/golden/union34-1-320c2fe802fab386957039aaff6399f1 create mode 100644 sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 create mode 100644 sql/hive/src/test/resources/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 create mode 100644 sql/hive/src/test/resources/golden/union34-3-c179e14d840249d6953cb552279822f1 create mode 100644 sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 create mode 100644 sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f create mode 100644 sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 create mode 100644 sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b create mode 100644 sql/hive/src/test/resources/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 create mode 100644 sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/union4-1-88deb3987655dc41989f39c0827cec74 create mode 100644 sql/hive/src/test/resources/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 create mode 100644 sql/hive/src/test/resources/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b create mode 100644 sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 create mode 100644 sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 create mode 100644 sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 create mode 100644 sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d create mode 100644 sql/hive/src/test/resources/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 create mode 100644 sql/hive/src/test/resources/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 create mode 100644 sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 create mode 100644 sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 create mode 100644 sql/hive/src/test/resources/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 create mode 100644 sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 create mode 100644 sql/hive/src/test/resources/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e create mode 100644 sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 create mode 100644 sql/hive/src/test/resources/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 create mode 100644 sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 create mode 100644 sql/hive/src/test/resources/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 create mode 100644 sql/hive/src/test/resources/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c create mode 100644 sql/hive/src/test/resources/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 create mode 100644 sql/hive/src/test/resources/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 create mode 100644 sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c create mode 100644 sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d create mode 100644 sql/hive/src/test/resources/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 create mode 100644 sql/hive/src/test/resources/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 create mode 100644 sql/hive/src/test/resources/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 create mode 100644 sql/hive/src/test/resources/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 create mode 100644 sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a create mode 100644 sql/hive/src/test/resources/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b create mode 100644 sql/hive/src/test/resources/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 create mode 100644 sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb create mode 100644 sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d create mode 100644 sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d create mode 100644 sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b create mode 100644 sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 create mode 100644 sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a create mode 100644 sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a create mode 100644 sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf create mode 100644 sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 create mode 100644 sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e create mode 100644 sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 create mode 100644 sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 create mode 100644 sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d create mode 100644 sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 create mode 100644 sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 create mode 100644 sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 create mode 100644 sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e create mode 100644 sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 create mode 100644 sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae create mode 100644 sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 create mode 100644 sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 create mode 100644 sql/hive/src/test/resources/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af create mode 100644 sql/hive/src/test/resources/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 create mode 100644 sql/hive/src/test/resources/golden/varchar_2-2-779a555944479991520be53f7f247626 create mode 100644 sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da create mode 100644 sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f create mode 100644 sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 create mode 100644 sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f create mode 100644 sql/hive/src/test/resources/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 create mode 100644 sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 create mode 100644 sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 create mode 100644 sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 create mode 100644 sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 create mode 100644 sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a create mode 100644 sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 create mode 100644 sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 create mode 100644 sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 create mode 100644 sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 create mode 100644 sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 create mode 100644 sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee create mode 100644 sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b create mode 100644 sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 create mode 100644 sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a create mode 100644 sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 create mode 100644 sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b create mode 100644 sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 create mode 100644 sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 create mode 100644 sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 create mode 100644 sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 create mode 100644 sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf create mode 100644 sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb create mode 100644 sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 create mode 100644 sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de create mode 100644 sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 create mode 100644 sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 create mode 100644 sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a create mode 100644 sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 create mode 100644 sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb create mode 100644 sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 create mode 100644 sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 create mode 100644 sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 create mode 100644 sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe create mode 100644 sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d create mode 100644 sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 create mode 100644 sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 create mode 100644 sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 create mode 100644 sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 create mode 100644 sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 create mode 100644 sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 create mode 100644 sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 create mode 100644 sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 create mode 100644 sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 create mode 100644 sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_part_exist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_part_multiple.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alias_casted_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/allcolref_in_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_format_loc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_skewed_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_table_not_sorted.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_table_serde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_table_serde2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_view_as_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_view_rename.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_multi.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join17.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join19.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join21.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join22.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join24.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join26.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join27.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join28.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join29.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join30.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join31.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/autogen_colalias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_change_schema.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_schema_error_message.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_schema_literal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table_udfs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table_union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_output_format.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_bincolserde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/case_sensitivity.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cluster.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_empty_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/convert_enum_to_string.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cp_mj_rc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_big_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_default_prop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_escape.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_genericudaf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_genericudf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_insert_outputformat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like_tbl_props.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_or_replace_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_skewed_table1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_udaf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ct_case_insensitive.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_colname.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_uses_database_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_varchar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_properties.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ddltime.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/default_partition_name.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_comment_indent.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_comment_nonascii.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_database_json.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_pretty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_table_json.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_xpath.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/diff_part_input_formats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/driverhook.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_function.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_multi_partitions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_table2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/enforce_order.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_clusterby1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_distributeby1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_orderby1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_sortby1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_dependency.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_dependency2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_logical.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explode_null.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fetch_aggregation.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fileformat_mix.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fileformat_sequencefile.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fileformat_text.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_nomap.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_complex_types.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_neg_float.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_position.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_ppd.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_ppr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/having.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_order.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/implicit_cast1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_update.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/init_file.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/innerjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inoutdriver.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input11_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input14_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input17.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input1_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input24.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input25.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input26.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input28.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input2_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input30.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input31.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input32.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input34.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input35.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input36.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input38.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input41.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input42.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input46.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input49.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_columnarserde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dynamicserde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_lazyserde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_limit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part10_win.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part9.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_compressed.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insertexternal1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join17.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join18_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join19.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join21.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join22.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join24.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join25.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join26.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join27.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join28.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join29.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join30.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join31.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join32.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join32_lessSize.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join33.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join34.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join35.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join36.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join37.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join38.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join39.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join40.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join41.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_empty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters_overlap.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_literals.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_map_ppr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_rc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_thrift.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_vc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/keyword_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_cp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_outer.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lineage1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part14_win.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/louter_join_ppr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_test_outer.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mergejoins.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mergejoins_mixed.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadataonly1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/misc_json.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_gby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_gby2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_gby3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_join_union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_sahooks.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multigroupby_singlemr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nestedvirtual.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/no_hooks.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/noalias_subq1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullinput.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullinput2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optional_outer.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_files.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/outer_join_ppr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/overridden_confs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parenthesis_star_by.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partInit.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_schema1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_serde_format.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partitions_json.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/plan_json.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_clusterby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_constant_expr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_constant_where.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_gby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_gby2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_gby_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join_filter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_random.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_repeated_alias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udf_case.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udf_col.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_vc.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/print_header.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/protectmode.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/protectmode2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/push_or.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/query_properties.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/query_result_fileformat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/query_with_semi.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rand_partitionpruner1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rand_partitionpruner2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rand_partitionpruner3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_columnar.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_default_format.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_null_value.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/recursive_dir.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reducesink_dedup.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regexp_extract.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_external_partition_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_partition_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_table_location.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reset_conf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/router_join_ppr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/script_env_var1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/script_env_var2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/script_pipe.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_as_omitted.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_transform_hint.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_unquote_and.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_unquote_not.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_unquote_or.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/semicolon.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/semijoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_reported_schema.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_user_properties.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_variable_sub.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_columns.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_alter.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_db_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_delimited.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_partitioned.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_serde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_describe_func_quotes.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_functions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tables.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tblproperties.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/showparts.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_17.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_19.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_21.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_22.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats0.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_empty_partition.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tablename_with_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/touch.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform_ppr1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform_ppr2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_buckets.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_number_format.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_10_trims.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_add.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_avg.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bigint.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_and.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_not.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_or.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_xor.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_boolean.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_column_pruning.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ceil.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ceiling.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_insert1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_insert2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_context_aware.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_count.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_date_add.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_date_sub.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_datediff.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_day.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_dayofmonth.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_double.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_exp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_float.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_floor.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_from_unixtime.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_int.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnotnull.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lcase.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthanorequal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_like.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ln.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_locate.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_log.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_log10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_log2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_logic_java_boolean.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lower.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lpad.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ltrim.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_keys.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_map_values.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_max.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_min.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_minute.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_modulo.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_month.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_named_struct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_negative.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_not.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notequal.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_or.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_parse_url.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_percentile.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_positive.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pow.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_power.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rand.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp_extract.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp_replace.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rlike.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rtrim.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sentences.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_smallint.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sqrt.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_std.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev_pop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev_samp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_string.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substring.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_subtract.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sum.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tinyint.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_trim.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ucase.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_upper.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_var_pop.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_var_samp.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_variance.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_stack.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/unicode_notation.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union17.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union19.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union21.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union22.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union24.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union25.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union26.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union27.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union28.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union29.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union30.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union31.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union32.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union33.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_lateralview.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_ppr.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_script.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/unset_table_view_property.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/updateAccessTime.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_nested_types.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_udf1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_inputs.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/virtual_column.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q create mode 100755 sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q create mode 100644 sql/hive/src/test/resources/ql/src/test/queries/positive/union.q diff --git a/dev/download-hive-tests.sh b/dev/download-hive-tests.sh deleted file mode 100755 index 6c412a8493c18..0000000000000 --- a/dev/download-hive-tests.sh +++ /dev/null @@ -1,4 +0,0 @@ -#!/bin/sh - -wget -O hiveTests.tgz http://cs.berkeley.edu/~marmbrus/tmp/hiveTests.tgz -tar zxf hiveTests.tgz \ No newline at end of file diff --git a/dev/run-tests b/dev/run-tests index b62a25f42599e..cf0b940c09a81 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -21,9 +21,6 @@ FWDIR="$(cd `dirname $0`/..; pwd)" cd $FWDIR -# Download Hive Compatability Files -dev/download-hive-tests.sh - # Remove work directory rm -rf ./work diff --git a/sql/hive/src/test/resources/data/conf/hive-log4j.properties b/sql/hive/src/test/resources/data/conf/hive-log4j.properties new file mode 100644 index 0000000000000..885c86f2b94f4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/conf/hive-site.xml b/sql/hive/src/test/resources/data/conf/hive-site.xml new file mode 100644 index 0000000000000..4e6ff16135833 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/SortCol1Col2.txt b/sql/hive/src/test/resources/data/files/SortCol1Col2.txt new file mode 100644 index 0000000000000..21f11729cc002 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/SortCol1Col2.txt @@ -0,0 +1,4 @@ +110 +111 +210 +211 diff --git a/sql/hive/src/test/resources/data/files/SortCol2Col1.txt b/sql/hive/src/test/resources/data/files/SortCol2Col1.txt new file mode 100644 index 0000000000000..83ad1ea0f89f3 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/SortCol2Col1.txt @@ -0,0 +1,5 @@ +110 +210 +111 +211 + diff --git a/sql/hive/src/test/resources/data/files/SortDescCol1Col2.txt b/sql/hive/src/test/resources/data/files/SortDescCol1Col2.txt new file mode 100644 index 0000000000000..d18cfac40495e --- /dev/null +++ b/sql/hive/src/test/resources/data/files/SortDescCol1Col2.txt @@ -0,0 +1,4 @@ +211 +210 +111 +110 diff --git a/sql/hive/src/test/resources/data/files/SortDescCol2Col1.txt b/sql/hive/src/test/resources/data/files/SortDescCol2Col1.txt new file mode 100644 index 0000000000000..ea929de39006c --- /dev/null +++ b/sql/hive/src/test/resources/data/files/SortDescCol2Col1.txt @@ -0,0 +1,4 @@ +211 +111 +210 +110 diff --git a/sql/hive/src/test/resources/data/files/T1.txt b/sql/hive/src/test/resources/data/files/T1.txt new file mode 100644 index 0000000000000..79158bee62d21 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/T1.txt @@ -0,0 +1,6 @@ +111 +212 +313 +717 +818 +828 diff --git a/sql/hive/src/test/resources/data/files/T2.txt b/sql/hive/src/test/resources/data/files/T2.txt new file mode 100644 index 0000000000000..ebd65dbdb9031 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/T2.txt @@ -0,0 +1,6 @@ +222 +313 +414 +515 +818 +818 diff --git a/sql/hive/src/test/resources/data/files/T3.txt b/sql/hive/src/test/resources/data/files/T3.txt new file mode 100644 index 0000000000000..957d4c8cfc0bb --- /dev/null +++ b/sql/hive/src/test/resources/data/files/T3.txt @@ -0,0 +1,4 @@ +212 +414 +616 +717 diff --git a/sql/hive/src/test/resources/data/files/TestSerDe.jar b/sql/hive/src/test/resources/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

    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/sql/hive/src/test/resources/data/files/UserVisits.dat b/sql/hive/src/test/resources/data/files/UserVisits.dat new file mode 100644 index 0000000000000..f56c5a1b67b36 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/apache.access.2.log b/sql/hive/src/test/resources/data/files/apache.access.2.log new file mode 100644 index 0000000000000..cd7fea4e3fb5d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/apache.access.log b/sql/hive/src/test/resources/data/files/apache.access.log new file mode 100644 index 0000000000000..14c341efc16ed --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/archive_corrupt.rc b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/array_table.txt b/sql/hive/src/test/resources/data/files/array_table.txt new file mode 100644 index 0000000000000..463365f71b81b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/binary.txt b/sql/hive/src/test/resources/data/files/binary.txt new file mode 100644 index 0000000000000..c6fbdee959041 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/bool.txt b/sql/hive/src/test/resources/data/files/bool.txt new file mode 100644 index 0000000000000..065359b9f9325 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/complex.seq b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/docurl.txt b/sql/hive/src/test/resources/data/files/docurl.txt new file mode 100644 index 0000000000000..5862d78b4bc53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/double.txt b/sql/hive/src/test/resources/data/files/double.txt new file mode 100644 index 0000000000000..66c030bc47252 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/employee.dat b/sql/hive/src/test/resources/data/files/employee.dat new file mode 100644 index 0000000000000..c9d0197539efc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/employee2.dat b/sql/hive/src/test/resources/data/files/employee2.dat new file mode 100644 index 0000000000000..ad2d99a112511 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/employee_part.txt b/sql/hive/src/test/resources/data/files/employee_part.txt new file mode 100644 index 0000000000000..8a10b9ea0298c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/empty1.txt b/sql/hive/src/test/resources/data/files/empty1.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/data/files/empty2.txt b/sql/hive/src/test/resources/data/files/empty2.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/data/files/episodes.avro b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/kv1.string-sorted.txt b/sql/hive/src/test/resources/data/files/kv1.string-sorted.txt new file mode 100644 index 0000000000000..5c1575d34285e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/kv1.txt b/sql/hive/src/test/resources/data/files/kv1.txt new file mode 100644 index 0000000000000..9825414ecf8f2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/kv1.val.sorted.txt b/sql/hive/src/test/resources/data/files/kv1.val.sorted.txt new file mode 100644 index 0000000000000..916a0f13c70f6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/kv1_broken.seq b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/map_table.txt b/sql/hive/src/test/resources/data/files/map_table.txt new file mode 100644 index 0000000000000..f3a01d8f180be --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/nested_complex.txt b/sql/hive/src/test/resources/data/files/nested_complex.txt new file mode 100644 index 0000000000000..fee462c44e349 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/nested_complex.txt @@ -0,0 +1,2 @@ +3012k1v1k2v2a102 +2032k1v1k3v3b102 diff --git a/sql/hive/src/test/resources/data/files/null.txt b/sql/hive/src/test/resources/data/files/null.txt new file mode 100644 index 0000000000000..7014a64126de4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/nullfile.txt b/sql/hive/src/test/resources/data/files/nullfile.txt new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/data/files/nulls.txt b/sql/hive/src/test/resources/data/files/nulls.txt new file mode 100644 index 0000000000000..21cb1b5627c62 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/orc_create.txt b/sql/hive/src/test/resources/data/files/orc_create.txt new file mode 100644 index 0000000000000..211e047346309 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/orc_create_people.txt b/sql/hive/src/test/resources/data/files/orc_create_people.txt new file mode 100644 index 0000000000000..884598981a13c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/over10k b/sql/hive/src/test/resources/data/files/over10k new file mode 100644 index 0000000000000..adc75519ee3e4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/over1k b/sql/hive/src/test/resources/data/files/over1k new file mode 100644 index 0000000000000..1b3a052be97e8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/part.rc b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/part.seq b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/part_tiny.txt b/sql/hive/src/test/resources/data/files/part_tiny.txt new file mode 100644 index 0000000000000..f6973fd90a226 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/person age.txt b/sql/hive/src/test/resources/data/files/person age.txt new file mode 100644 index 0000000000000..c902284c11155 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/primitive_type_arrays.txt b/sql/hive/src/test/resources/data/files/primitive_type_arrays.txt new file mode 100644 index 0000000000000..2ab5f42c55979 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/pw17.txt b/sql/hive/src/test/resources/data/files/pw17.txt new file mode 100644 index 0000000000000..041944a1b022b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/sales.txt b/sql/hive/src/test/resources/data/files/sales.txt new file mode 100644 index 0000000000000..6407b72051b51 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/sales.txt @@ -0,0 +1,2 @@ +Joe 2 +Hank 2 diff --git a/sql/hive/src/test/resources/data/files/sample-queryplan-in-history.txt b/sql/hive/src/test/resources/data/files/sample-queryplan-in-history.txt new file mode 100644 index 0000000000000..79c034dabed01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/sample-queryplan.txt b/sql/hive/src/test/resources/data/files/sample-queryplan.txt new file mode 100644 index 0000000000000..3b9167b4ac718 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/smallsrcsortbucket1outof4.txt b/sql/hive/src/test/resources/data/files/smallsrcsortbucket1outof4.txt new file mode 100644 index 0000000000000..1175eae354dc9 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/smallsrcsortbucket1outof4.txt @@ -0,0 +1,5 @@ +0val_0 +103val_103 +169val_169 +172val_172 +374val_374 diff --git a/sql/hive/src/test/resources/data/files/smallsrcsortbucket2outof4.txt b/sql/hive/src/test/resources/data/files/smallsrcsortbucket2outof4.txt new file mode 100644 index 0000000000000..59347ff9a9170 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/smallsrcsortbucket2outof4.txt @@ -0,0 +1,5 @@ +180val_180 +221val_221 +379val_379 +478val_478 +74val_74 diff --git a/sql/hive/src/test/resources/data/files/smallsrcsortbucket3outof4.txt b/sql/hive/src/test/resources/data/files/smallsrcsortbucket3outof4.txt new file mode 100644 index 0000000000000..354f45da73862 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/smallsrcsortbucket3outof4.txt @@ -0,0 +1,5 @@ +233val_233 +424val_424 +468val_468 +53val_53 +97val_97 diff --git a/sql/hive/src/test/resources/data/files/smallsrcsortbucket4outof4.txt b/sql/hive/src/test/resources/data/files/smallsrcsortbucket4outof4.txt new file mode 100644 index 0000000000000..6aecf76a4022e --- /dev/null +++ b/sql/hive/src/test/resources/data/files/smallsrcsortbucket4outof4.txt @@ -0,0 +1,5 @@ +146val_146 +193val_193 +432val_432 +65val_65 +83val_83 diff --git a/sql/hive/src/test/resources/data/files/smb_bucket_input.rc b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/smb_bucket_input.txt b/sql/hive/src/test/resources/data/files/smb_bucket_input.txt new file mode 100644 index 0000000000000..e3a7e2977bc68 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/smbbucket_1.rc b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/smbbucket_2.txt b/sql/hive/src/test/resources/data/files/smbbucket_2.txt new file mode 100644 index 0000000000000..430a0ca887730 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/smbbucket_2.txt @@ -0,0 +1,4 @@ +20val_20 +23val_23 +25val_25 +30val_30 diff --git a/sql/hive/src/test/resources/data/files/smbbucket_3.rc b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/symlink1.txt b/sql/hive/src/test/resources/data/files/symlink1.txt new file mode 100644 index 0000000000000..dc1a7c5682a26 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/symlink1.txt @@ -0,0 +1,2 @@ +../data/files/T1.txt +../data/files/T3.txt diff --git a/sql/hive/src/test/resources/data/files/symlink2.txt b/sql/hive/src/test/resources/data/files/symlink2.txt new file mode 100644 index 0000000000000..8436a30adf366 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/symlink2.txt @@ -0,0 +1 @@ +../data/files/T2.txt diff --git a/sql/hive/src/test/resources/data/files/tbl.txt b/sql/hive/src/test/resources/data/files/tbl.txt new file mode 100644 index 0000000000000..10287c34423ca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/test.dat b/sql/hive/src/test/resources/data/files/test.dat new file mode 100644 index 0000000000000..cf0389a5afee1 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/test.dat @@ -0,0 +1,6 @@ +1 +2 +3 +4 +5 +6 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/test2.dat b/sql/hive/src/test/resources/data/files/test2.dat new file mode 100644 index 0000000000000..4b284c13ffafd --- /dev/null +++ b/sql/hive/src/test/resources/data/files/test2.dat @@ -0,0 +1,6 @@ +101 +202 +303 +404 +505 +606 \ No newline at end of file diff --git a/sql/hive/src/test/resources/data/files/text-en.txt b/sql/hive/src/test/resources/data/files/text-en.txt new file mode 100644 index 0000000000000..480c818a875fa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/things.txt b/sql/hive/src/test/resources/data/files/things.txt new file mode 100644 index 0000000000000..7eedbcfae2066 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/things.txt @@ -0,0 +1 @@ +2 Tie diff --git a/sql/hive/src/test/resources/data/files/things2.txt b/sql/hive/src/test/resources/data/files/things2.txt new file mode 100644 index 0000000000000..7eedbcfae2066 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/things2.txt @@ -0,0 +1 @@ +2 Tie diff --git a/sql/hive/src/test/resources/data/files/tiny_a.txt b/sql/hive/src/test/resources/data/files/tiny_a.txt new file mode 100644 index 0000000000000..c7867e6c62e00 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/tiny_a.txt @@ -0,0 +1 @@ +103200920268924913312 diff --git a/sql/hive/src/test/resources/data/files/tiny_b.txt b/sql/hive/src/test/resources/data/files/tiny_b.txt new file mode 100644 index 0000000000000..c8cdef26d7ba2 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/tiny_b.txt @@ -0,0 +1,2 @@ +10320092002467760023374853555556606672767778808187889090919092971001031041071081081091101131131131131131131141161161161171161171171171151151171171171211201311311311251251241241281281311311321331341341341342626\N\N\N\N116 +1032009202689249102591061311421592443203984174335536167108269179711046105110931112114212151220122612321267136415491646194821702272232524332534285229252992311932073279332334123637364536343450347336383688373637583812386238733868388341184134412741704216\N\N\N\N3139 diff --git a/sql/hive/src/test/resources/data/files/types/primitives/090101.txt b/sql/hive/src/test/resources/data/files/types/primitives/090101.txt new file mode 100644 index 0000000000000..35041d5be9a46 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/types/primitives/090201.txt b/sql/hive/src/test/resources/data/files/types/primitives/090201.txt new file mode 100644 index 0000000000000..f2a21f77b1028 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/types/primitives/090301.txt b/sql/hive/src/test/resources/data/files/types/primitives/090301.txt new file mode 100644 index 0000000000000..7f475f9295f24 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/types/primitives/090401.txt b/sql/hive/src/test/resources/data/files/types/primitives/090401.txt new file mode 100644 index 0000000000000..a3a0d69a22a46 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/union_input.txt b/sql/hive/src/test/resources/data/files/union_input.txt new file mode 100644 index 0000000000000..e3020be086b86 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/union_input.txt @@ -0,0 +1,8 @@ +011oneone +12.02twotwo +2threefour3threefour +35five5fivefive +2sixseven6sixseven +38eight8eighteight +099ninenine +110.010tenten diff --git a/sql/hive/src/test/resources/data/files/v1.txt b/sql/hive/src/test/resources/data/files/v1.txt new file mode 100644 index 0000000000000..1ce88332978db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/files/v2.txt b/sql/hive/src/test/resources/data/files/v2.txt new file mode 100644 index 0000000000000..5e4c94c2e6079 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/v2.txt @@ -0,0 +1,9 @@ +111199999 +222299999 +222299999 +444499999 +555599999 +666699999 +666699999 +666699999 +888899999 diff --git a/sql/hive/src/test/resources/data/files/vc1.txt b/sql/hive/src/test/resources/data/files/vc1.txt new file mode 100644 index 0000000000000..d0f99523dc2fd --- /dev/null +++ b/sql/hive/src/test/resources/data/files/vc1.txt @@ -0,0 +1,3 @@ +1abc +2abc +3 abc diff --git a/sql/hive/src/test/resources/data/files/x.txt b/sql/hive/src/test/resources/data/files/x.txt new file mode 100644 index 0000000000000..6407b72051b51 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/x.txt @@ -0,0 +1,2 @@ +Joe 2 +Hank 2 diff --git a/sql/hive/src/test/resources/data/files/y.txt b/sql/hive/src/test/resources/data/files/y.txt new file mode 100644 index 0000000000000..7eedbcfae2066 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/y.txt @@ -0,0 +1 @@ +2 Tie diff --git a/sql/hive/src/test/resources/data/files/z.txt b/sql/hive/src/test/resources/data/files/z.txt new file mode 100644 index 0000000000000..7eedbcfae2066 --- /dev/null +++ b/sql/hive/src/test/resources/data/files/z.txt @@ -0,0 +1 @@ +2 Tie diff --git a/sql/hive/src/test/resources/data/metadb/.gitignore b/sql/hive/src/test/resources/data/metadb/.gitignore new file mode 100644 index 0000000000000..0dd98905045cd --- /dev/null +++ b/sql/hive/src/test/resources/data/metadb/.gitignore @@ -0,0 +1 @@ +# Dummy file to make Git recognize this empty directory diff --git a/sql/hive/src/test/resources/data/scripts/cat.py b/sql/hive/src/test/resources/data/scripts/cat.py new file mode 100644 index 0000000000000..2395b2cdeb391 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/cat_error.py b/sql/hive/src/test/resources/data/scripts/cat_error.py new file mode 100644 index 0000000000000..9642efec8ecb4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/doubleescapedtab.py b/sql/hive/src/test/resources/data/scripts/doubleescapedtab.py new file mode 100644 index 0000000000000..d373067baed2c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/dumpdata_script.py b/sql/hive/src/test/resources/data/scripts/dumpdata_script.py new file mode 100644 index 0000000000000..c96c9e529bbb1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/error_script b/sql/hive/src/test/resources/data/scripts/error_script new file mode 100755 index 0000000000000..8d86b62f0f744 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/escapedcarriagereturn.py b/sql/hive/src/test/resources/data/scripts/escapedcarriagereturn.py new file mode 100644 index 0000000000000..475928a2430f6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/escapednewline.py b/sql/hive/src/test/resources/data/scripts/escapednewline.py new file mode 100644 index 0000000000000..0d5751454bed7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/escapedtab.py b/sql/hive/src/test/resources/data/scripts/escapedtab.py new file mode 100644 index 0000000000000..549c91e444632 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/input20_script b/sql/hive/src/test/resources/data/scripts/input20_script new file mode 100755 index 0000000000000..e8e41189c17cf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/newline.py b/sql/hive/src/test/resources/data/scripts/newline.py new file mode 100644 index 0000000000000..6500d900dd8ab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/q_test_cleanup.sql b/sql/hive/src/test/resources/data/scripts/q_test_cleanup.sql new file mode 100644 index 0000000000000..31bd7205d8591 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/q_test_init.sql b/sql/hive/src/test/resources/data/scripts/q_test_init.sql new file mode 100644 index 0000000000000..12afdf391132e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/data/scripts/test_init_file.sql b/sql/hive/src/test/resources/data/scripts/test_init_file.sql new file mode 100644 index 0000000000000..776a46be084c8 --- /dev/null +++ b/sql/hive/src/test/resources/data/scripts/test_init_file.sql @@ -0,0 +1 @@ +create table tbl_created_by_init(i int); diff --git a/sql/hive/src/test/resources/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb b/sql/hive/src/test/resources/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/'1' + '1'-0-77504a9f3d712143beb52f3c25a904cb @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/'1' + 1-0-130514c6116c311d808590a075b187b b/sql/hive/src/test/resources/golden/'1' + 1-0-130514c6116c311d808590a075b187b new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/'1' + 1-0-130514c6116c311d808590a075b187b @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e b/sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/'1' + 1.0-0-5db3b55120a19863d96460d399c2d0e @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 b/sql/hive/src/test/resources/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/'1' + 1L-0-657763a2cfaa4fe3f73c5b68bc84a548 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 b/sql/hive/src/test/resources/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/'1' + 1S-0-c3a1c44bebcde38c4d43bd73e3849630 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a b/sql/hive/src/test/resources/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/'1' + 1Y-0-aa608227a4f07c5cb98fcafe1e20488a @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a b/sql/hive/src/test/resources/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1 + '1'-0-4d39227e4121e2dc9a25f21fa27f89a @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 b/sql/hive/src/test/resources/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1 + 1-0-83de1c24fd6dee00409e8fdd99306ed6 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 b/sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1 + 1.0-0-4f5da98a11db8e7192423c27db767ca6 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e b/sql/hive/src/test/resources/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1 + 1L-0-917a033ac7f8f8b3a2e8e961dc91f35e @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e b/sql/hive/src/test/resources/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1 + 1S-0-2e99da48f67f588c9e632a57c713522e @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f b/sql/hive/src/test/resources/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1 + 1Y-0-1ff4db1fdac05de5b092095c2402fc5f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 b/sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1.0 + '1'-0-a6ec78b3b93d52034aab829d43210e73 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 b/sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1.0 + 1-0-30a4b1c8227906931cd0532367bebc43 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d b/sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1.0 + 1.0-0-87321b2e30ee2986b00b631d0e4f4d8d @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 b/sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1.0 + 1L-0-44bb88a1c9280952e8119a3ab1bb4205 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 b/sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1.0 + 1S-0-31fbe14d01fb532176c1689680398368 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 b/sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1.0 + 1Y-0-12bcf6e49e83abd2aa36ea612b418d43 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a b/sql/hive/src/test/resources/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1L + '1'-0-6e39c7be301f3846efa9b4c939815b4a @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d b/sql/hive/src/test/resources/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1L + 1-0-1864a260554255a09e4f28b8551eef9d @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 b/sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1L + 1.0-0-95a30c4b746f520f1251981a66cef5c8 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f b/sql/hive/src/test/resources/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1L + 1L-0-e54a673c779f31597acdc5bd7d315d9f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 b/sql/hive/src/test/resources/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1L + 1S-0-b8e70f71c32aac77e2683ba20ab99688 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 b/sql/hive/src/test/resources/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1L + 1Y-0-55de31e21660fa7d213b1f68d636cbf9 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 b/sql/hive/src/test/resources/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1S + '1'-0-c3cf30b2c4bffc76100e7b43e7b2aec5 @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 b/sql/hive/src/test/resources/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1S + 1-0-c378b0b2a57c54b3815e8a64686756d3 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e b/sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1S + 1.0-0-8dfa46ec33c1be5ffba2e40cbfe5349e @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 b/sql/hive/src/test/resources/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1S + 1L-0-9d3ff8b5d4883a4a5a1dc0dd4f0c1116 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e b/sql/hive/src/test/resources/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1S + 1S-0-5054df6e72be611d6ee2a4e135bd949e @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b b/sql/hive/src/test/resources/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1S + 1Y-0-e59bc8279cd364224476ffc504c7685b @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb b/sql/hive/src/test/resources/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1Y + '1'-0-bdc0f1c52b94a852b595e54997eb9dfb @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1Y + 1-0-a4541db51882b19503649138fbb295f b/sql/hive/src/test/resources/golden/1Y + 1-0-a4541db51882b19503649138fbb295f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1Y + 1-0-a4541db51882b19503649138fbb295f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d b/sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d new file mode 100644 index 0000000000000..415b19fc36234 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1Y + 1.0-0-3ad5e3db0d0300312d33231e7c2a6c8d @@ -0,0 +1 @@ +2.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 b/sql/hive/src/test/resources/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1Y + 1L-0-2aa9a7b23c741d78032def0641a21cb1 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 b/sql/hive/src/test/resources/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1Y + 1S-0-93a44c4952c4d343d3885edfc95b4b80 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 b/sql/hive/src/test/resources/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/1Y + 1Y-0-3d9619d963e7f1cb4ab3e8b5e24ff0d5 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be b/sql/hive/src/test/resources/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be new file mode 100644 index 0000000000000..913a609968137 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Column pruning: filter alias in-lining - query test-0-cb0737d268260da971487a8ac5e3d8be @@ -0,0 +1,3 @@ +86 +27 +98 diff --git a/sql/hive/src/test/resources/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 b/sql/hive/src/test/resources/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 new file mode 100644 index 0000000000000..9a276bc794c0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/Column pruning: non-trivial top project with aliases - query test-0-c89dcddb8539bae211ea28cd1e695e35 @@ -0,0 +1,3 @@ +476 +172 +622 diff --git a/sql/hive/src/test/resources/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa b/sql/hive/src/test/resources/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/sql/hive/src/test/resources/golden/Column pruning: projects alias substituting - query test-0-9589ac54d6de575083765ee64051ebaa @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/sql/hive/src/test/resources/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 b/sql/hive/src/test/resources/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 new file mode 100644 index 0000000000000..913a609968137 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Column pruning: simple top project without aliases - query test-0-d6d2726928df920ab99078015b8ff494 @@ -0,0 +1,3 @@ +86 +27 +98 diff --git a/sql/hive/src/test/resources/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 b/sql/hive/src/test/resources/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/sql/hive/src/test/resources/golden/Column pruning: with multiple projects - query test-0-cfe78c68302bd143d989c383c3b0477 @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/sql/hive/src/test/resources/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee b/sql/hive/src/test/resources/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/sql/hive/src/test/resources/golden/Column pruning: with non-partitioned table - query test-0-358525951e893ed9686c46bf1f7447ee @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/sql/hive/src/test/resources/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 b/sql/hive/src/test/resources/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/sql/hive/src/test/resources/golden/Column pruning: with partitioned table - query test-0-d9d50146537344742300ce7b52c18c91 @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/sql/hive/src/test/resources/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 b/sql/hive/src/test/resources/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 new file mode 100644 index 0000000000000..1c8e28554a60e --- /dev/null +++ b/sql/hive/src/test/resources/golden/Column pruning: without filters - query test-0-c25cdafe7d85c42a9d9cd559d88a6049 @@ -0,0 +1,3 @@ +238 +86 +311 diff --git a/sql/hive/src/test/resources/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a b/sql/hive/src/test/resources/golden/DISTINCT-0-3af674dcb5dd91ad17722d2022a8d59a new file mode 100644 index 0000000000000..5710fb29e7e02 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 b/sql/hive/src/test/resources/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 new file mode 100644 index 0000000000000..bbe37f8e2a790 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Escape sequences-0-2f25c33d97c43f3276171624d988a286 @@ -0,0 +1 @@ +86 \ \ \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 b/sql/hive/src/test/resources/golden/IgnoreExplain-0-85d398864d4aa1d0e10ffd668fdf1a59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 b/sql/hive/src/test/resources/golden/LIKE-0-8a6078c9da5f15ea95ba3682fd66e672 new file mode 100644 index 0000000000000..8007988316af0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 b/sql/hive/src/test/resources/golden/Partition pruning: all partitions pruned - query test-0-63ab78355dbc6fa0d7619c8a35b0f7a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 b/sql/hive/src/test/resources/golden/Partition pruning: left only 1 partition - query test-0-3adc3a7f76b2abd059904ba81a595db3 new file mode 100644 index 0000000000000..0fe6b905e7781 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb b/sql/hive/src/test/resources/golden/Partition pruning: non-partitioned, non-trivial project - query test-0-3893ec7a86062215d9f817f1495a69bb new file mode 100644 index 0000000000000..444039e75fbac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Partition pruning: pruning with both column key and partition key - query test-0-75148567b91227053f728d72f7dc7c10 b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 b/sql/hive/src/test/resources/golden/Partition pruning: with filter on int partition key - query test-0-4e4d2651e1e6e3c224242078d0201190 new file mode 100644 index 0000000000000..d83d8b62b7095 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 b/sql/hive/src/test/resources/golden/Partition pruning: with filter on string partition key - query test-0-b6e59464383591f02408f8765ac6a5d5 new file mode 100644 index 0000000000000..83bc6257ba98b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 b/sql/hive/src/test/resources/golden/Partiton pruning: non-partitioned table - query test-0-681332efaff7b12e3ca12d9f021cd344 new file mode 100644 index 0000000000000..7ba6caf9d8d59 --- /dev/null +++ b/sql/hive/src/test/resources/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 diff --git a/sql/hive/src/test/resources/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 b/sql/hive/src/test/resources/golden/Read and write with LazySimpleSerDe (tab separated)-0-779101eb00fd8bb9f08908ab29e90c03 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 b/sql/hive/src/test/resources/golden/Read with AvroSerDe-0-805f15ffbb03db90ec5757b328666d04 new file mode 100644 index 0000000000000..ee2a405fb6ab8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa b/sql/hive/src/test/resources/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa new file mode 100644 index 0000000000000..6407b72051b51 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Read with RegexSerDe-0-9b96fab8d55a0e19fae00d8adb57ffaa @@ -0,0 +1,2 @@ +Joe 2 +Hank 2 diff --git a/sql/hive/src/test/resources/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 b/sql/hive/src/test/resources/golden/Simple Average + 1 with group-0-f52ca483a3e5eadc1b20ba8320d029a7 new file mode 100644 index 0000000000000..595d7af04a69a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 b/sql/hive/src/test/resources/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 new file mode 100644 index 0000000000000..3d2dbbd731543 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Simple Average + 1-0-5e296b921c470f0f0b5d099f28bd5935 @@ -0,0 +1 @@ +261.182 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 b/sql/hive/src/test/resources/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 new file mode 100644 index 0000000000000..cbc70c89f8859 --- /dev/null +++ b/sql/hive/src/test/resources/golden/Simple Average-0-c197ea78c4d8f85f1a317805b6da07e5 @@ -0,0 +1 @@ +260.182 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 b/sql/hive/src/test/resources/golden/add_part_exist-0-e58d09864bc6898e44fae80abe328702 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-1-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 b/sql/hive/src/test/resources/golden/add_part_exist-10-ecb27eb754e731429659224b5b6ac583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/sql/hive/src/test/resources/golden/add_part_exist-11-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 b/sql/hive/src/test/resources/golden/add_part_exist-12-2900a5065dd3adbb0b56a712bf848750 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 b/sql/hive/src/test/resources/golden/add_part_exist-13-d69279050a0f44c3f8b775698fd29be0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/add_part_exist-14-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 b/sql/hive/src/test/resources/golden/add_part_exist-15-e58d09864bc6898e44fae80abe328702 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-16-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b b/sql/hive/src/test/resources/golden/add_part_exist-17-3432935e802ae46c6b2151cc4ebf783b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..f1ea8b0260d2c --- /dev/null +++ b/sql/hive/src/test/resources/golden/add_part_exist-18-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1 @@ +ds=2010-01-01 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 b/sql/hive/src/test/resources/golden/add_part_exist-19-83cecdbfddb070a417050f9a18fff752 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b b/sql/hive/src/test/resources/golden/add_part_exist-2-3432935e802ae46c6b2151cc4ebf783b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..f1ea8b0260d2c --- /dev/null +++ b/sql/hive/src/test/resources/golden/add_part_exist-20-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1 @@ +ds=2010-01-01 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f b/sql/hive/src/test/resources/golden/add_part_exist-21-2df4b20f67e7f15d3f4150bcfea43b0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-22-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..121aebdfab9af --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb b/sql/hive/src/test/resources/golden/add_part_exist-23-a3859b63665ce3bbfd19683e6e694dcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-24-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..64467c12563c8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..f1ea8b0260d2c --- /dev/null +++ b/sql/hive/src/test/resources/golden/add_part_exist-3-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1 @@ +ds=2010-01-01 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 b/sql/hive/src/test/resources/golden/add_part_exist-4-83cecdbfddb070a417050f9a18fff752 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..f1ea8b0260d2c --- /dev/null +++ b/sql/hive/src/test/resources/golden/add_part_exist-5-d824f22606f48dfca48ce241a7505f5b @@ -0,0 +1 @@ +ds=2010-01-01 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f b/sql/hive/src/test/resources/golden/add_part_exist-6-2df4b20f67e7f15d3f4150bcfea43b0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-7-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..121aebdfab9af --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb b/sql/hive/src/test/resources/golden/add_part_exist-8-a3859b63665ce3bbfd19683e6e694dcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b b/sql/hive/src/test/resources/golden/add_part_exist-9-d824f22606f48dfca48ce241a7505f5b new file mode 100644 index 0000000000000..64467c12563c8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 b/sql/hive/src/test/resources/golden/add_part_multiple-0-9c55143a4c92f0cfe7669f7681d7aa98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 b/sql/hive/src/test/resources/golden/add_part_multiple-1-4d9d4efbabc9fffef8841cc049f479c1 new file mode 100644 index 0000000000000..da21345385b2a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 b/sql/hive/src/test/resources/golden/add_part_multiple-2-187930e0daa44eed17e092e961ab7955 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef b/sql/hive/src/test/resources/golden/add_part_multiple-3-59fb141ee5c3e8f9463fe0478dbfd7ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c b/sql/hive/src/test/resources/golden/add_part_multiple-4-7950c676506564b085b41426ed41747c new file mode 100644 index 0000000000000..fc4021c39737f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 b/sql/hive/src/test/resources/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/add_partition_no_whitelist-0-3806584ff765bca682594008b90fc304 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af b/sql/hive/src/test/resources/golden/add_partition_no_whitelist-1-22eb96fe7d338e488182b5755c90d5af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 b/sql/hive/src/test/resources/golden/add_partition_no_whitelist-2-923fa18234ae73103c43722f70e000c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f b/sql/hive/src/test/resources/golden/add_partition_no_whitelist-3-b7c0bb09609fabad407feb6fdf2c748f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/sql/hive/src/test/resources/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/add_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 b/sql/hive/src/test/resources/golden/add_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 b/sql/hive/src/test/resources/golden/add_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 b/sql/hive/src/test/resources/golden/add_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d b/sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d new file mode 100644 index 0000000000000..5f4de85940513 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d @@ -0,0 +1 @@ +0 val_0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 b/sql/hive/src/test/resources/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alias.attr-0-42104e7e35985f4504c6d9a79b1bb4b8 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 b/sql/hive/src/test/resources/golden/alias_casted_column-0-f69b60c6e896fcd1a69d9525fd988c66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 b/sql/hive/src/test/resources/golden/alias_casted_column-1-4432aec015f9423ed991b08cfb2af0e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 b/sql/hive/src/test/resources/golden/alter2-0-85e494848d1525843a3ff9b2b77f92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-1-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..62a8ea1d115fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da b/sql/hive/src/test/resources/golden/alter2-10-2d1f665a92fe72bd253ae57c46d7b9da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-11-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..72621add45bb3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-12-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-12-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-13-ca51e03a3de391983429b6ad877e573c b/sql/hive/src/test/resources/golden/alter2-13-ca51e03a3de391983429b6ad877e573c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-14-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..72621add45bb3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..15e6bc8823150 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter2-15-75a213649242c2410ea6846f08c91d75 @@ -0,0 +1 @@ +insertdate=2008-01-01 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter2-16-77500c83ffcece95511a4b21d67382dd b/sql/hive/src/test/resources/golden/alter2-16-77500c83ffcece95511a4b21d67382dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-17-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..72621add45bb3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-18-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..d31318a383fc1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 b/sql/hive/src/test/resources/golden/alter2-19-3d80bb2d1c541460b5b17c4124fa647 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-2-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-2-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter2-20-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 b/sql/hive/src/test/resources/golden/alter2-21-a6ea9efed3b9c680fca93588019ac5e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 b/sql/hive/src/test/resources/golden/alter2-22-ea07b8f664208f93a1a8b97fd486d226 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-23-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter2-23-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-24-775d69742a1c07df8da87e8a017d955 b/sql/hive/src/test/resources/golden/alter2-24-775d69742a1c07df8da87e8a017d955 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-25-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..83184977e9da8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-26-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-26-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f b/sql/hive/src/test/resources/golden/alter2-27-ba521286c12ba29329bfa71bb185c62f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-28-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..83184977e9da8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..15e6bc8823150 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter2-29-63f13c364546ddce5d2176c6604a948f @@ -0,0 +1 @@ +insertdate=2008-01-01 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter2-3-ca51e03a3de391983429b6ad877e573c b/sql/hive/src/test/resources/golden/alter2-3-ca51e03a3de391983429b6ad877e573c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae b/sql/hive/src/test/resources/golden/alter2-30-a336adf1d0ff00633c53600fc75ca3ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-31-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..83184977e9da8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-32-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..d31318a383fc1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-33-1934026d0228967097280eed35551f74 b/sql/hive/src/test/resources/golden/alter2-33-1934026d0228967097280eed35551f74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b b/sql/hive/src/test/resources/golden/alter2-34-23b00f9c0101348e87da8a339b9da8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-35-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..6e30936a881bd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-36-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-36-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f b/sql/hive/src/test/resources/golden/alter2-37-ba521286c12ba29329bfa71bb185c62f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-38-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..6e30936a881bd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..15e6bc8823150 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter2-39-63f13c364546ddce5d2176c6604a948f @@ -0,0 +1 @@ +insertdate=2008-01-01 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-4-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..62a8ea1d115fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae b/sql/hive/src/test/resources/golden/alter2-40-a336adf1d0ff00633c53600fc75ca3ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5 b/sql/hive/src/test/resources/golden/alter2-41-4ef75e12575453225738ea167c4617e5 new file mode 100644 index 0000000000000..6e30936a881bd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f b/sql/hive/src/test/resources/golden/alter2-42-63f13c364546ddce5d2176c6604a948f new file mode 100644 index 0000000000000..d31318a383fc1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-43-1934026d0228967097280eed35551f74 b/sql/hive/src/test/resources/golden/alter2-43-1934026d0228967097280eed35551f74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/alter2-44-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c b/sql/hive/src/test/resources/golden/alter2-45-adbc01277c01cc5647e89c8a2430b8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..15e6bc8823150 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter2-5-75a213649242c2410ea6846f08c91d75 @@ -0,0 +1 @@ +insertdate=2008-01-01 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter2-6-77500c83ffcece95511a4b21d67382dd b/sql/hive/src/test/resources/golden/alter2-6-77500c83ffcece95511a4b21d67382dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 b/sql/hive/src/test/resources/golden/alter2-7-aac9c2c7033fd7264c9a107a88ff591 new file mode 100644 index 0000000000000..62a8ea1d115fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75 b/sql/hive/src/test/resources/golden/alter2-8-75a213649242c2410ea6846f08c91d75 new file mode 100644 index 0000000000000..d31318a383fc1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e b/sql/hive/src/test/resources/golden/alter2-9-1986a53bb1944fe5f43e3e65693e7b1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade b/sql/hive/src/test/resources/golden/alter3-0-1c1a75eeb97d4d8b9120d762aa0d2ade new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-1-75be487df30e301e156a22eee075633d b/sql/hive/src/test/resources/golden/alter3-1-75be487df30e301e156a22eee075633d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 b/sql/hive/src/test/resources/golden/alter3-10-bd9604a8b7eaecd785bc1e5163ec53a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 b/sql/hive/src/test/resources/golden/alter3-11-10e58aa21d9af1817c71d83ec8e3a4d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 b/sql/hive/src/test/resources/golden/alter3-12-2fcb7fc251f682a584ad513fddfac506 new file mode 100644 index 0000000000000..ba1746da5ce69 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b b/sql/hive/src/test/resources/golden/alter3-13-4d7fd1a73dc8dde03c5627fb5e1dc17b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f b/sql/hive/src/test/resources/golden/alter3-14-3fa4d8a690a45cbf7b44cecfd352864f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 b/sql/hive/src/test/resources/golden/alter3-15-5c6b489b14a4d8bc4ce9a26d8465d6f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter3-16-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b b/sql/hive/src/test/resources/golden/alter3-17-a9908f67f97588cbf15c0e7caddcbb0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d b/sql/hive/src/test/resources/golden/alter3-18-581b65f9f467d0d4a33a16dda144a31d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-19-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter3-19-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f b/sql/hive/src/test/resources/golden/alter3-2-5a67d369d700eb96f806f8320c04d61f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 b/sql/hive/src/test/resources/golden/alter3-20-8114bed96bb7bff5b4fa18069c8d6d00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 b/sql/hive/src/test/resources/golden/alter3-21-231db1adbff5fc90e57cca6a087f3df5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 b/sql/hive/src/test/resources/golden/alter3-22-362c1a2c9cb223f05b33c3cc193a4d24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 b/sql/hive/src/test/resources/golden/alter3-23-7ad62b397f6c9341da6bf0e9361314e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 b/sql/hive/src/test/resources/golden/alter3-24-9c23b682abda3841f01b4d9b750c68d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face b/sql/hive/src/test/resources/golden/alter3-25-568a59760e5d3241b63d65cce595face new file mode 100644 index 0000000000000..8e609740f34a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e b/sql/hive/src/test/resources/golden/alter3-26-3c725018f74a69f4d859c66af2f5b11e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 b/sql/hive/src/test/resources/golden/alter3-27-54ad133b447f67c6d1ed7d4c43803a87 new file mode 100644 index 0000000000000..0e12e0e166336 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 b/sql/hive/src/test/resources/golden/alter3-28-5332228ea451105c897d0c8c3c8f2773 new file mode 100644 index 0000000000000..4acb920388ceb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c b/sql/hive/src/test/resources/golden/alter3-29-b8fba19b9009131caffbb5fe7468b67c new file mode 100644 index 0000000000000..8e609740f34a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 b/sql/hive/src/test/resources/golden/alter3-3-f031aa27bf7b494cb8de20a305be7064 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 b/sql/hive/src/test/resources/golden/alter3-30-cd5c7d666fdea990be3cf66e43c7b8f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 b/sql/hive/src/test/resources/golden/alter3-31-5a41cf8c1a828ac2c372536ee4afd962 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 b/sql/hive/src/test/resources/golden/alter3-32-327744965ee8ed630f56fa3e4a3c5c65 new file mode 100644 index 0000000000000..12b038204bef4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter3-4-399fc26b344c98ababa104522601c0cc b/sql/hive/src/test/resources/golden/alter3-4-399fc26b344c98ababa104522601c0cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 b/sql/hive/src/test/resources/golden/alter3-5-bf2a8fd1884bb584059c848332e30c97 new file mode 100644 index 0000000000000..8e609740f34a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 b/sql/hive/src/test/resources/golden/alter3-6-fe6db2a539df10e4bc4715e2ed755135 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 b/sql/hive/src/test/resources/golden/alter3-7-30be5698ca15c1fd836686e7ad48ad8 new file mode 100644 index 0000000000000..0c47fb0ac64aa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 b/sql/hive/src/test/resources/golden/alter3-8-8f0a466bd1d021e40690865b7ae52a43 new file mode 100644 index 0000000000000..1165de3a92fd0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 b/sql/hive/src/test/resources/golden/alter3-9-b8a2a12aeddb84f56c7a1f6346bae3d2 new file mode 100644 index 0000000000000..8e609740f34a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 b/sql/hive/src/test/resources/golden/alter4-0-c261e1fa9f838dd034d37af38305e9c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-1-7ead71f9870ae36dd1cb50b51b41fad7 new file mode 100644 index 0000000000000..8d2aebeb4d29d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-10-7ead71f9870ae36dd1cb50b51b41fad7 new file mode 100644 index 0000000000000..cf67a009ff291 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d b/sql/hive/src/test/resources/golden/alter4-11-7db7af854e4e10fb6f0338b85d65549d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-12-7ead71f9870ae36dd1cb50b51b41fad7 new file mode 100644 index 0000000000000..a6375189a7a23 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f b/sql/hive/src/test/resources/golden/alter4-13-e9879d6bebc109340bbeecc3ca77492f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/alter4-14-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 b/sql/hive/src/test/resources/golden/alter4-15-63a545ee0e751a2729c8758a14712da5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 new file mode 100644 index 0000000000000..331d858ce9b12 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter4-16-549981e00a3d95f03dd5a9ef6044aa20 @@ -0,0 +1 @@ +default \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d b/sql/hive/src/test/resources/golden/alter4-2-7db7af854e4e10fb6f0338b85d65549d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 b/sql/hive/src/test/resources/golden/alter4-3-7ead71f9870ae36dd1cb50b51b41fad7 new file mode 100644 index 0000000000000..6b79a37a85f0c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 b/sql/hive/src/test/resources/golden/alter4-4-cc9c0034efdeb0bf94ad774aeb703c39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter4-5-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter4-6-acd58e84952d310aeddf78579c36286 b/sql/hive/src/test/resources/golden/alter4-6-acd58e84952d310aeddf78579c36286 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 b/sql/hive/src/test/resources/golden/alter4-7-8e6ec3396f25c124de5b212d8ce6c568 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-8-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter4-8-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 b/sql/hive/src/test/resources/golden/alter4-9-c261e1fa9f838dd034d37af38305e9c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-0-953553e14d835682fa47338dcfffe227 b/sql/hive/src/test/resources/golden/alter5-0-953553e14d835682fa47338dcfffe227 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b b/sql/hive/src/test/resources/golden/alter5-1-b8349afaf8e62dc6608a889c04ee3d4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-10-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter5-10-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..85c1918f46567 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter5-10-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart diff --git a/sql/hive/src/test/resources/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 b/sql/hive/src/test/resources/golden/alter5-11-e63fa029ab22ac4f5c880f2848f1b956 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 b/sql/hive/src/test/resources/golden/alter5-12-6b160869b8a9c846bc55a14f85bc5b52 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-13-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter5-13-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 b/sql/hive/src/test/resources/golden/alter5-14-92fbb4bc42ef462dcb4a06442b0c4023 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b b/sql/hive/src/test/resources/golden/alter5-15-b8349afaf8e62dc6608a889c04ee3d4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d b/sql/hive/src/test/resources/golden/alter5-16-4b76b7ff0df6adeded64b2a2f305530d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-17-9176dc5fb5206209fa907a289db1263d b/sql/hive/src/test/resources/golden/alter5-17-9176dc5fb5206209fa907a289db1263d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-18-2a9c8219c1468a1cf0534c665d1fcebf new file mode 100644 index 0000000000000..e180d4c53ae73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 b/sql/hive/src/test/resources/golden/alter5-19-2f6ab691e291c74ecc4305eeb30e3438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d b/sql/hive/src/test/resources/golden/alter5-2-4b76b7ff0df6adeded64b2a2f305530d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 b/sql/hive/src/test/resources/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 new file mode 100644 index 0000000000000..de522457dcada --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter5-20-d1779a2fe5ccc205e0499fae4c3942b1 @@ -0,0 +1,6 @@ +1 a +2 a +3 a +4 a +5 a +6 a diff --git a/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-21-2a9c8219c1468a1cf0534c665d1fcebf new file mode 100644 index 0000000000000..3d4ba2636ab66 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter5-3-2fc59e32c07186869811705c89aafadc b/sql/hive/src/test/resources/golden/alter5-3-2fc59e32c07186869811705c89aafadc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-4-2a9c8219c1468a1cf0534c665d1fcebf new file mode 100644 index 0000000000000..6669b628fc224 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 b/sql/hive/src/test/resources/golden/alter5-5-2f6ab691e291c74ecc4305eeb30e3438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 b/sql/hive/src/test/resources/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 new file mode 100644 index 0000000000000..de522457dcada --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter5-6-d1779a2fe5ccc205e0499fae4c3942b1 @@ -0,0 +1,6 @@ +1 a +2 a +3 a +4 a +5 a +6 a diff --git a/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf b/sql/hive/src/test/resources/golden/alter5-7-2a9c8219c1468a1cf0534c665d1fcebf new file mode 100644 index 0000000000000..e866ae0fa7654 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 b/sql/hive/src/test/resources/golden/alter5-8-89c414c65a129f2fc408b3124f292b29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter5-9-1e085f9741197e659413828c42386733 b/sql/hive/src/test/resources/golden/alter5-9-1e085f9741197e659413828c42386733 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/alter_index-0-21bcf37075b02097f16c8fc8130a83b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e b/sql/hive/src/test/resources/golden/alter_index-1-4c8f6b48c437bf0be109fc0be1dc840e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/alter_index-2-f36cb2eed39691ca949b25182e2dd31 new file mode 100644 index 0000000000000..2860d5c6b4cfb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb b/sql/hive/src/test/resources/golden/alter_index-3-33474b65c86b949d266541e0385bc6bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/alter_index-4-f36cb2eed39691ca949b25182e2dd31 new file mode 100644 index 0000000000000..2860d5c6b4cfb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/alter_index-5-21bcf37075b02097f16c8fc8130a83b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_index-6-489b4ceb2f4301a7132628303f99240d @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 b/sql/hive/src/test/resources/golden/alter_merge_2-0-48044f1a60d3e15e4e17f8f95098d995 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 b/sql/hive/src/test/resources/golden/alter_merge_2-1-3a102e7798dbcc7948223c18ddaa8cb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 b/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 new file mode 100644 index 0000000000000..c80ef36c96ad4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_merge_2-10-69fe9bb96263a49b9cca70cea7eb57e1 @@ -0,0 +1 @@ +754 -7678496319 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c b/sql/hive/src/test/resources/golden/alter_merge_2-11-efe4e50f2330d4f0a737183ea51836c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 b/sql/hive/src/test/resources/golden/alter_merge_2-2-bf243aa10b608872b9e8286f89c5ff30 new file mode 100644 index 0000000000000..618c1d01b726d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d b/sql/hive/src/test/resources/golden/alter_merge_2-3-cfef140167765d259320ed1e8aba718d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be b/sql/hive/src/test/resources/golden/alter_merge_2-4-47aa9e3236da627ab505a6bd01e563be new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f b/sql/hive/src/test/resources/golden/alter_merge_2-5-43bd090cda258e25037e7c32d500a85f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 b/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_merge_2-6-f2eeb518a957cece4250cc7558839e02 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 b/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 new file mode 100644 index 0000000000000..c80ef36c96ad4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_merge_2-7-69fe9bb96263a49b9cca70cea7eb57e1 @@ -0,0 +1 @@ +754 -7678496319 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc b/sql/hive/src/test/resources/golden/alter_merge_2-8-c1b9f23f413b1cceaeea94d3a86f09cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 b/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_merge_2-9-f2eeb518a957cece4250cc7558839e02 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a b/sql/hive/src/test/resources/golden/alter_partition_coltype-0-76649a6f1ff2f3ffa8310c34f0438e3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 b/sql/hive/src/test/resources/golden/alter_partition_coltype-1-b9bb847419afb2b70d0cb887894f0b90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 b/sql/hive/src/test/resources/golden/alter_partition_coltype-10-aa739a81271c760437de572a6c951eb9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 b/sql/hive/src/test/resources/golden/alter_partition_coltype-11-94b98dca970e36e6d4d65a795c9413d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 b/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 new file mode 100644 index 0000000000000..410b14d2ce6f9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_partition_coltype-12-84807e0be3e91250d4b8f2dc7938a256 @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e b/sql/hive/src/test/resources/golden/alter_partition_coltype-13-60ab4c242e19b882e4b1571544311e7e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 b/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 new file mode 100644 index 0000000000000..410b14d2ce6f9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_partition_coltype-14-fdad866a990591083719bd45e4113f58 @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 b/sql/hive/src/test/resources/golden/alter_partition_coltype-15-634775da8cebfb8ce45b3965a0ae2880 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e b/sql/hive/src/test/resources/golden/alter_partition_coltype-16-b0534980e325d1fee63c2636ad3f8a4e new file mode 100644 index 0000000000000..b28ab5ccf8a1b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f b/sql/hive/src/test/resources/golden/alter_partition_coltype-17-13c0443789a696bde8d08c05f526643f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c b/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c new file mode 100644 index 0000000000000..c5b431b6cba29 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_partition_coltype-18-fbdd117c3b1ec9c92c7c33d52d94e42c @@ -0,0 +1 @@ +50 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 b/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 new file mode 100644 index 0000000000000..a76c74dcec6ab --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_partition_coltype-19-63799ee74ccc42d9bb817e6d00a6fae3 @@ -0,0 +1 @@ +75 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a b/sql/hive/src/test/resources/golden/alter_partition_coltype-2-17e04afbb81a724fe8c47c07b642f9a new file mode 100644 index 0000000000000..316ca7f65ba20 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 b/sql/hive/src/test/resources/golden/alter_partition_coltype-20-b4411edb9c52a474a971640f037c8a30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a b/sql/hive/src/test/resources/golden/alter_partition_coltype-21-17e04afbb81a724fe8c47c07b642f9a new file mode 100644 index 0000000000000..a7382fabfcb49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 b/sql/hive/src/test/resources/golden/alter_partition_coltype-22-639cdccdea5f4b1863f9b17b04183c93 new file mode 100644 index 0000000000000..a7382fabfcb49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a b/sql/hive/src/test/resources/golden/alter_partition_coltype-23-325be9a5d7c0277498a139c0a9fdb26a new file mode 100644 index 0000000000000..a7382fabfcb49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 b/sql/hive/src/test/resources/golden/alter_partition_coltype-24-71de9e476503e83f82baf1ab17da87f6 new file mode 100644 index 0000000000000..a7382fabfcb49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 b/sql/hive/src/test/resources/golden/alter_partition_coltype-25-40b997fef00cf1a26f73ddb5013bbeb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 b/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 new file mode 100644 index 0000000000000..410b14d2ce6f9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_partition_coltype-3-179315b6f54dc18e6eeffd7aaa947fa5 @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d b/sql/hive/src/test/resources/golden/alter_partition_coltype-4-f6ba42faebdf8ec8781716fec6f7813d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d b/sql/hive/src/test/resources/golden/alter_partition_coltype-5-eeb71f1bc830750174b8b0d03de8c37d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd b/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd new file mode 100644 index 0000000000000..410b14d2ce6f9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_partition_coltype-6-db84ad9022cdc1828c24a0340321c8fd @@ -0,0 +1 @@ +25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 b/sql/hive/src/test/resources/golden/alter_partition_coltype-7-f3d9e8fc026c62c23e1ae0d191c89bc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 b/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_partition_coltype-8-42a70ae131fbb834c79074fdbd7beea0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 b/sql/hive/src/test/resources/golden/alter_partition_coltype-9-2f7e10db0fcc7939dea528e04f460d42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-0-72ba9397f487a914380dc15afaef1058 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-1-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..db182e444d31d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-10-71631c1e516c81ffdceac80f2d57ce09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-11-fe39b84ddc86b6bf042dc30c1b612321 new file mode 100644 index 0000000000000..81e23f0bc1951 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-12-1553ad79b098b737ea8def91134eb0e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-13-fe39b84ddc86b6bf042dc30c1b612321 new file mode 100644 index 0000000000000..6dc1f3ca2c187 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-14-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..4b754043d63ab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-15-bc83e8a2f8edf84f603109d14440dc83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-16-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..f44c28ee36760 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-17-7e411fcfdd8f169c503ed89dc56ee335 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-18-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..e739ad4992ec9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-19-56cadf0f555e355726dfed1929ad0508 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-2-bc83e8a2f8edf84f603109d14440dc83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-3-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..092e5ed6e8a46 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-4-7e411fcfdd8f169c503ed89dc56ee335 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-5-30348eedd3afb892ac9d825dd7fdb5d8 new file mode 100644 index 0000000000000..197e67d09bf49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-6-56cadf0f555e355726dfed1929ad0508 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-7-cee355b012efdc3bc7d584268a7025c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-8-e4c52934f1ff0024f7f0bbb78d4ae3f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 b/sql/hive/src/test/resources/golden/alter_partition_format_loc-9-fe39b84ddc86b6bf042dc30c1b612321 new file mode 100644 index 0000000000000..29b5b693b8589 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-0-2a230c069b09232acdd0d556007be97f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-1-fbbdf7be607407661749730f1a0efd9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-10-d71b99098bdb7f13db278dfa299b820d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-11-482182c9d90710fb16b6803d602a0d8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-12-b29d5391cda48aa2bd8f3bb37cc63750 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-13-19ceced1d8238509f2416029ddfbbc4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-14-90d009f94408102945d43860e4a6c68a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-15-7ab0e8f289c6846f9872edee0c40a628 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-16-577e1c164866c3955a9d8587ef7918a4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-2-79b9075b2f86f16f2356d9fa2a9afd56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-3-dd92c46e933d94b35c225daeef0285d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-4-9114d1f6859382a125fc4221d2d3ab6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-5-3cc094c5aa537b12f98895b95765329c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-6-beb03691c7cc6cf1597d3ff16ef98d17 new file mode 100644 index 0000000000000..02d72f4292749 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-7-5439426a18bb2d3918b91d589dbbd014 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-8-71e8c12c533654c30e044a8f062598ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d b/sql/hive/src/test/resources/golden/alter_partition_protect_mode-9-d1b12be1d01eabaf244f41e74d902d9d new file mode 100644 index 0000000000000..bca47334cedaa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-0-3c23ae800b2f6fb93620890da2dba196 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-1-67a0dbca9fecb7d34cceeecf2184c484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-2-e6a91be97431de63e372088d370c6d36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-3-b9a6b4e8acbfea5e1938eda085c4b893 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c b/sql/hive/src/test/resources/golden/alter_partition_with_whitelist-4-f42e9ca89ed2944213a5d994a587391c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 b/sql/hive/src/test/resources/golden/alter_rename_partition-0-1ed18256c5230de3439fe75d925ea73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada b/sql/hive/src/test/resources/golden/alter_rename_partition-1-2f79bceed6fc8ada34a670396ee6aada new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 b/sql/hive/src/test/resources/golden/alter_rename_partition-10-e3d9a36d53d30de215b855095c58d0d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 b/sql/hive/src/test/resources/golden/alter_rename_partition-11-bc84e38ccad173f5b47474d91db244d7 new file mode 100644 index 0000000000000..e881b2a4b74eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 b/sql/hive/src/test/resources/golden/alter_rename_partition-12-1ed18256c5230de3439fe75d925ea73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada b/sql/hive/src/test/resources/golden/alter_rename_partition-13-2f79bceed6fc8ada34a670396ee6aada new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-14-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 b/sql/hive/src/test/resources/golden/alter_rename_partition-15-f3b7bcb5d95a356fee54c0ce7d60c611 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 b/sql/hive/src/test/resources/golden/alter_rename_partition-16-611cf586cf3a1adc93c543d2da574c24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter_rename_partition-17-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b b/sql/hive/src/test/resources/golden/alter_rename_partition-18-bf6f780173f7b523b7ebd7925789372b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c b/sql/hive/src/test/resources/golden/alter_rename_partition-19-b2c9ded072d49abe14831bf48290319c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_rename_partition-2-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 b/sql/hive/src/test/resources/golden/alter_rename_partition-20-ee3ad861d109dd98db10bc86c5bf7105 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 b/sql/hive/src/test/resources/golden/alter_rename_partition-21-d92bfe92d250f66b3df45cb4ab50c0e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 b/sql/hive/src/test/resources/golden/alter_rename_partition-22-d50111b57d14f1ded1c47c773b0e0ac2 new file mode 100644 index 0000000000000..684f1da0fd0a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f b/sql/hive/src/test/resources/golden/alter_rename_partition-23-aedbaca33604c76b65137905fd42e98f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 b/sql/hive/src/test/resources/golden/alter_rename_partition-24-21dd05d56ebba285a8eb5bde5904d6a3 new file mode 100644 index 0000000000000..fc31ec62a1280 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 b/sql/hive/src/test/resources/golden/alter_rename_partition-25-9595f5b6ab31162c107076c35657c9f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b b/sql/hive/src/test/resources/golden/alter_rename_partition-26-270655c514bb6f04acd4459df52dd77b new file mode 100644 index 0000000000000..e881b2a4b74eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b b/sql/hive/src/test/resources/golden/alter_rename_partition-3-b465c6126edd94e8d45f61e2a19d005b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f b/sql/hive/src/test/resources/golden/alter_rename_partition-4-3479a886936571d5028971aecade705f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 b/sql/hive/src/test/resources/golden/alter_rename_partition-5-6cc4e3014e34a862602a47357f4fb9f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 b/sql/hive/src/test/resources/golden/alter_rename_partition-6-3324664e6500e2d256d0b8b3b8a14c24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 b/sql/hive/src/test/resources/golden/alter_rename_partition-7-e3d9a36d53d30de215b855095c58d0d7 new file mode 100644 index 0000000000000..684f1da0fd0a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 b/sql/hive/src/test/resources/golden/alter_rename_partition-8-d3ea111b1a37613bdda2c6eae13790c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 b/sql/hive/src/test/resources/golden/alter_rename_partition-9-21dd05d56ebba285a8eb5bde5904d6a3 new file mode 100644 index 0000000000000..fc31ec62a1280 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c b/sql/hive/src/test/resources/golden/alter_table_serde-0-35d2014351106b918c8e337a1919470c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-1-5bc931a540f0fec54e852ff10f52f879 new file mode 100644 index 0000000000000..ccd6518a50f7c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa b/sql/hive/src/test/resources/golden/alter_table_serde-10-ed0059ecd1cf948e8f75153593c8a5aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-11-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..d135e450e6e8e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc b/sql/hive/src/test/resources/golden/alter_table_serde-12-63a8168d2eae62132c3cd9b90b3cdbcc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-13-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..d135e450e6e8e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def b/sql/hive/src/test/resources/golden/alter_table_serde-14-ab6d7ed387e6e2f1f8f32272e3d31def new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-15-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..c3fac88f91a36 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 b/sql/hive/src/test/resources/golden/alter_table_serde-16-c6bb65a44a8b0ba1fa454218a31a2a44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-17-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..6c8f91de7cded --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 b/sql/hive/src/test/resources/golden/alter_table_serde-18-1649ba756fd9238f0608e4cb3affa3c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa b/sql/hive/src/test/resources/golden/alter_table_serde-2-ed0059ecd1cf948e8f75153593c8a5aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-3-5bc931a540f0fec54e852ff10f52f879 new file mode 100644 index 0000000000000..37a5b2cc47bad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc b/sql/hive/src/test/resources/golden/alter_table_serde-4-63a8168d2eae62132c3cd9b90b3cdbcc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 b/sql/hive/src/test/resources/golden/alter_table_serde-5-5bc931a540f0fec54e852ff10f52f879 new file mode 100644 index 0000000000000..0348dd15fd4f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 b/sql/hive/src/test/resources/golden/alter_table_serde-6-1649ba756fd9238f0608e4cb3affa3c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b b/sql/hive/src/test/resources/golden/alter_table_serde-7-9ca4794d2e2cb6ae6f8e4f33f7ff290b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 b/sql/hive/src/test/resources/golden/alter_table_serde-8-78d739d2409b59c0e01cde962451d295 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 b/sql/hive/src/test/resources/golden/alter_table_serde-9-6ee4b3a60659ec5496f06347eda232a8 new file mode 100644 index 0000000000000..d135e450e6e8e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 b/sql/hive/src/test/resources/golden/alter_varchar1-0-5fa6071842a0443346cf6db677a33412 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea b/sql/hive/src/test/resources/golden/alter_varchar1-1-be11cb1f18ab19550011417126264fea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 b/sql/hive/src/test/resources/golden/alter_varchar1-10-c1a57b45952193d04b5411c5b6a31139 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 b/sql/hive/src/test/resources/golden/alter_varchar1-11-fa89c704636fa7bd937cf1a975bb2ae6 new file mode 100644 index 0000000000000..dd347f3e8f58e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd b/sql/hive/src/test/resources/golden/alter_varchar1-12-a694df5b2a8f2101f6fd2b936eeb2bfd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 b/sql/hive/src/test/resources/golden/alter_varchar1-13-fa89c704636fa7bd937cf1a975bb2ae6 new file mode 100644 index 0000000000000..12087837cebf1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 b/sql/hive/src/test/resources/golden/alter_varchar1-14-5fa6071842a0443346cf6db677a33412 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 b/sql/hive/src/test/resources/golden/alter_varchar1-2-ba9453c6b6a627286691f3930c2b26d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 b/sql/hive/src/test/resources/golden/alter_varchar1-3-fa89c704636fa7bd937cf1a975bb2ae6 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d b/sql/hive/src/test/resources/golden/alter_varchar1-4-c9a8643e08d6ed320f82c26e1ffa8b5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 b/sql/hive/src/test/resources/golden/alter_varchar1-5-2756ef8fbe2cfa4609808a3855f50969 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 b/sql/hive/src/test/resources/golden/alter_varchar1-6-f7d529dc66c022b64e0b287c82f92778 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f b/sql/hive/src/test/resources/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f new file mode 100644 index 0000000000000..879a6e7bcbd18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_varchar1-7-818f2ce0a782a1d3cb02fd85bd1d3f9f @@ -0,0 +1,5 @@ +0 val +0 val +0 val +2 val +4 val diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 b/sql/hive/src/test/resources/golden/alter_varchar1-8-bdde28ebc875c39f9630d95379eee68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 b/sql/hive/src/test/resources/golden/alter_varchar1-9-5e48ee7bcd9439e68aa6dbc850ad8771 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 b/sql/hive/src/test/resources/golden/alter_varchar2-0-22c4186110b5770deaf7f03cf08326b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc b/sql/hive/src/test/resources/golden/alter_varchar2-1-ecc82a01a8f681a8a2d44a67a8a3f1cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 b/sql/hive/src/test/resources/golden/alter_varchar2-2-325238d61f56d84c17e29033105d7b19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 b/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 new file mode 100644 index 0000000000000..40818a7de46d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_varchar2-3-fb3191f771e2396d5fc80659a8c68797 @@ -0,0 +1 @@ +val_238 7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb b/sql/hive/src/test/resources/golden/alter_varchar2-4-9a4bf0db2b90d54ea0eeff2ec356fcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c new file mode 100644 index 0000000000000..827220bd4996f --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_varchar2-5-84e700f9dc6033c1f237fcdb95e31a0c @@ -0,0 +1 @@ +1 val_238 7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 b/sql/hive/src/test/resources/golden/alter_varchar2-6-eb0f1b170900bb995674b0bc1968c656 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c b/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c new file mode 100644 index 0000000000000..827220bd4996f --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_varchar2-7-84e700f9dc6033c1f237fcdb95e31a0c @@ -0,0 +1 @@ +1 val_238 7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c b/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c new file mode 100644 index 0000000000000..8a8234a35f6bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/alter_varchar2-8-4c12c4c53d99338796be34e603dc612c @@ -0,0 +1 @@ +2 238 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 b/sql/hive/src/test/resources/golden/alter_view_as_select-0-9f40bf1c2b92465189583446a6b40910 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad b/sql/hive/src/test/resources/golden/alter_view_as_select-1-5ba1b5ca1199ad7281ff9b5b71105aad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-2-1ac845048a8c714a36a719ea8e4f570b new file mode 100644 index 0000000000000..a99747531cef4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 b/sql/hive/src/test/resources/golden/alter_view_as_select-3-9280ae6c369a9f30d3d021d00e435f01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-4-1ac845048a8c714a36a719ea8e4f570b new file mode 100644 index 0000000000000..499c73127d890 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 b/sql/hive/src/test/resources/golden/alter_view_as_select-5-48b435d96e34065b03c6d7e4e891fbe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b b/sql/hive/src/test/resources/golden/alter_view_as_select-6-1ac845048a8c714a36a719ea8e4f570b new file mode 100644 index 0000000000000..a5fba77abdf07 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 b/sql/hive/src/test/resources/golden/alter_view_rename-0-bb255b994b5207324fba6988caa937e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d b/sql/hive/src/test/resources/golden/alter_view_rename-1-2a83c96363ca8d12cd2e9181209c8d8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb b/sql/hive/src/test/resources/golden/alter_view_rename-2-67e47ee2746463594d5c48b10ba1bb new file mode 100644 index 0000000000000..ee76e02af3aba --- /dev/null +++ b/sql/hive/src/test/resources/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:tnachen, createTime:1392426511, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:foo, type:int, comment:null), FieldSchema(name:bar, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426511}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a b/sql/hive/src/test/resources/golden/alter_view_rename-3-95655e33f22fc8f66549a9708812589a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 b/sql/hive/src/test/resources/golden/alter_view_rename-4-19c1c00f0aa99d81b7466958c15d88e3 new file mode 100644 index 0000000000000..8603577477bfc --- /dev/null +++ b/sql/hive/src/test/resources/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:tnachen, createTime:1392426511, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:foo, type:int, comment:null), FieldSchema(name:bar, type:string, comment:null), FieldSchema(name:ds, type:string, comment:null)], location:null, inputFormat:org.apache.hadoop.mapred.SequenceFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:null, parameters:{}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426511, transient_lastDdlTime=1392426511}, viewOriginalText:SELECT * FROM invites, viewExpandedText:SELECT `invites`.`foo`, `invites`.`bar`, `invites`.`ds` FROM `default`.`invites`, tableType:VIRTUAL_VIEW) diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d b/sql/hive/src/test/resources/golden/ambiguous_col-0-b4fe82679efdf6a15e9ecff53baf8d8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab b/sql/hive/src/test/resources/golden/ambiguous_col-1-dadfa3854356dead14b93c5a71a5d8ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b b/sql/hive/src/test/resources/golden/ambiguous_col-2-70509ccd2765d90b98666b6dff8afe1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-0-89cd75b0565e8d96910d5528db9984e7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/archive-10-f845008104fd12eb0f13f4b113f95cf1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-11-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/archive-12-f07653bca86e1ecb614ffd0296790d05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-13-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 new file mode 100644 index 0000000000000..21b3b13a81191 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-14-2cde1a2d3bfcec814985f498eba0fb8 @@ -0,0 +1 @@ +0 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive-15-c9f39b388ae698e385e092d0ffeb3c73 new file mode 100644 index 0000000000000..5e5f6ff96623f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/archive-16-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/archive-16-892147913578bcf60620b7dd73893dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-17-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a b/sql/hive/src/test/resources/golden/archive-18-8f980275ab3a5bcfc1784f4acd46447a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f b/sql/hive/src/test/resources/golden/archive-19-f8a52a8a40141409a667a9ba2cf9630f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-2-713efc113418b01f76ffd589840193c8 b/sql/hive/src/test/resources/golden/archive-2-713efc113418b01f76ffd589840193c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-20-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/archive-21-f07653bca86e1ecb614ffd0296790d05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-22-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-23-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/archive-23-892147913578bcf60620b7dd73893dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-24-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f b/sql/hive/src/test/resources/golden/archive-25-56bea24ffa83d9b5932893a8ff1cb44f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a b/sql/hive/src/test/resources/golden/archive-26-44fa61c2bd0fd9acfa0d889e59880d8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 b/sql/hive/src/test/resources/golden/archive-27-9ae773ebe64a3d437a035e9d94f49e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 b/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 new file mode 100644 index 0000000000000..18a1a7925ff29 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-28-188eb7912265ed8dffa5200517bbe526 @@ -0,0 +1 @@ +48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 b/sql/hive/src/test/resources/golden/archive-29-a0c6922e3a1dca51861c8a872dc8af19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/archive-3-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a new file mode 100644 index 0000000000000..18a1a7925ff29 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive-30-bea4ae5a0d219d544ea0b53bf29ecc7a @@ -0,0 +1 @@ +48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/archive-31-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-32-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/archive-32-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/archive-4-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/archive-5-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-6-528ab9750a558af7f1a43b3108e793dd b/sql/hive/src/test/resources/golden/archive-6-528ab9750a558af7f1a43b3108e793dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/archive-7-e8d1d10c308a73eef78dde414a5e40ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/archive-8-af459a0264559a2aeaa1341ce779ab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/archive-9-48b10f27e1459bb8e62d6c71484e2cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-0-89cd75b0565e8d96910d5528db9984e7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-10-f845008104fd12eb0f13f4b113f95cf1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-11-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-12-f07653bca86e1ecb614ffd0296790d05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-13-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 new file mode 100644 index 0000000000000..21b3b13a81191 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-14-2cde1a2d3bfcec814985f498eba0fb8 @@ -0,0 +1 @@ +0 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-15-c9f39b388ae698e385e092d0ffeb3c73 new file mode 100644 index 0000000000000..5e5f6ff96623f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-16-892147913578bcf60620b7dd73893dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-17-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-18-8f980275ab3a5bcfc1784f4acd46447a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-19-f8a52a8a40141409a667a9ba2cf9630f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-2-433a1b642df2cebe37927d6d89e0b301 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-20-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-21-f07653bca86e1ecb614ffd0296790d05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-22-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-23-892147913578bcf60620b7dd73893dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 new file mode 100644 index 0000000000000..69ca68f501ff1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-24-530277b0fee8b05c37b26846bceef827 @@ -0,0 +1,6 @@ +0 +0 +0 +10 +20 +30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-25-56bea24ffa83d9b5932893a8ff1cb44f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-26-44fa61c2bd0fd9acfa0d889e59880d8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-27-9ae773ebe64a3d437a035e9d94f49e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 new file mode 100644 index 0000000000000..18a1a7925ff29 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-28-188eb7912265ed8dffa5200517bbe526 @@ -0,0 +1 @@ +48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-29-a0c6922e3a1dca51861c8a872dc8af19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-3-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a new file mode 100644 index 0000000000000..18a1a7925ff29 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-30-bea4ae5a0d219d544ea0b53bf29ecc7a @@ -0,0 +1 @@ +48656137 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-31-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-32-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-4-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-5-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-6-528ab9750a558af7f1a43b3108e793dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-7-e8d1d10c308a73eef78dde414a5e40ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-8-af459a0264559a2aeaa1341ce779ab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/archive_excludeHadoop20-9-48b10f27e1459bb8e62d6c71484e2cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 b/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_multi-0-89cd75b0565e8d96910d5528db9984e7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_multi-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 b/sql/hive/src/test/resources/golden/archive_multi-10-f845008104fd12eb0f13f4b113f95cf1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e b/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_multi-11-cf5431cd843666b95ad2a82b334ac01e @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 b/sql/hive/src/test/resources/golden/archive_multi-12-8419ad4ed6683ebd15f993f703975b31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_multi-13-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 b/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 new file mode 100644 index 0000000000000..21b3b13a81191 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_multi-14-2cde1a2d3bfcec814985f498eba0fb8 @@ -0,0 +1 @@ +0 3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 b/sql/hive/src/test/resources/golden/archive_multi-15-c9f39b388ae698e385e092d0ffeb3c73 new file mode 100644 index 0000000000000..5e5f6ff96623f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d b/sql/hive/src/test/resources/golden/archive_multi-16-ad80f33c39be583ad7ebf0c8f350d11d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb b/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb new file mode 100644 index 0000000000000..5cd5fb9874d67 --- /dev/null +++ b/sql/hive/src/test/resources/golden/archive_multi-17-27895cbe0ee6d24d7fc866314491e1bb @@ -0,0 +1 @@ +48479881068 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/archive_multi-2-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/archive_multi-3-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/archive_multi-4-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/archive_multi-5-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/archive_multi-6-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/archive_multi-7-e8d1d10c308a73eef78dde414a5e40ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/archive_multi-8-af459a0264559a2aeaa1341ce779ab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf b/sql/hive/src/test/resources/golden/archive_multi-9-48b10f27e1459bb8e62d6c71484e2cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba b/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/attr-0-24e06ffd262f2a5a6eec3314445d83ba @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join0-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 b/sql/hive/src/test/resources/golden/auto_join0-1-383f34dec3ac939b7af2c9093a557641 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c b/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c new file mode 100644 index 0000000000000..308fc0924e670 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join0-2-7bd04899197b027d81c24e45a99ad15c @@ -0,0 +1 @@ +34298511120 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a b/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join1-0-443afb71720bad780b5dbfb6dbf4b51a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e b/sql/hive/src/test/resources/golden/auto_join1-1-f1293ebf768eb04f2f0bfe6297c4509e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 b/sql/hive/src/test/resources/golden/auto_join1-2-8a9624554e208e3d8fbe42908c715b92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 b/sql/hive/src/test/resources/golden/auto_join1-3-f6046c5229e3b0aa21498a3872f43b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e b/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join1-4-ae1247a065c41ce0329ca6078ab586e @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join10-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 b/sql/hive/src/test/resources/golden/auto_join10-1-dc932cb87d4d1a90dc5733c544b6a3d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 b/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join10-2-eef4ee52e0783b15fb5fe17378806b13 @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join11-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 b/sql/hive/src/test/resources/golden/auto_join11-1-82ea193ec76c2c6acd5f7178ef5ec417 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 b/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 new file mode 100644 index 0000000000000..69dbf8c3143e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join11-2-5496e81f60ba1d8a95d8375589c71e05 @@ -0,0 +1 @@ +-101339664144 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join12-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 b/sql/hive/src/test/resources/golden/auto_join12-1-c2efec9ea2ba761603b723afc0d5d145 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 b/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 new file mode 100644 index 0000000000000..eff107c7ce6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join12-2-4df549c5f0b6bff0c843008fa35b1320 @@ -0,0 +1 @@ +-136852761207 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join13-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 b/sql/hive/src/test/resources/golden/auto_join13-1-f5e043288a21ea691c74fef2e39a52b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 b/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 new file mode 100644 index 0000000000000..de6c015da2059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join13-2-5ff417533a1243cd6fc556960fa170c9 @@ -0,0 +1 @@ +-97676500536 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join14-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 b/sql/hive/src/test/resources/golden/auto_join14-1-c85f3dcbab496811604ea0ab84d0e995 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join14-2-9b141c1e5917ca82c6bc36a9a2950a1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join14-3-2b9ccaa793eae0e73bf76335d3d6880 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join14-4-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 b/sql/hive/src/test/resources/golden/auto_join14-5-2454f1ce2e8d0d03a30c479f7dcd8153 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 b/sql/hive/src/test/resources/golden/auto_join14-6-de39302191b63d7aa8f92885b089fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 b/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 new file mode 100644 index 0000000000000..f1871a4957ddb --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join14-7-5b5ded1412301eae5f8f705a39e6832 @@ -0,0 +1 @@ +404554174174 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-1-98b7542190092fafcc8b1ad5b0024a22 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-2-db1cd54a4cb36de2087605f32e41824f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-3-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-4-2454f1ce2e8d0d03a30c479f7dcd8153 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-5-de39302191b63d7aa8f92885b089fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 new file mode 100644 index 0000000000000..0f27a9bde401c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join14_hadoop20-6-5b5ded1412301eae5f8f705a39e6832 @@ -0,0 +1 @@ +404554174174 diff --git a/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join15-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 b/sql/hive/src/test/resources/golden/auto_join15-1-e23b9aa655061fb5a70d1f8f28f170f5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 b/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 new file mode 100644 index 0000000000000..006e1f82c0a47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join15-2-7bf2df40dd30fb2f8c4af9a0d09e24f9 @@ -0,0 +1 @@ +-793937029770 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join16-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf b/sql/hive/src/test/resources/golden/auto_join16-1-bba773956a3bad8d400fe4216a3fa8bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 b/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join16-2-66e56dcda38eb09819ac49e47e40d125 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join17-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 b/sql/hive/src/test/resources/golden/auto_join17-1-387dd86b1e13f788ec677a08dc162c97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 b/sql/hive/src/test/resources/golden/auto_join17-2-c2fd9f9c4cc80f21ea8c10edaaf03808 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 b/sql/hive/src/test/resources/golden/auto_join17-3-478a9f270a5d70f6f82f81e6962fb251 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c b/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c new file mode 100644 index 0000000000000..006e1f82c0a47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join17-4-11d706a64d44a8b0d41b290c4671c29c @@ -0,0 +1 @@ +-793937029770 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join18-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b b/sql/hive/src/test/resources/golden/auto_join18-1-3839d176ee45fb0fc6702f4a7794ca1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f b/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f new file mode 100644 index 0000000000000..0c9b518e65ece --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join18-2-f633ade9577c8b0e89d89124194c8d0f @@ -0,0 +1 @@ +2358131334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join19-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/auto_join19-1-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 b/sql/hive/src/test/resources/golden/auto_join19-2-70f3756d8b44d637ac4596cbbd48dc77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 b/sql/hive/src/test/resources/golden/auto_join19-3-a3751c195480244a5ed497fd053cd433 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f new file mode 100644 index 0000000000000..795166629df40 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join19-4-eaa70da463b92e85e1796277f016c18f @@ -0,0 +1 @@ +407444119660 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join2-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca b/sql/hive/src/test/resources/golden/auto_join2-1-3d1692c4710db1ff716d35e921f2bcca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 b/sql/hive/src/test/resources/golden/auto_join2-2-15d7a6cb2e2f21077de2447c656e7a34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a b/sql/hive/src/test/resources/golden/auto_join2-3-d4673c03d04084b838fcd8149f59ad9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 b/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 new file mode 100644 index 0000000000000..1c958900f5013 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join2-4-9d8144612cb3132ad9f7c8fa93586185 @@ -0,0 +1 @@ +33815990627 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join20-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd b/sql/hive/src/test/resources/golden/auto_join20-1-2afb0510178c4b66876dd91c7ca441fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 b/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 new file mode 100644 index 0000000000000..b1a6075f768c8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join20-2-903ee25e327188edaaf2040fec5a8e52 @@ -0,0 +1 @@ +-24276731469 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee b/sql/hive/src/test/resources/golden/auto_join20-3-cf1c52393ea3a7e21782a1c52b83f0ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 b/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 new file mode 100644 index 0000000000000..b1a6075f768c8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join20-4-e48e08d5b94719d56a58284eaef757f2 @@ -0,0 +1 @@ +-24276731469 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join21-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 b/sql/hive/src/test/resources/golden/auto_join21-1-9dd59784ca1555b607df0137d2666fb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f b/sql/hive/src/test/resources/golden/auto_join21-2-3536b7d78713e86ee67f5f6c9b88958f new file mode 100644 index 0000000000000..9672e21fa0323 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join22-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 b/sql/hive/src/test/resources/golden/auto_join22-1-4044be0e5116357bd88b4eda0f9ccaa8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb b/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb new file mode 100644 index 0000000000000..dba3bca53f72d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join22-2-a4abc288c20edee53ede45d248cf3abb @@ -0,0 +1 @@ +344337359100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join23-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join23-1-b31437533a2b890788938455cb32f679 b/sql/hive/src/test/resources/golden/auto_join23-1-b31437533a2b890788938455cb32f679 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 b/sql/hive/src/test/resources/golden/auto_join23-2-6d2c5b58222f31658a0cf957e093a150 new file mode 100644 index 0000000000000..5707ed08e7e54 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join24-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b b/sql/hive/src/test/resources/golden/auto_join24-1-721dfa03bfea05e55506c571b6c3585b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d b/sql/hive/src/test/resources/golden/auto_join24-2-36de83b0ed6c9fdc03661b2f65b23a3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 b/sql/hive/src/test/resources/golden/auto_join24-3-fa8b2736440ff35687dadb1bcae32666 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 b/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join24-4-d79325ef6494aa87843fdfd78de7c812 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join25-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/sql/hive/src/test/resources/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join25-1-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e b/sql/hive/src/test/resources/golden/auto_join25-10-f1293ebf768eb04f2f0bfe6297c4509e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 b/sql/hive/src/test/resources/golden/auto_join25-11-f6046c5229e3b0aa21498a3872f43b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e b/sql/hive/src/test/resources/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e new file mode 100644 index 0000000000000..d14fbdc94256c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join25-12-ae1247a065c41ce0329ca6078ab586e @@ -0,0 +1 @@ +101861029915 diff --git a/sql/hive/src/test/resources/golden/auto_join25-2-8180638a57b64557e02815c863031755 b/sql/hive/src/test/resources/golden/auto_join25-2-8180638a57b64557e02815c863031755 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join25-2-8180638a57b64557e02815c863031755 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e b/sql/hive/src/test/resources/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join25-3-9aa914a687f1f63faf48eb500627855e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 b/sql/hive/src/test/resources/golden/auto_join25-4-d83e6df8cd60d8ebeebd2100c51002d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 b/sql/hive/src/test/resources/golden/auto_join25-5-a3751c195480244a5ed497fd053cd433 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f new file mode 100644 index 0000000000000..069b64b649977 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join25-6-eaa70da463b92e85e1796277f016c18f @@ -0,0 +1 @@ +407444119660 diff --git a/sql/hive/src/test/resources/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca b/sql/hive/src/test/resources/golden/auto_join25-7-3d1692c4710db1ff716d35e921f2bcca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a b/sql/hive/src/test/resources/golden/auto_join25-8-d4673c03d04084b838fcd8149f59ad9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 b/sql/hive/src/test/resources/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 new file mode 100644 index 0000000000000..b8f473bf53aa3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join25-9-9d8144612cb3132ad9f7c8fa93586185 @@ -0,0 +1 @@ +33815990627 diff --git a/sql/hive/src/test/resources/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 b/sql/hive/src/test/resources/golden/auto_join26-0-54a7280ab9eed0d2e3b33df35a721b66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join26-1-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 b/sql/hive/src/test/resources/golden/auto_join26-2-5ebef1af539734d0335dbe6aacae3e13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 b/sql/hive/src/test/resources/golden/auto_join26-3-fed383a65bd118b43de6b00be10fecb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/auto_join26-4-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..71094ee7360db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join27-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 b/sql/hive/src/test/resources/golden/auto_join27-1-c83f56f364b1da3def90d48953665fe5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e b/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e new file mode 100644 index 0000000000000..dd35c6b71fc80 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join27-2-fceaa1ebd63334061d2d8daf961e935e @@ -0,0 +1 @@ +548 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c b/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join28-0-10a2c01dccc8980fe6aff1f9dd65042c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join28-1-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 b/sql/hive/src/test/resources/golden/auto_join28-2-9dd59784ca1555b607df0137d2666fb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 b/sql/hive/src/test/resources/golden/auto_join28-3-30739ff22c62b3becf56694642b7ae81 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 b/sql/hive/src/test/resources/golden/auto_join28-4-c178253e7ce91b5aa35c2cc424bfa27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e b/sql/hive/src/test/resources/golden/auto_join28-5-142850e84341feb3f7f40dd4553f72e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join3-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/auto_join3-1-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d b/sql/hive/src/test/resources/golden/auto_join3-2-6bed7d8089695e23914b29edaab2537d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 b/sql/hive/src/test/resources/golden/auto_join3-3-e9f6d17b15064f953a588fb40aee2f90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join3-4-eaa70da463b92e85e1796277f016c18f @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join30-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 b/sql/hive/src/test/resources/golden/auto_join30-1-8cef272a7680529de5c6bd227a83cbc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a b/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join30-10-820f36ed1bdf14c1deb383f508a5ed7a @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 b/sql/hive/src/test/resources/golden/auto_join30-11-53e43f2e48f605ec92c8a18c53e80620 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d b/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join30-12-de6e26f52f6f9ea5ef9a4868e57d36d @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba b/sql/hive/src/test/resources/golden/auto_join30-13-9ee597656aa92e48475d6542339915ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 b/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join30-14-7a248488c218919ab50e072fdbdecb73 @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 b/sql/hive/src/test/resources/golden/auto_join30-15-47b7efabbd6046e2befcbbea7da62553 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 b/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join30-16-f4f5bc179d84baf57e14cd2f8bd39436 @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 b/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join30-2-bc472f95600f47d5ea60fdeddc59dbc7 @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 b/sql/hive/src/test/resources/golden/auto_join30-3-57e5f7c770abbe1de38cbbcd5fd332a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 b/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join30-4-f5083eca9c3df277988d8b345b8d43 @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 b/sql/hive/src/test/resources/golden/auto_join30-5-3916f4b640f3579035153f6940113ef2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 b/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join30-6-4a9144326fc7d066c9aadb13d1b95031 @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e b/sql/hive/src/test/resources/golden/auto_join30-7-f07b674c31ca9fdf837406cb9a96108e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 b/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join30-8-8a27209399df7f9c4d15988b11753a61 @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 b/sql/hive/src/test/resources/golden/auto_join30-9-4ee48fa9bfeb818c81768b6de0517263 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join31-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 b/sql/hive/src/test/resources/golden/auto_join31-1-5a2b4475d9a88e53a2b6ec29279253c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 b/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 new file mode 100644 index 0000000000000..1434bb76ee93f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join31-2-a64aa9cb44edc6b85ed945fb13ca9c2 @@ -0,0 +1 @@ +344360994461 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join32-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 b/sql/hive/src/test/resources/golden/auto_join32-1-2e533cf988f613d5fc3fbde67ffd9118 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 b/sql/hive/src/test/resources/golden/auto_join32-10-2e8ccb343bce61564bae209a589cca85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 b/sql/hive/src/test/resources/golden/auto_join32-11-4d6fc319375b6962eca0aa63dfabfdc1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 b/sql/hive/src/test/resources/golden/auto_join32-12-a8b69002151b3df4383d2c354dbaa7d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e b/sql/hive/src/test/resources/golden/auto_join32-13-63241e3791725baad8baa00fb833ef5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f b/sql/hive/src/test/resources/golden/auto_join32-14-b754b2485c6a8d0caa5e65b1c63bbd0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 b/sql/hive/src/test/resources/golden/auto_join32-15-187fd938996ae7d96e60475fb69b8d35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 b/sql/hive/src/test/resources/golden/auto_join32-16-2e8ccb343bce61564bae209a589cca85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 b/sql/hive/src/test/resources/golden/auto_join32-17-4d6fc319375b6962eca0aa63dfabfdc1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f b/sql/hive/src/test/resources/golden/auto_join32-18-d1d78b19d484e55d9da8a320253ece0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 b/sql/hive/src/test/resources/golden/auto_join32-19-83b9df41bc46afbbafd0cd30cb982332 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d b/sql/hive/src/test/resources/golden/auto_join32-2-865207407ff1acbccb47473d87e87e8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 b/sql/hive/src/test/resources/golden/auto_join32-20-e67740fb52998f1d3afcfa667505cf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b b/sql/hive/src/test/resources/golden/auto_join32-21-da5b4647f5605dff66aa47e059f9ec8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c b/sql/hive/src/test/resources/golden/auto_join32-22-a1d339a0d904c3f35771192a026c7f9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 b/sql/hive/src/test/resources/golden/auto_join32-23-1948951cc3c06cdf962d59e932a84588 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 b/sql/hive/src/test/resources/golden/auto_join32-24-cda0994eb851b57fdb80e16b033d1b73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b b/sql/hive/src/test/resources/golden/auto_join32-25-e46226186de575c81cfab296607e1b4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a b/sql/hive/src/test/resources/golden/auto_join32-26-97d265cd7defca44e488c38bac4c5b7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc b/sql/hive/src/test/resources/golden/auto_join32-27-b034eeb850810b5004ddff1f2a530bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 b/sql/hive/src/test/resources/golden/auto_join32-28-751550ac0550e6a7dd737cad01d6d82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 b/sql/hive/src/test/resources/golden/auto_join32-29-34ecfdabf9c769027706f53fa2d66ed3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 b/sql/hive/src/test/resources/golden/auto_join32-3-9ccdfe4052062a1dfc72c711179d9e43 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 b/sql/hive/src/test/resources/golden/auto_join32-30-d3903985844b06c4af11334b72f383d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd b/sql/hive/src/test/resources/golden/auto_join32-31-2415fd7a0c2e37b09679bb4c64f321bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf b/sql/hive/src/test/resources/golden/auto_join32-4-a28f563122d1f0debd04f74c534523cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join32-5-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join32-6-442e69416faaea9309bb8c2a3eb73ef @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join32-7-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee b/sql/hive/src/test/resources/golden/auto_join32-8-999683fa0291bf439b03557edec7dcee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a b/sql/hive/src/test/resources/golden/auto_join32-9-1e6d4ec86f29d74828891c17986e84a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join4-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd b/sql/hive/src/test/resources/golden/auto_join4-1-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a b/sql/hive/src/test/resources/golden/auto_join4-2-4a36be16bcfa2bc35a8c33fb71ce2c7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 b/sql/hive/src/test/resources/golden/auto_join4-3-dc967001beb776f3a859e9360823c361 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 new file mode 100644 index 0000000000000..f7d1e92d77207 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join4-4-998c3a307b074a6505bb7fcef276be04 @@ -0,0 +1 @@ +5079148035 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join5-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd b/sql/hive/src/test/resources/golden/auto_join5-1-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 b/sql/hive/src/test/resources/golden/auto_join5-2-a13b6523395e55c551ad42d92f9dbcd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 b/sql/hive/src/test/resources/golden/auto_join5-3-b07bb1fdcd0eeeb62a8f7acb70cd2330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 new file mode 100644 index 0000000000000..e859b7c4ada7b --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join5-4-998c3a307b074a6505bb7fcef276be04 @@ -0,0 +1 @@ +9766083196 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join6-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd b/sql/hive/src/test/resources/golden/auto_join6-1-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 b/sql/hive/src/test/resources/golden/auto_join6-2-46718fdfa123cc86fe288bff4185dc90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 b/sql/hive/src/test/resources/golden/auto_join6-3-bc6b6640b266ebe9b73702d3baf09c20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 new file mode 100644 index 0000000000000..f15ab2fb14eaa --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join6-4-998c3a307b074a6505bb7fcef276be04 @@ -0,0 +1 @@ +2607643291 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join7-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 b/sql/hive/src/test/resources/golden/auto_join7-1-8f82881057bec4abf5a4d770a6f35838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e b/sql/hive/src/test/resources/golden/auto_join7-2-8a7f50dc7e382a11884f4e116041aa1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 b/sql/hive/src/test/resources/golden/auto_join7-3-56a30a1aa948bcf5ee54481897fc2208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 b/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 new file mode 100644 index 0000000000000..5f7f06c079d24 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join7-4-30d0c1a49784347fedbac21a69c3a899 @@ -0,0 +1 @@ +-2315698213 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join8-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd b/sql/hive/src/test/resources/golden/auto_join8-1-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab b/sql/hive/src/test/resources/golden/auto_join8-2-cdb9885fe05daa90c228cf5b6550eeab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca b/sql/hive/src/test/resources/golden/auto_join8-3-f1196bca86a749375da35f134206a8ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 b/sql/hive/src/test/resources/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 new file mode 100644 index 0000000000000..d73e8745ecc00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join8-4-998c3a307b074a6505bb7fcef276be04 @@ -0,0 +1 @@ +-7158439905 diff --git a/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join9-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/auto_join9-1-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 b/sql/hive/src/test/resources/golden/auto_join9-2-62638666bf7f60c0d298547ea5e93ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 b/sql/hive/src/test/resources/golden/auto_join9-3-971c44e81ce17eb0849850b72ebd20f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f b/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f new file mode 100644 index 0000000000000..16f90efbe50f6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join9-4-eaa70da463b92e85e1796277f016c18f @@ -0,0 +1 @@ +101861029915 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 b/sql/hive/src/test/resources/golden/auto_join_filters-1-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e b/sql/hive/src/test/resources/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-10-b420f24d33b26cdf6c35eb702789904e @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 b/sql/hive/src/test/resources/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-11-c06b548171893bae8def6bb348b70dc8 @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 b/sql/hive/src/test/resources/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-12-42a4901e05e9ee92abcfcef008efaa65 @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 b/sql/hive/src/test/resources/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-13-222c404c6265ed682579342113221e29 @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 b/sql/hive/src/test/resources/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-14-ecd2885156f56973960d064211ee42f0 @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b b/sql/hive/src/test/resources/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-15-11a471880f5e8fbad81e3869fe56ca4b @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e b/sql/hive/src/test/resources/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-16-f26ad8c3537dc391ab1ca6a95470f75e @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 b/sql/hive/src/test/resources/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-17-8e085f9886e5ee97334512f84bd7ab54 @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a b/sql/hive/src/test/resources/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-18-d9438071b3c731dc3f6e3b7248a1042a @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d b/sql/hive/src/test/resources/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-19-58355bd5c4b12e15cf1d3e2d8b308c9d @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 b/sql/hive/src/test/resources/golden/auto_join_filters-2-100faa7fd01bfb4390c782bb262a9403 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 b/sql/hive/src/test/resources/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-20-486a302359aecff37a4567480264bd62 @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf b/sql/hive/src/test/resources/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-21-8018df5b9572e89304b449e618fdbbf @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 b/sql/hive/src/test/resources/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-22-f9a7bff2b42a03d21d8d3190d2702451 @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 b/sql/hive/src/test/resources/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-23-2b5149e29c224a47af98ca10bc5c23b3 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 b/sql/hive/src/test/resources/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-24-a2161a58f512b8bdd836b48cd8e16668 @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 b/sql/hive/src/test/resources/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-25-1b3177a066ba352539fd5473fbeda1a9 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd b/sql/hive/src/test/resources/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-26-ecfb8fe4bf85a05f321754d8ea8cbabd @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 b/sql/hive/src/test/resources/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-27-ca73726ffaa5826a3db039cda440e6d9 @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 b/sql/hive/src/test/resources/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-28-c21c2acf7f276c0a26f0c19e3234506 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e b/sql/hive/src/test/resources/golden/auto_join_filters-29-6d8955591f62d9cfc6af17df63d3d88e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e b/sql/hive/src/test/resources/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-3-64615cc6839c697c8c028ef8bb1ac40e @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 b/sql/hive/src/test/resources/golden/auto_join_filters-30-23ab7ac8229a53d391195be7ca092429 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/auto_join_filters-31-aa161b0d9fe9d1aad10654fce0e3670b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/auto_join_filters-32-3c52df82c7d78501610f3f898103f753 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/auto_join_filters-33-1d85bb008e02ef4025171a4bc0866a6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/auto_join_filters-34-e79c906b894fed049ddfab4496a4e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 b/sql/hive/src/test/resources/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-35-3e6612a89e9124592e790594775054b1 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac b/sql/hive/src/test/resources/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-36-60a5f56f33fc8854a2b687005f0d96ac @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-37-64cabe5164130a94f387288f37b62d71 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 b/sql/hive/src/test/resources/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-38-65c867e66bc773470f1487487086a180 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada b/sql/hive/src/test/resources/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-39-cc0cf9ff9ccbc44536b3187b27aa1ada @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 b/sql/hive/src/test/resources/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-4-7df3e2e761c272ddb3654e4dd86bd131 @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 b/sql/hive/src/test/resources/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-40-4dfa64337f711dc394a289adeac8666 @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 b/sql/hive/src/test/resources/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-41-a905633bccd782f14115643b2d707b13 @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 b/sql/hive/src/test/resources/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-42-ae4cc72ddbbbd748179e0abcc985726 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b b/sql/hive/src/test/resources/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-43-f64cec45b154c4ba5172afcdff623a2b @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 b/sql/hive/src/test/resources/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-44-c552dcc9b931dff05cf8c0d712e22841 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e b/sql/hive/src/test/resources/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-45-b420f24d33b26cdf6c35eb702789904e @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 b/sql/hive/src/test/resources/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-46-c06b548171893bae8def6bb348b70dc8 @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 b/sql/hive/src/test/resources/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-47-42a4901e05e9ee92abcfcef008efaa65 @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 b/sql/hive/src/test/resources/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-48-222c404c6265ed682579342113221e29 @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 b/sql/hive/src/test/resources/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 new file mode 100644 index 0000000000000..6c578b689ebc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-49-ecd2885156f56973960d064211ee42f0 @@ -0,0 +1 @@ +4937935 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc b/sql/hive/src/test/resources/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-5-87cf8865e2c35b680bba159b88b074bc @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b b/sql/hive/src/test/resources/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-50-11a471880f5e8fbad81e3869fe56ca4b @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e b/sql/hive/src/test/resources/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-51-f26ad8c3537dc391ab1ca6a95470f75e @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 b/sql/hive/src/test/resources/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-52-8e085f9886e5ee97334512f84bd7ab54 @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a b/sql/hive/src/test/resources/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-53-d9438071b3c731dc3f6e3b7248a1042a @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d b/sql/hive/src/test/resources/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-54-58355bd5c4b12e15cf1d3e2d8b308c9d @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 b/sql/hive/src/test/resources/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-55-486a302359aecff37a4567480264bd62 @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf b/sql/hive/src/test/resources/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-56-8018df5b9572e89304b449e618fdbbf @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 b/sql/hive/src/test/resources/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-57-f9a7bff2b42a03d21d8d3190d2702451 @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 b/sql/hive/src/test/resources/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-58-2b5149e29c224a47af98ca10bc5c23b3 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 b/sql/hive/src/test/resources/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-59-a2161a58f512b8bdd836b48cd8e16668 @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 b/sql/hive/src/test/resources/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 new file mode 100644 index 0000000000000..b7c1c01983241 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-6-9362466c777fff7e677dd8da072f8744 @@ -0,0 +1 @@ +4939870 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 b/sql/hive/src/test/resources/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-60-1b3177a066ba352539fd5473fbeda1a9 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd b/sql/hive/src/test/resources/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-61-ecfb8fe4bf85a05f321754d8ea8cbabd @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 b/sql/hive/src/test/resources/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 new file mode 100644 index 0000000000000..57ad71fe4195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-62-ca73726ffaa5826a3db039cda440e6d9 @@ -0,0 +1 @@ +3080335 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 b/sql/hive/src/test/resources/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-63-c21c2acf7f276c0a26f0c19e3234506 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 b/sql/hive/src/test/resources/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-7-ae4cc72ddbbbd748179e0abcc985726 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b b/sql/hive/src/test/resources/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-8-f64cec45b154c4ba5172afcdff623a2b @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 b/sql/hive/src/test/resources/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 new file mode 100644 index 0000000000000..476d8eeee571a --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_filters-9-c552dcc9b931dff05cf8c0d712e22841 @@ -0,0 +1 @@ +3078400 diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-0-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 b/sql/hive/src/test/resources/golden/auto_join_nulls-1-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 b/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 new file mode 100644 index 0000000000000..b201b9c4d9348 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-10-ad7e37acbc658b5a822ca342fd4b9d01 @@ -0,0 +1 @@ +4542003 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 b/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 new file mode 100644 index 0000000000000..d365cdf04366c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-11-141c550a314d15c4e200e5baeb246de2 @@ -0,0 +1 @@ +4542038 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c b/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c new file mode 100644 index 0000000000000..bc7bcdca25bfb --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-12-8a65225a88da0169af26848c06cb981c @@ -0,0 +1 @@ +4543491 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 b/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 new file mode 100644 index 0000000000000..b201b9c4d9348 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-13-e9c3ae95d7edd0c311c7d57e4cebdc80 @@ -0,0 +1 @@ +4542003 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 b/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 new file mode 100644 index 0000000000000..feea6ee0a8e0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-14-2be327f5d98b6ca8a45a6e1d97948ec8 @@ -0,0 +1 @@ +3079923 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 b/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 new file mode 100644 index 0000000000000..f713b04028bbd --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-15-d2ff8e87c24e152107bba1ebf659d0c8 @@ -0,0 +1 @@ +4509891 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af b/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af new file mode 100644 index 0000000000000..a94eda6b2c374 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-16-dbe244d2c21e477c3703c4ce1903e8af @@ -0,0 +1 @@ +3113558 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 b/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 new file mode 100644 index 0000000000000..feea6ee0a8e0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-17-f3cf64fcd82d5f33d249ed64bfc13621 @@ -0,0 +1 @@ +3079923 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb b/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb new file mode 100644 index 0000000000000..88c5f95e0d838 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-18-439a409bc50dfd86dee78c151c3de5eb @@ -0,0 +1 @@ +4543526 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 b/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 new file mode 100644 index 0000000000000..88c5f95e0d838 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-19-92641e46934ebbf3d44e6e60de1882f4 @@ -0,0 +1 @@ +4543526 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 b/sql/hive/src/test/resources/golden/auto_join_nulls-2-97641998eb9ddf2dff56de6758486aa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d b/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d new file mode 100644 index 0000000000000..88c5f95e0d838 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-20-e34b2b210059a5f93c0a873d91859b5d @@ -0,0 +1 @@ +4543526 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a b/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a new file mode 100644 index 0000000000000..88c5f95e0d838 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-21-c7aaa831acbd959c6d1793056e3c288a @@ -0,0 +1 @@ +4543526 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 b/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 new file mode 100644 index 0000000000000..f2ec932ce57f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-22-f5beafb0c5ed59a1852811c9710fe9a3 @@ -0,0 +1 @@ +3112070 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa b/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa new file mode 100644 index 0000000000000..a94eda6b2c374 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-23-5805a41289a26139c06604a40bf5a6fa @@ -0,0 +1 @@ +3113558 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 b/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 new file mode 100644 index 0000000000000..f2ec932ce57f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-24-80991af26d5d37e0864ecc2c8ab0b984 @@ -0,0 +1 @@ +3112070 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 b/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 new file mode 100644 index 0000000000000..4125efd2dd065 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-3-f0befc0275bda075e4f3cd61eafcccc7 @@ -0,0 +1 @@ +13630578 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 b/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 new file mode 100644 index 0000000000000..4125efd2dd065 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-4-fc1128c86cd430db8cd4ff834be4562 @@ -0,0 +1 @@ +13630578 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc b/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc new file mode 100644 index 0000000000000..4125efd2dd065 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-5-2b5f38b7537ed5c40c0ad478b08fc1fc @@ -0,0 +1 @@ +13630578 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 b/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 new file mode 100644 index 0000000000000..e877d44372ecb --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-6-d256ec23d7b98e1517cacf5dae2f4124 @@ -0,0 +1 @@ +3078400 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 b/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 new file mode 100644 index 0000000000000..18be36a9bdb54 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-7-8395fa78507105c2a018e88f717b95e2 @@ -0,0 +1 @@ +4509856 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 b/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 new file mode 100644 index 0000000000000..f2ec932ce57f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-8-fd992f2127a139aeb554d797e748ed54 @@ -0,0 +1 @@ +3112070 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e b/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e new file mode 100644 index 0000000000000..e877d44372ecb --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_nulls-9-e3a86622a437e910b7225d1e6108da9e @@ -0,0 +1 @@ +3078400 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-0-1d8e3d660bb4b29287df5700bfe63b63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-1-1247d9a5ffabd61647697dc186c7a2a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-10-f6d5bb38137da35b91da901ba310c2b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-2-3ce329282fc72110e9ed6c78fa914395 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-3-6e9d99d2a0cac78b7fe242dc1e43d3d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-4-7fb82039c95389f11b174d9f22aacb35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee b/sql/hive/src/test/resources/golden/auto_join_reordering_values-5-b5da89b0cb325cf684406b620eb9d8ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 b/sql/hive/src/test/resources/golden/auto_join_reordering_values-6-2c91dc4b7c00d5f09862119c12295532 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b b/sql/hive/src/test/resources/golden/auto_join_reordering_values-7-880ba1dba6057dd6cde89d1b17724a6b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed b/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_join_reordering_values-8-950af86c321a67ab3ed0fa5b63ea6aed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b b/sql/hive/src/test/resources/golden/auto_join_reordering_values-9-ae02756bd46266ec7fd9cc809bc4757b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-10-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-11-906a582602602372e1d4776243abeab5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-12-94538bc7322522a5534cafc0551d2189 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-13-1d603e61c2cb888499504ddab98ccc65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 new file mode 100644 index 0000000000000..62f9457511f87 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-14-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -0,0 +1 @@ +6 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-15-758d5532083d6279e169b54fd69bb580 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-16-2798f20aaf0fe5505c34b118e4b10bc5 new file mode 100644 index 0000000000000..2eafac63a9a98 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-17-ca77b5fb54de526972c16ad6118e86d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-18-21269869cd3aaf4ade2170d9017de018 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-19-853c4fa5a2385b92fdb39d0ac2926973 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-20-4e0e8cd0626a84b21ca7d2f633623578 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-21-e210ced77b32cd7ce87044bb3e3370d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-22-2fe7b834b341bf18e36cd79dd00ec16a @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-23-63d105e0acd3521bb29ba8cec9ac4583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-24-43ba2c72db9db1ec18d835ec978f8da1 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-25-7b2a1128afe35706f1540bfc251d0736 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-26-b66c416fdd98d76981f19e9c14b6a562 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-27-5438118dc1d9fab501a8e60eddd625a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-28-b889b147255231f7fe44bd57e1f8ba66 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-29-ff6eca271c60cb15a3ea2395ac737b0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce new file mode 100644 index 0000000000000..2ebc6516c7df1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-30-b9d66e78b8898a97a42d1118300fa0ce @@ -0,0 +1 @@ +56 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-31-d25f41c6f7c20044ed4d9a9905fdfcce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-32-b0ca9e20cd48457e6cf1c313d5505213 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-33-5f340dbc8126f7e336d3c85e9ab346b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-34-76ded9e08c765bf2e1b670b4ffb938b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-35-4e6a34e1f68538ad9e25b7c3a8d18e76 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-36-1ba279881865c861a793797ae84a3934 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-37-a45927057c01fd54818b5dd50e77f60e new file mode 100644 index 0000000000000..3d2e6576f591f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-38-a988727daa49fb5e190f81c027bb7005 new file mode 100644 index 0000000000000..34d56da297220 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-39-20c26228d10872eec10dbb9322dd74da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-40-4c57b1c6c081294cbd72626ff0fd940e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-41-81b3db8d18d4b3843ed0be6eca5d793c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-42-80db3a67d59c8710edf9f695e7eeb37c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-43-a45927057c01fd54818b5dd50e77f60e new file mode 100644 index 0000000000000..3d2e6576f591f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-44-48b32dd521ddf1af1c8075ecbeccaa75 new file mode 100644 index 0000000000000..4a9735f855f96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-0-b24f5a262c6693f31ed376a5da0787f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-10-9666fb18356436e2800550df9ca90c04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-11-a54cefeeb6d79c72f01c61035e9dcf15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-12-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-13-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-14-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-15-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-16-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-17-30259eb1873d8f5d00dccd8af0b0ccbc @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-18-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-19-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-20-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-21-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-22-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-23-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-24-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-3-18bd222285d3a3bd71d3cfa217d9b1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-4-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-5-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-6-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-7-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-8-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_1-9-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-10-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-11-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-12-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-13-469a09efa93fa9aec154a5967eec09c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-14-95e18bd00f2de246efca1756681c1e87 @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-15-6a7fdb423721e7aefa2efda26785e1a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e new file mode 100644 index 0000000000000..301160a93062d --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-16-caa641c820fcc5f601758c5f0385b4e @@ -0,0 +1 @@ +8 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-5-9140b367b5680860f4c7c0238377583f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-6-70c1d66123d434d3a1e1801e0b19bc3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-7-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-8-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_10-9-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-0-4705fafa08c6d927aa01337e19605c8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-10-9666fb18356436e2800550df9ca90c04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-11-a54cefeeb6d79c72f01c61035e9dcf15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-12-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-13-398b81a1928284f29e832838ec3764fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-14-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-15-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-16-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-17-442e69416faaea9309bb8c2a3eb73ef @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-18-d8260daa82c8439e0c80a63998bd5d2e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-19-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-20-62fab16c00f510c001f146c929360c71 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-21-c4d55c247c9326f474d89b29b81d60aa @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-3-b4a6a67ac771394140ed695810930ac6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-4-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-5-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-6-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-7-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-8-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_11-9-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-0-d7af66a0aa504ad44bf01d5a2e7cdcec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-10-9666fb18356436e2800550df9ca90c04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-11-a54cefeeb6d79c72f01c61035e9dcf15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-12-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-13-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-14-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-15-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-16-746f0c0dd71cd5cb6673fbd53ef05a1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-17-a8c60901367229310f86a8521a26478a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-18-f50f21c997d775c369fd52f9bafb9b36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-19-fe49b6f198661d2e020a0c8bd26c9237 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-20-3404f0b912c898d6c81aa88bf0cd8c11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc new file mode 100644 index 0000000000000..83be903e06482 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-21-4ecd65f0e26e981b66770b3e91e128fc @@ -0,0 +1 @@ +570 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-3-18bd222285d3a3bd71d3cfa217d9b1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-4-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-5-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-6-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-7-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-8-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_12-9-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-10-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-11-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-12-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-13-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-14-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-15-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-16-14ad1ec6ac3dbedb29d43cf178fa8552 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-17-b366dcf84021a7dc4a17a52fe381b5f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-18-cc27d771c9a20d3d83f87802e1a9dbe2 new file mode 100644 index 0000000000000..251ff85eda52d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-19-4b2ac2865384fbca7f374191d8021d51 new file mode 100644 index 0000000000000..af8f457e93476 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-20-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-21-ea23403b9eb55e8b06d1c198e439569f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-22-14ad1ec6ac3dbedb29d43cf178fa8552 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-23-b366dcf84021a7dc4a17a52fe381b5f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-24-cc27d771c9a20d3d83f87802e1a9dbe2 new file mode 100644 index 0000000000000..251ff85eda52d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-25-4b2ac2865384fbca7f374191d8021d51 new file mode 100644 index 0000000000000..af8f457e93476 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-26-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-27-14ad1ec6ac3dbedb29d43cf178fa8552 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-28-b366dcf84021a7dc4a17a52fe381b5f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-29-cc27d771c9a20d3d83f87802e1a9dbe2 new file mode 100644 index 0000000000000..251ff85eda52d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-30-4b2ac2865384fbca7f374191d8021d51 new file mode 100644 index 0000000000000..af8f457e93476 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-5-d73046d4785e9c89acb10eea77d32ca8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-6-3d02238197b076b6f77daacb81aa2cb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-7-6f949602369ac3af6ded9884bc525310 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-8-b334c03af5acdcb136072bb427683bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_13-9-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-10-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-11-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-12-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-13-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-14-51e4c81f56c64f6aa25322055694f641 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a new file mode 100644 index 0000000000000..1758dddccea2b --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-15-43ad2152b18d711adbdd1aeb940b662a @@ -0,0 +1 @@ +32 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-16-51ee88184cbc22b5bef4e96856e41e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-17-334529f1a720bfb408efee90bc8be61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd new file mode 100644 index 0000000000000..bea0d09c49935 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-18-a16ff76d72ad428fb9d43ab910f259fd @@ -0,0 +1 @@ +207 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-5-945b37381c2719e18e2945bf8b4e56ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-6-3d02238197b076b6f77daacb81aa2cb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-7-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-8-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_14-9-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-10-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-11-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-12-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-13-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-14-62b7e43463386c11e031cf7e4f584a53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-15-3c37a21813ee34d4d1f9e01f5f3ef5d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-5-945b37381c2719e18e2945bf8b4e56ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-6-3d02238197b076b6f77daacb81aa2cb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-7-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-8-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_15-9-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-0-ac562e10c3d4dd7c7cce920d29cde65d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-10-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-11-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-12-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-13-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-14-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-15-ec410b2c54c1ce7001abe7130a3b1c21 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-16-af6016f3db000e6e180e2f3b10f120ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-17-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-18-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-19-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-20-8180638a57b64557e02815c863031755 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-21-e6283ea14d493b0d7bf390249665f289 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-22-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-3-55c1e972192580d734fad7f57dd62e6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-4-210f08b7e8c20c9ff364c215af412d87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-5-bfc04dda7e11f06d01689f2b57959ed7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-6-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-7-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-8-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_2-9-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-0-71378da1900d130fd68aaebc45f87313 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-10-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-11-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-12-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-13-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-14-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-15-30259eb1873d8f5d00dccd8af0b0ccbc @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-16-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-17-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-18-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-19-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-20-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-21-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-22-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-3-6876517daaf54cadefb6bbbf54bd4a24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-4-cd25b8502c668759783aaba4d550a05f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-5-18bd222285d3a3bd71d3cfa217d9b1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-6-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-7-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-8-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_3-9-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-0-1528e7173b91cd90f101ca27f51d963c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-10-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-11-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-12-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-13-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-14-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-15-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-16-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-17-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-18-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-19-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-20-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-21-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-22-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-23-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..c24b6ae77df02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-24-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +38 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-3-55c1e972192580d734fad7f57dd62e6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-4-210f08b7e8c20c9ff364c215af412d87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-5-6876517daaf54cadefb6bbbf54bd4a24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-6-cd25b8502c668759783aaba4d550a05f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-7-c20aa9939d703c529c4538994dc6f066 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-8-baa1253610c081917208199feb52a768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_4-9-bfc04dda7e11f06d01689f2b57959ed7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-0-9f8764dddb7b106f879d1a7c4318310d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-1-fac79d1e5c34142393fc328b2935a9b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-10-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-11-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-12-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-13-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..dec2bf5d6199c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-14-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-15-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..dec2bf5d6199c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-16-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-17-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-18-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-19-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-2-7282f71445d3b6acef073be9b7cbab98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..dec2bf5d6199c --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-20-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-3-56f0862dbe9f7c0eecafe22d5d185c7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-4-be71f06ad593935a8e81d61b695b2052 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-5-ce5ee903a36a074293fa509149d94447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-6-12e01dc9146f45ded0a6655cb04467b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-7-ec1aaae06a8dbb2faf36b53246124d4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-8-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_5-9-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-10-34779e6a90b2e9968a9a98b048cdaab6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-11-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-12-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-13-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-14-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-15-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-16-ea23403b9eb55e8b06d1c198e439569f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-17-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-18-32efb3656e05e40f9f928bbcb11d010 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-19-cf41f7ce9478536e823107d1810ff1d7 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-20-b23f9ec74e58e5c53417bfff6794e8fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-21-b55506a213ec710004e6d7f3462834d0 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-22-4b066e39be51ea19a1790c1287ad0d2c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-23-4281442c87dcf6007f8bd42504eba186 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-24-12ec3636a4c62fd56b40da2979f53f5f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-25-52f0e65724d29e2b4054b59a50d2837b @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-26-bf7478a041a164ef219964cb865aa63b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-27-961f141836f2cc9521f681cadbc3d140 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-28-171974ff7145ffc85c8ba2724ef1f31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-29-fd0cc412e0987569a4ed879454b53fb0 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-30-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-31-940f423a57afd2734f62d93bcd4d7caf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-32-cf41f7ce9478536e823107d1810ff1d7 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-33-12ec3636a4c62fd56b40da2979f53f5f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-34-52f0e65724d29e2b4054b59a50d2837b @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-35-bf7478a041a164ef219964cb865aa63b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-36-961f141836f2cc9521f681cadbc3d140 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-37-171974ff7145ffc85c8ba2724ef1f31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 new file mode 100644 index 0000000000000..92c15ec11569f --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-38-fd0cc412e0987569a4ed879454b53fb0 @@ -0,0 +1 @@ +2654 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-5-612f97716b8efe4b659206938e5ea5f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-6-64856231335fc5fec61c3fd3aceefcc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-7-b34c9b18481df043912e910ed3a5f149 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-8-57e68163453d4632ef740ce1223f44d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b b/sql/hive/src/test/resources/golden/auto_sortmerge_join_6-9-36d2a698f88e53ab2a66e8baa980299b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-0-fa10661c7e8791fb319ade49f3cca50 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-10-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-11-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-12-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-13-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-14-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-15-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-16-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-17-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-18-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-19-b1607a2f1e7da8ac0a9a035b99f81d28 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-20-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-21-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-22-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-23-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-24-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-25-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-26-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-3-55c1e972192580d734fad7f57dd62e6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-4-210f08b7e8c20c9ff364c215af412d87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-5-6876517daaf54cadefb6bbbf54bd4a24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-6-cd25b8502c668759783aaba4d550a05f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-7-c20aa9939d703c529c4538994dc6f066 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-8-baa1253610c081917208199feb52a768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_7-9-bfc04dda7e11f06d01689f2b57959ed7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-0-42977c556a54aaaee9d1e59a6dcc06aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-1-e3951e29e1e87b77ea735f40fd58735 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-10-310c8d652c6f549b7759baec6012b77d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-11-b806b5b4eb8a703b2ba43afdce4d0bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-12-9666fb18356436e2800550df9ca90c04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-13-a54cefeeb6d79c72f01c61035e9dcf15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-14-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-15-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-16-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-17-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-18-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-19-30259eb1873d8f5d00dccd8af0b0ccbc @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-2-c5a30be03ba36f1fb6cc0b4e7c978838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-20-a6ef74ae9c70c382abb4d361e6f1e070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-21-325432a220aa3ebe8b816069916924d8 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-22-6f2f1144ab2b8b12684f3fbc55e27bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-23-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-24-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-25-a0fc12fc2b968d7e85e6c1e2fd70cd94 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-26-8180638a57b64557e02815c863031755 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-27-e6283ea14d493b0d7bf390249665f289 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 new file mode 100644 index 0000000000000..aa92725341cfd --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-28-7d0c37fc09323ce11aae0b58dc687660 @@ -0,0 +1 @@ +76 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-3-6876517daaf54cadefb6bbbf54bd4a24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-4-cd25b8502c668759783aaba4d550a05f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-5-18bd222285d3a3bd71d3cfa217d9b1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-6-bb969d3ec0038215a2698afceeb02b3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-7-2c3617157639fcd296a8ea2f121c58ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-8-307339322d96b8f923d57c0dc9cdcb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_8-9-94cc219f61413ab321916821e1288152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-10-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-11-cda81d86d127fca0e2fbc2161e91400d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-12-906a582602602372e1d4776243abeab5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-13-94538bc7322522a5534cafc0551d2189 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-14-133023474337f2cdc53ee82ffeb1c13e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-15-63261d35ddda973eeeb97b994ab7a476 new file mode 100644 index 0000000000000..4a9735f855f96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-16-1d603e61c2cb888499504ddab98ccc65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 new file mode 100644 index 0000000000000..62f9457511f87 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-17-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -0,0 +1 @@ +6 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-18-758d5532083d6279e169b54fd69bb580 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-19-2798f20aaf0fe5505c34b118e4b10bc5 new file mode 100644 index 0000000000000..2eafac63a9a98 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-20-ca77b5fb54de526972c16ad6118e86d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-21-21269869cd3aaf4ade2170d9017de018 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-22-853c4fa5a2385b92fdb39d0ac2926973 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-23-4e0e8cd0626a84b21ca7d2f633623578 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-24-e210ced77b32cd7ce87044bb3e3370d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-25-2fe7b834b341bf18e36cd79dd00ec16a @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-26-63d105e0acd3521bb29ba8cec9ac4583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-27-43ba2c72db9db1ec18d835ec978f8da1 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-28-5965c5c6ef08240eb27eb9620cc2338 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-29-b66c416fdd98d76981f19e9c14b6a562 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-30-4376bdd8412f94fe184d46481fee345d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-31-b889b147255231f7fe44bd57e1f8ba66 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-32-c5c3668b2434a5b90f308ca4bbdcd647 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-33-aa8d713ad4e19b72b5bd7628d60c295e @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-34-ff6eca271c60cb15a3ea2395ac737b0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce new file mode 100644 index 0000000000000..2ebc6516c7df1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-35-b9d66e78b8898a97a42d1118300fa0ce @@ -0,0 +1 @@ +56 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-36-d25f41c6f7c20044ed4d9a9905fdfcce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-37-b0ca9e20cd48457e6cf1c313d5505213 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-38-f135547e33c01d1f543c8b1349d60348 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-39-906a582602602372e1d4776243abeab5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-40-94538bc7322522a5534cafc0551d2189 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-41-133023474337f2cdc53ee82ffeb1c13e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-42-63261d35ddda973eeeb97b994ab7a476 new file mode 100644 index 0000000000000..4a9735f855f96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-43-1d603e61c2cb888499504ddab98ccc65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 new file mode 100644 index 0000000000000..62f9457511f87 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-44-d5183dfa8d9fb9175478fb1c2f2edb97 @@ -0,0 +1 @@ +6 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-45-758d5532083d6279e169b54fd69bb580 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-46-2798f20aaf0fe5505c34b118e4b10bc5 new file mode 100644 index 0000000000000..2eafac63a9a98 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-47-ca77b5fb54de526972c16ad6118e86d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-48-21269869cd3aaf4ade2170d9017de018 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-49-853c4fa5a2385b92fdb39d0ac2926973 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-5-9140b367b5680860f4c7c0238377583f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-50-4e0e8cd0626a84b21ca7d2f633623578 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-51-e210ced77b32cd7ce87044bb3e3370d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-52-2fe7b834b341bf18e36cd79dd00ec16a @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-53-63d105e0acd3521bb29ba8cec9ac4583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-54-43ba2c72db9db1ec18d835ec978f8da1 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-55-4376bdd8412f94fe184d46481fee345d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-56-b889b147255231f7fe44bd57e1f8ba66 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-57-c5c3668b2434a5b90f308ca4bbdcd647 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-58-aa8d713ad4e19b72b5bd7628d60c295e @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-59-ff6eca271c60cb15a3ea2395ac737b0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-6-70c1d66123d434d3a1e1801e0b19bc3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce new file mode 100644 index 0000000000000..2ebc6516c7df1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-60-b9d66e78b8898a97a42d1118300fa0ce @@ -0,0 +1 @@ +56 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-61-d25f41c6f7c20044ed4d9a9905fdfcce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-62-b0ca9e20cd48457e6cf1c313d5505213 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-7-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-8-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/auto_sortmerge_join_9-9-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/ba_table1-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 b/sql/hive/src/test/resources/golden/ba_table1-1-7b43ffa8083fda74ab342029dce2e3d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table1-2-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..d0eea8a3ca661 --- /dev/null +++ b/sql/hive/src/test/resources/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:tnachen, createTime:1392426673, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426673}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/ba_table2-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a b/sql/hive/src/test/resources/golden/ba_table2-1-6f3e37cab4fdc60491dea1ff6fc9931a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 b/sql/hive/src/test/resources/golden/ba_table2-2-8491941c2baa0c7d96e17b8f47dfebe7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table2-3-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..0a3c9f383a6ef --- /dev/null +++ b/sql/hive/src/test/resources/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:tnachen, createTime:1392426674, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:from deserializer), FieldSchema(name:ba_val, type:binary, comment:from deserializer)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazybinary.LazyBinarySerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426674, transient_lastDdlTime=1392426674}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/ba_table3-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d b/sql/hive/src/test/resources/golden/ba_table3-1-28b12606c5369c783e63c17826a18d0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/ba_table_udfs-0-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b b/sql/hive/src/test/resources/golden/ba_table_udfs-1-3baac4c47f9cb2895f01bcfccd5f904b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/ba_table_union-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 b/sql/hive/src/test/resources/golden/ba_table_union-1-3f8df0a4ab12f1a31a7906e77e9b7b75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/ba_table_union-2-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..66d33b789f1a7 --- /dev/null +++ b/sql/hive/src/test/resources/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:tnachen, createTime:1392426676, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.mapred.TextInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.lazy.LazySimpleSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426676}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 b/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 new file mode 100644 index 0000000000000..2e65efe2a145d --- /dev/null +++ b/sql/hive/src/test/resources/golden/binary_constant-0-8c922b2264278dd481ef0dff2088e2b8 @@ -0,0 +1 @@ +a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 b/sql/hive/src/test/resources/golden/binary_output_format-0-483cdc9eade175b0c89b9f5b3eb505f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef b/sql/hive/src/test/resources/golden/binary_output_format-1-9e0909b6330578a25806527dd0ecf7ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 b/sql/hive/src/test/resources/golden/binary_output_format-2-a42be5ce444ef1d2a1dbe654a57d6f55 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 b/sql/hive/src/test/resources/golden/binary_output_format-3-84db2ef4a7f2151e26457db559b862d9 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/binary_table_bincolserde-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 b/sql/hive/src/test/resources/golden/binary_table_bincolserde-1-3defb7199b65cfd10cb2f46275c581b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e b/sql/hive/src/test/resources/golden/binary_table_bincolserde-2-706a062089583074f30fb13c661fc81e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/binary_table_bincolserde-3-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..df14abbcc193f --- /dev/null +++ b/sql/hive/src/test/resources/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:tnachen, createTime:1392426678, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:from deserializer), FieldSchema(name:ba_val, type:binary, comment:from deserializer)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{last_modified_by=tnachen, last_modified_time=1392426678, transient_lastDdlTime=1392426678}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 b/sql/hive/src/test/resources/golden/binary_table_colserde-0-943f9cca5ed3bba5b2c22f49885722c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 b/sql/hive/src/test/resources/golden/binary_table_colserde-1-179ac81920d8dfa6e324cc881b5f1624 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 b/sql/hive/src/test/resources/golden/binary_table_colserde-2-7e72160489bbb59dadf24e0cc239a5f4 new file mode 100644 index 0000000000000..0c246ae33e56f --- /dev/null +++ b/sql/hive/src/test/resources/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:tnachen, createTime:1392426679, lastAccessTime:0, retention:0, sd:StorageDescriptor(cols:[FieldSchema(name:ba_key, type:binary, comment:null), FieldSchema(name:ba_val, type:binary, comment:null)], location:file:/tmp/sharkWarehouse2805388002645706641/ba_test, inputFormat:org.apache.hadoop.hive.ql.io.RCFileInputFormat, outputFormat:org.apache.hadoop.hive.ql.io.RCFileOutputFormat, compressed:false, numBuckets:-1, serdeInfo:SerDeInfo(name:null, serializationLib:org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe, parameters:{serialization.format=1}), bucketCols:[], sortCols:[], parameters:{}, skewedInfo:SkewedInfo(skewedColNames:[], skewedColValues:[], skewedColValueLocationMaps:{}), storedAsSubDirectories:false), partitionKeys:[], parameters:{transient_lastDdlTime=1392426679}, viewOriginalText:null, viewExpandedText:null, tableType:MANAGED_TABLE) diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 b/sql/hive/src/test/resources/golden/binarysortable_1-0-3562c2ed956a59cc98362d2f64e19ce1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 b/sql/hive/src/test/resources/golden/binarysortable_1-1-4a0ed18480313e66b869ec4f49371cf5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 b/sql/hive/src/test/resources/golden/binarysortable_1-2-faa8d95365e4116734a056c911350c05 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 b/sql/hive/src/test/resources/golden/binarysortable_1-3-d6518380547e7eef338886f3bdc7bdd2 new file mode 100644 index 0000000000000..27687b47813a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 b/sql/hive/src/test/resources/golden/cast1-0-b0e8966b7c06be9e044ed30b487d0661 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 b/sql/hive/src/test/resources/golden/cast1-1-1ee16b8209701131017533cfa6eb4680 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c b/sql/hive/src/test/resources/golden/cast1-2-3fe73e7435e30b37266ef6a33537dc4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a b/sql/hive/src/test/resources/golden/cast1-3-18dc2ce8a8b2486d268bceef63aa0c2a new file mode 100644 index 0000000000000..9bdd310949be8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 b/sql/hive/src/test/resources/golden/cast_to_int-0-ec8e07c04f0e9bc9bb34db97ee6faa98 new file mode 100644 index 0000000000000..bbe268ea91ddf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 b/sql/hive/src/test/resources/golden/cluster-0-16681f9c2bdd44278817d72c138b6ee1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 b/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 new file mode 100644 index 0000000000000..87d0f8dd52b68 --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-1-707a2295731e0d631a6c5f71c745c8d5 @@ -0,0 +1 @@ +10 val_10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 b/sql/hive/src/test/resources/golden/cluster-10-bba339a22907c06d1d01ba9cd7ea8f5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e b/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-11-dcf78a6537ba2b4d4b828a9a27cf545e @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 b/sql/hive/src/test/resources/golden/cluster-12-6ad920e2ae83c78fccb06ff65308a438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 b/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-13-12635b4b7f34eba5554d5b892b5b64e7 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 b/sql/hive/src/test/resources/golden/cluster-14-cd2e125bceb1611137f0750f5d69c475 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 b/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-15-b4c15c85c18f310f1b5bc56a78ad94b0 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 b/sql/hive/src/test/resources/golden/cluster-16-a2d5e5ec2504041ea1a62856c7086451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 b/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 new file mode 100644 index 0000000000000..1a018b640eb6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-17-62979aa9e6b4e6ffb44ec452aabbef65 @@ -0,0 +1 @@ +20 val_20 20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 b/sql/hive/src/test/resources/golden/cluster-18-e01f450969ae7e1cd018e6ef0cc67141 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 b/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 new file mode 100644 index 0000000000000..333a4cf9af123 --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-19-e5284c7a7c36ee55740bd127ef4bf8c7 @@ -0,0 +1 @@ +20 val_20 20 val_20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 b/sql/hive/src/test/resources/golden/cluster-2-50197277eb03ad20696a135bd7d18de7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-20-294891c1d956245540a80aa800ba393d b/sql/hive/src/test/resources/golden/cluster-20-294891c1d956245540a80aa800ba393d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f b/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f new file mode 100644 index 0000000000000..333a4cf9af123 --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-21-4787b258a786cf195bcb59cd90f6013f @@ -0,0 +1 @@ +20 val_20 20 val_20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 b/sql/hive/src/test/resources/golden/cluster-22-8801aa93cf6dba7e13e99a0260fde68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e b/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e new file mode 100644 index 0000000000000..1a018b640eb6f --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-23-b66ed6ead4deecd49f0f67de1f2bab2e @@ -0,0 +1 @@ +20 val_20 20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae b/sql/hive/src/test/resources/golden/cluster-24-f492a7f78faf180621e83e5a69aa1eae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a b/sql/hive/src/test/resources/golden/cluster-25-f57ce48b6a6e671b58c96535ab482b6a new file mode 100644 index 0000000000000..a79654385b09f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f b/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-3-530671e2a5b8983c60cfedaf486f0f0f @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc b/sql/hive/src/test/resources/golden/cluster-4-cb4af90f52f2626213f918fda3b81dfc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 b/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-5-e99040f6a24c53a4c89801ff3663ff72 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 b/sql/hive/src/test/resources/golden/cluster-6-56f8e3e7abe504522a2bfd77b5be3270 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 b/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-7-a22600d60c81a25061b1e20b6726c691 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d b/sql/hive/src/test/resources/golden/cluster-8-251b178e4fe39ea03a30d2b9bd40710d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 b/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 new file mode 100644 index 0000000000000..54864d264245d --- /dev/null +++ b/sql/hive/src/test/resources/golden/cluster-9-cc36ac7ee5e8c6ea21b956abbc2506e2 @@ -0,0 +1 @@ +20 val_20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d b/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-0-d300f67f11082f3614a8e93e8808960d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c b/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-1-3a1329c4145738961e1b8bdbd056497c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 b/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-2-b4b94bc85ee3bdef2b458d974d36935 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f b/sql/hive/src/test/resources/golden/columnarserde_create_shortcut-3-a66dbbe24c9eb33f40dd353ed5a5c14f new file mode 100644 index 0000000000000..15e92afeeca27 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c b/sql/hive/src/test/resources/golden/columnstats_partlvl-0-78efaffd8fd417371fb888d6d1ba995c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 b/sql/hive/src/test/resources/golden/columnstats_partlvl-1-6483a7ac7f2312cbbf3fce4c4740edf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 b/sql/hive/src/test/resources/golden/columnstats_partlvl-2-eb06998a353abc3022a9e0a17d5dba59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd b/sql/hive/src/test/resources/golden/columnstats_partlvl-3-3ab5479f002e412965f259485075f6bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 b/sql/hive/src/test/resources/golden/columnstats_partlvl-4-30d92b61681b9ae7786ed46c3c3e808 new file mode 100644 index 0000000000000..6ad4db1788424 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 b/sql/hive/src/test/resources/golden/columnstats_partlvl-5-f37a302cb19b4fe1c8280c08153294a3 new file mode 100644 index 0000000000000..91ce2a521cde1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 b/sql/hive/src/test/resources/golden/columnstats_partlvl-6-86ba38eff353a720bbabf726365b6712 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 b/sql/hive/src/test/resources/golden/columnstats_partlvl-7-1f91b01f40c5e87aa33ceb9b5fa0b2f1 new file mode 100644 index 0000000000000..777024f6946e3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce b/sql/hive/src/test/resources/golden/columnstats_partlvl-8-dc5682403f4154cef30860f2b4e37bce new file mode 100644 index 0000000000000..cd72c7efbf56f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe b/sql/hive/src/test/resources/golden/columnstats_partlvl-9-ddd27c2a530d8cea3df6f2a4917aabe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-0-ada4896eb3d16ba1cd5ed5b439f2a875 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c b/sql/hive/src/test/resources/golden/columnstats_tbllvl-1-d5b5623715ee672e2f12b3fa775dc67c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-2-9cfeaeeb342d7eda1f9be97b4f7991f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-3-7c45bd1125420b85a0374fecbf947a95 new file mode 100644 index 0000000000000..d1e5e7375467d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-4-d20bef3e7fe811a9029c969dec1b6770 new file mode 100644 index 0000000000000..3f3aa581b43f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c b/sql/hive/src/test/resources/golden/columnstats_tbllvl-5-ebf2d35321a3af996c150c6072d16a8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 b/sql/hive/src/test/resources/golden/columnstats_tbllvl-6-46c090f169c8dc7dbc24c2264da20f55 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d b/sql/hive/src/test/resources/golden/columnstats_tbllvl-7-ce5ad528f8b9ad2c309aea199cbe769d new file mode 100644 index 0000000000000..4ff444febde63 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f b/sql/hive/src/test/resources/golden/columnstats_tbllvl-8-716c74ca8a0fc8c88e898163a8e41b8f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine1-0-84b74227c9f1563f530cd3ac3b333e54 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine1-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine1-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine1-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine1-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine1-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 b/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine1-6-1d1f97cce07323812de3027920b04b75 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 b/sql/hive/src/test/resources/golden/combine1-7-f7d8d6d0e9504b4dd2f1a557c0c69c30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c b/sql/hive/src/test/resources/golden/combine1-8-da1fda96db80592bf2bbda8f22b5687c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 b/sql/hive/src/test/resources/golden/combine1-9-e5ce23369b0ad260512a0f61c6969b73 new file mode 100644 index 0000000000000..8f8e1f4b21fe3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/combine2-0-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a b/sql/hive/src/test/resources/golden/combine2-10-54649f87d403c6fcb163c4d51e382d3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 b/sql/hive/src/test/resources/golden/combine2-11-2d2d73a929c7d995ea57b40529b74b56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2-12-cd15ffd140539cf86090814729ec4748 b/sql/hive/src/test/resources/golden/combine2-12-cd15ffd140539cf86090814729ec4748 new file mode 100644 index 0000000000000..4c538dc5ccda1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 b/sql/hive/src/test/resources/golden/combine2-13-5ac3e540fd24f94fee378e49597817b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c b/sql/hive/src/test/resources/golden/combine2-14-4695309eb4e91ef29c9857aa8fd6130c new file mode 100644 index 0000000000000..d492cb3452d87 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 b/sql/hive/src/test/resources/golden/combine2-15-dd652175dac4463fed3c56aded11e6c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2-16-557997716a68312e8cae75428e3ce31 b/sql/hive/src/test/resources/golden/combine2-16-557997716a68312e8cae75428e3ce31 new file mode 100644 index 0000000000000..48082f72f087c --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-16-557997716a68312e8cae75428e3ce31 @@ -0,0 +1 @@ +12 diff --git a/sql/hive/src/test/resources/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 b/sql/hive/src/test/resources/golden/combine2-17-8e4598e3f0701478ed12042438699ce5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 b/sql/hive/src/test/resources/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 new file mode 100644 index 0000000000000..41be2c5b41f01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-18-2af7419c1d84fe155e23f3972e049b97 @@ -0,0 +1,2 @@ +2008-04-08 1000 +2008-04-09 1000 diff --git a/sql/hive/src/test/resources/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine2-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/combine2-7-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-7-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 b/sql/hive/src/test/resources/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-8-99d1f07b2ce904afd6a809fd1814efe9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 b/sql/hive/src/test/resources/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2-9-30cb07965e4b5025545361b948fc83c2 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/combine2_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 b/sql/hive/src/test/resources/golden/combine2_hadoop20-10-a5c8b73241600b6e2af8b3a41f5f5055 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 b/sql/hive/src/test/resources/golden/combine2_hadoop20-11-e25fd9e055710371ec90e0730c92f272 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 b/sql/hive/src/test/resources/golden/combine2_hadoop20-12-cd15ffd140539cf86090814729ec4748 new file mode 100644 index 0000000000000..80fa68b84c17e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 b/sql/hive/src/test/resources/golden/combine2_hadoop20-13-5ac3e540fd24f94fee378e49597817b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c b/sql/hive/src/test/resources/golden/combine2_hadoop20-14-4695309eb4e91ef29c9857aa8fd6130c new file mode 100644 index 0000000000000..ff6141674e603 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 b/sql/hive/src/test/resources/golden/combine2_hadoop20-15-dd652175dac4463fed3c56aded11e6c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 b/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 new file mode 100644 index 0000000000000..3cacc0b93c9c9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-16-557997716a68312e8cae75428e3ce31 @@ -0,0 +1 @@ +12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 b/sql/hive/src/test/resources/golden/combine2_hadoop20-17-8e4598e3f0701478ed12042438699ce5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 b/sql/hive/src/test/resources/golden/combine2_hadoop20-18-2af7419c1d84fe155e23f3972e049b97 new file mode 100644 index 0000000000000..1a0aa74952afa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-7-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 b/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-8-99d1f07b2ce904afd6a809fd1814efe9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 b/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_hadoop20-9-30cb07965e4b5025545361b948fc83c2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-0-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-1-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 b/sql/hive/src/test/resources/golden/combine2_win-10-66ee1339a5a8cc224e83d583acc709c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 b/sql/hive/src/test/resources/golden/combine2_win-11-cd15ffd140539cf86090814729ec4748 new file mode 100644 index 0000000000000..80fa68b84c17e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 b/sql/hive/src/test/resources/golden/combine2_win-12-5ac3e540fd24f94fee378e49597817b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c b/sql/hive/src/test/resources/golden/combine2_win-13-4695309eb4e91ef29c9857aa8fd6130c new file mode 100644 index 0000000000000..ff6141674e603 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 b/sql/hive/src/test/resources/golden/combine2_win-14-dd652175dac4463fed3c56aded11e6c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 b/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 new file mode 100644 index 0000000000000..3cacc0b93c9c9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-15-557997716a68312e8cae75428e3ce31 @@ -0,0 +1 @@ +12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 b/sql/hive/src/test/resources/golden/combine2_win-16-8e4598e3f0701478ed12042438699ce5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 b/sql/hive/src/test/resources/golden/combine2_win-17-2af7419c1d84fe155e23f3972e049b97 new file mode 100644 index 0000000000000..1a0aa74952afa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-2-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-3-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-4-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-6-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 b/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-7-99d1f07b2ce904afd6a809fd1814efe9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 b/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine2_win-8-30cb07965e4b5025545361b948fc83c2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af b/sql/hive/src/test/resources/golden/combine2_win-9-7fd472d5ba7a41eb391f723c6dcf42af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine3-0-84b74227c9f1563f530cd3ac3b333e54 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine3-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 b/sql/hive/src/test/resources/golden/combine3-10-fb173ed4483e732d367e51f88be793b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 b/sql/hive/src/test/resources/golden/combine3-11-30b8b79049113252bec1cbeac4018a3 new file mode 100644 index 0000000000000..80665a4d4c983 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b b/sql/hive/src/test/resources/golden/combine3-12-f4baee25e0ad813258d67d707a6fc43b new file mode 100644 index 0000000000000..5a87a3aec7cf5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc b/sql/hive/src/test/resources/golden/combine3-13-1c359bedf474e8e26f3b3562f7af6edc new file mode 100644 index 0000000000000..7b6455db7834b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine3-14-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine3-15-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 b/sql/hive/src/test/resources/golden/combine3-16-6635f7f5c55557b06ad3acc321eaa739 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a b/sql/hive/src/test/resources/golden/combine3-17-8cb751103da7c909276db6bddb50ae6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b b/sql/hive/src/test/resources/golden/combine3-18-31fae7c6be75b97d475512bd75a58a0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f b/sql/hive/src/test/resources/golden/combine3-19-e30d6cd92dc5a7a86fb2b9154497b04f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine3-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd b/sql/hive/src/test/resources/golden/combine3-20-5bd4bb8b05f31b14bbc59287dff01ffd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 b/sql/hive/src/test/resources/golden/combine3-21-8ba8f8723c1530be062cefc2d9246e56 new file mode 100644 index 0000000000000..d57cb5369e219 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/combine3-22-11025483569617a9f014b5defd71e933 b/sql/hive/src/test/resources/golden/combine3-22-11025483569617a9f014b5defd71e933 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-23-4725c48df09565618cbffd05953a5f62 b/sql/hive/src/test/resources/golden/combine3-23-4725c48df09565618cbffd05953a5f62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine3-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine3-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/combine3-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/combine3-6-4725c48df09565618cbffd05953a5f62 b/sql/hive/src/test/resources/golden/combine3-6-4725c48df09565618cbffd05953a5f62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 b/sql/hive/src/test/resources/golden/combine3-7-53a5c509ebc9ee8458f27cc9bac46d00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b b/sql/hive/src/test/resources/golden/combine3-8-68399bc39f71ddc99ed09ed9d2fd897b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 b/sql/hive/src/test/resources/golden/combine3-9-b5703b76bbe99c41cbb63582a09a6e69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 b/sql/hive/src/test/resources/golden/compute_stats_binary-0-16dcd4810ff82419cf1ae914d1860f21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 b/sql/hive/src/test/resources/golden/compute_stats_binary-1-8e576a57fc67a7fa78ce1d8c8a63a043 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c b/sql/hive/src/test/resources/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_binary-2-c5a68f035051eef3e1c8d44d8b90017c @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 b/sql/hive/src/test/resources/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 new file mode 100644 index 0000000000000..fe2e2d7663026 --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_binary-3-bcac92c4a17678873b01779e3d0e84e3 @@ -0,0 +1 @@ +{"columntype":"Binary","maxlength":36,"avglength":20.0,"countnulls":0} diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 b/sql/hive/src/test/resources/golden/compute_stats_boolean-0-e39ee7d1e03efae1334a4cafc94af1d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 b/sql/hive/src/test/resources/golden/compute_stats_boolean-1-8300537a2a508b3390c3172cd69c69b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e b/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e new file mode 100644 index 0000000000000..dc7b54ad01435 --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_boolean-2-fbea367698de65e22d4d660a518ea95e @@ -0,0 +1 @@ +33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d b/sql/hive/src/test/resources/golden/compute_stats_boolean-3-a14d8a5835c94829271f9f463d96d83d new file mode 100644 index 0000000000000..dd487e6fea3ff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce b/sql/hive/src/test/resources/golden/compute_stats_double-0-76e8d4ba13c67a0834987b6dcd1d05ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 b/sql/hive/src/test/resources/golden/compute_stats_double-1-a23a25a680139ed823c77f3f9f486065 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 b/sql/hive/src/test/resources/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 new file mode 100644 index 0000000000000..b6a7d89c68e0c --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_double-2-8f988b757fd62f318f35447a3fd65452 @@ -0,0 +1 @@ +16 diff --git a/sql/hive/src/test/resources/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 b/sql/hive/src/test/resources/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 new file mode 100644 index 0000000000000..31a1d0792beeb --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_double-3-756f6b2d1c63fd5cb0bb25f76d5bfce2 @@ -0,0 +1 @@ +{"columntype":"Double","min":-87.2,"max":435.33,"countnulls":2,"numdistinctvalues":11} diff --git a/sql/hive/src/test/resources/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d b/sql/hive/src/test/resources/golden/compute_stats_empty_table-0-12161b12442ad9b664b51e443fabaf5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 b/sql/hive/src/test/resources/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_empty_table-1-1f720211105d9a6d611c5d378ee45ec0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f b/sql/hive/src/test/resources/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f new file mode 100644 index 0000000000000..8d4251bcc08c8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_empty_table-2-372df408a15de1e6f05e807a3aff223f @@ -0,0 +1 @@ +{"columntype":"Boolean","counttrues":0,"countfalses":0,"countnulls":0} diff --git a/sql/hive/src/test/resources/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b b/sql/hive/src/test/resources/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b new file mode 100644 index 0000000000000..ce543362c3ffc --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_empty_table-3-73f6626835884f34a47e8e78396bc25b @@ -0,0 +1 @@ +{"columntype":"Long","min":0,"max":0,"countnulls":0,"numdistinctvalues":0} diff --git a/sql/hive/src/test/resources/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf b/sql/hive/src/test/resources/golden/compute_stats_empty_table-4-7f356d57c8c6125e4083f51ed4bae5cf new file mode 100644 index 0000000000000..51062737dd0de --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 b/sql/hive/src/test/resources/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 new file mode 100644 index 0000000000000..9a834007cec70 --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_empty_table-5-294a33b7c457eb7846335a4b1775ddc4 @@ -0,0 +1 @@ +{"columntype":"String","maxlength":0,"avglength":0.0,"countnulls":0,"numdistinctvalues":0} diff --git a/sql/hive/src/test/resources/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 b/sql/hive/src/test/resources/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 new file mode 100644 index 0000000000000..f6c0cb997b2a6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_empty_table-6-137180ebd2a072f08b5d849bdd9a464 @@ -0,0 +1 @@ +{"columntype":"Binary","maxlength":0,"avglength":0.0,"countnulls":0} diff --git a/sql/hive/src/test/resources/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 b/sql/hive/src/test/resources/golden/compute_stats_long-0-2ee7f9e4c307417d4da2660e303c07c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 b/sql/hive/src/test/resources/golden/compute_stats_long-1-a7bc730f9862198709539e35c0208248 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 b/sql/hive/src/test/resources/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 new file mode 100644 index 0000000000000..48082f72f087c --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_long-2-71ffbc32647b3b562ccdc7e3db71e7a8 @@ -0,0 +1 @@ +12 diff --git a/sql/hive/src/test/resources/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d b/sql/hive/src/test/resources/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d new file mode 100644 index 0000000000000..cb4c03d1da303 --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_long-3-cf4bb755ade3079409e2251c7cd0118d @@ -0,0 +1 @@ +{"columntype":"Long","min":0,"max":344,"countnulls":1,"numdistinctvalues":11} diff --git a/sql/hive/src/test/resources/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b b/sql/hive/src/test/resources/golden/compute_stats_string-0-3b932d28809fb8f40d81b4f8dfe2693b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f b/sql/hive/src/test/resources/golden/compute_stats_string-1-3bddaed6594ed44249e4a30c43e83d1f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a b/sql/hive/src/test/resources/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_string-2-b87a68ae5ffa689bada75425169d131a @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc b/sql/hive/src/test/resources/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc new file mode 100644 index 0000000000000..a8fc88cbd539d --- /dev/null +++ b/sql/hive/src/test/resources/golden/compute_stats_string-3-cea908dd41c78490990ee6b681d19fc @@ -0,0 +1 @@ +{"columntype":"String","maxlength":11,"avglength":3.9,"countnulls":0,"numdistinctvalues":7} diff --git a/sql/hive/src/test/resources/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 b/sql/hive/src/test/resources/golden/convert_enum_to_string-0-f22bc1aaadc6f36ba36420073ea04543 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 b/sql/hive/src/test/resources/golden/convert_enum_to_string-1-db089ff46f9826c7883198adacdfad59 new file mode 100644 index 0000000000000..c615f0148ccd1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer11-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 b/sql/hive/src/test/resources/golden/correlationoptimizer11-1-3a7f180f26dd2aec3ceab769f0cd965 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer11-10-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 b/sql/hive/src/test/resources/golden/correlationoptimizer11-11-f7918ee4d4941d3272e0262a750de700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 b/sql/hive/src/test/resources/golden/correlationoptimizer11-12-5cefedc27b914d45b1512c92ad36c6e4 new file mode 100644 index 0000000000000..7c33b34887d6b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer11-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d b/sql/hive/src/test/resources/golden/correlationoptimizer11-14-c16dc98e6fc3e9ea52f7f3ca04ad953d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 b/sql/hive/src/test/resources/golden/correlationoptimizer11-15-5cefedc27b914d45b1512c92ad36c6e4 new file mode 100644 index 0000000000000..7c33b34887d6b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 b/sql/hive/src/test/resources/golden/correlationoptimizer11-2-e148026f8994e22ca756c68753a0cc26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 b/sql/hive/src/test/resources/golden/correlationoptimizer11-3-b04195464e014cb47fd20a76b5f9ac0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer11-4-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 b/sql/hive/src/test/resources/golden/correlationoptimizer11-5-88d8dfbff4269af34724bf30ff4fec34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 b/sql/hive/src/test/resources/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 new file mode 100644 index 0000000000000..b97104d25bb51 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer11-6-fce66bdc5987a642f8f93471e62c1748 @@ -0,0 +1,2 @@ +66 1 +98 2 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer11-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 b/sql/hive/src/test/resources/golden/correlationoptimizer11-8-45d4d690886288ef04addbb659397ad1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 b/sql/hive/src/test/resources/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 new file mode 100644 index 0000000000000..b97104d25bb51 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer11-9-fce66bdc5987a642f8f93471e62c1748 @@ -0,0 +1,2 @@ +66 1 +98 2 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 b/sql/hive/src/test/resources/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-0-aa047b3a8b40b68b93c4ad11e173c767 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-1-d0a93f40892e3894460553b443c77428 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda b/sql/hive/src/test/resources/golden/correlationoptimizer14-10-a837e66f8c37cc3b2f6d3596b03a6eda new file mode 100644 index 0000000000000..e8eb644933e82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-11-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 b/sql/hive/src/test/resources/golden/correlationoptimizer14-12-99f81dd0f33197c724eb58398542ff22 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 b/sql/hive/src/test/resources/golden/correlationoptimizer14-13-5e19a84c0c3ede17b8d9685a22f0a1e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-14-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf b/sql/hive/src/test/resources/golden/correlationoptimizer14-15-78fed7defb6154e01abbd97a0741adf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 b/sql/hive/src/test/resources/golden/correlationoptimizer14-16-7ebe26e8a3620830e824b4099519395 new file mode 100644 index 0000000000000..e8eb644933e82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-17-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 b/sql/hive/src/test/resources/golden/correlationoptimizer14-18-aa18035e1d8fdcedb91b76f9a32b11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 b/sql/hive/src/test/resources/golden/correlationoptimizer14-19-7ebe26e8a3620830e824b4099519395 new file mode 100644 index 0000000000000..e8eb644933e82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-2-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-20-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 b/sql/hive/src/test/resources/golden/correlationoptimizer14-21-e85444100b2e0c71b3d792e4bf1486d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 b/sql/hive/src/test/resources/golden/correlationoptimizer14-22-aeed9fe2c3ffdf99eba2821ecfa18242 new file mode 100644 index 0000000000000..0a6a1f70d34ef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-23-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c b/sql/hive/src/test/resources/golden/correlationoptimizer14-24-8e88fd43a2c216b3409bee768425772c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 b/sql/hive/src/test/resources/golden/correlationoptimizer14-25-aeed9fe2c3ffdf99eba2821ecfa18242 new file mode 100644 index 0000000000000..0a6a1f70d34ef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f b/sql/hive/src/test/resources/golden/correlationoptimizer14-3-88b3974a7639097ed915402827e8941f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f b/sql/hive/src/test/resources/golden/correlationoptimizer14-4-f58c909a1cbcbca3ea64bada41b0a18f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-5-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c b/sql/hive/src/test/resources/golden/correlationoptimizer14-6-dcc6819f5848ff3d68b1d28c8787d41c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda b/sql/hive/src/test/resources/golden/correlationoptimizer14-7-a837e66f8c37cc3b2f6d3596b03a6eda new file mode 100644 index 0000000000000..e8eb644933e82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer14-8-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 b/sql/hive/src/test/resources/golden/correlationoptimizer14-9-e5f8709d75fbe813609cbdc8ed707489 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 b/sql/hive/src/test/resources/golden/correlationoptimizer15-2-f3001b5ee3fe7b9b01c82b0c79c2df02 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-3-e149747103059314a9984235702b24b6 new file mode 100644 index 0000000000000..0b1ee37d7831c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer15-5-9914f44ecb6ae7587b62e5349ff60d04 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd b/sql/hive/src/test/resources/golden/correlationoptimizer15-6-3bc6f1aef3516fd7ba8c7527f2865dcd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer15-7-e149747103059314a9984235702b24b6 new file mode 100644 index 0000000000000..0b1ee37d7831c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab b/sql/hive/src/test/resources/golden/correlationoptimizer2-11-165752d0d250e5c9cddca50cf0c9cab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 b/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 new file mode 100644 index 0000000000000..6c6dc691bbff2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-12-e6b368bfaea4d2838f8038b3bd29db06 @@ -0,0 +1 @@ +79136 500 3556 15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-13-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd b/sql/hive/src/test/resources/golden/correlationoptimizer2-14-b8a58e660d7416485c3d9a97d610bdfd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 b/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 new file mode 100644 index 0000000000000..f4bb720dfd7f8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-15-18f10d12e8bfa473a916c2f528500538 @@ -0,0 +1 @@ +3556 37 3556 25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-16-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 b/sql/hive/src/test/resources/golden/correlationoptimizer2-17-2a3e19b28e5262eb8c467e237df34421 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 b/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 new file mode 100644 index 0000000000000..f4bb720dfd7f8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-18-18f10d12e8bfa473a916c2f528500538 @@ -0,0 +1 @@ +3556 37 3556 25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-19-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 b/sql/hive/src/test/resources/golden/correlationoptimizer2-2-354213872b92046d7aec97c9ba7b4e97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf b/sql/hive/src/test/resources/golden/correlationoptimizer2-20-6907d13fadd74e0df33c7a99c1de0baf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e b/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e new file mode 100644 index 0000000000000..4acbb60e81661 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-21-c0c5744805b82aa8a6e3a62dfdb8097e @@ -0,0 +1 @@ +79136 500 3556 25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-22-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b b/sql/hive/src/test/resources/golden/correlationoptimizer2-23-38b273c68d644e801695d5414c0e119b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e b/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e new file mode 100644 index 0000000000000..4acbb60e81661 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-24-c0c5744805b82aa8a6e3a62dfdb8097e @@ -0,0 +1 @@ +79136 500 3556 25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-25-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 b/sql/hive/src/test/resources/golden/correlationoptimizer2-26-12408fe0bc5e6a22e838ee5ccef144d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 b/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 new file mode 100644 index 0000000000000..a95fce80fd7b4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-27-9b22dad2843cdc379d90687745561104 @@ -0,0 +1 @@ +79136 310 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab b/sql/hive/src/test/resources/golden/correlationoptimizer2-29-3781a20b72437434ef8fa7174edf36ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d b/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d new file mode 100644 index 0000000000000..c6243d7056353 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-3-d915fbdd493869aec42f548bdb66598d @@ -0,0 +1 @@ +3556 37 3556 15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 b/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 new file mode 100644 index 0000000000000..a95fce80fd7b4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-30-9b22dad2843cdc379d90687745561104 @@ -0,0 +1 @@ +79136 310 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-31-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f b/sql/hive/src/test/resources/golden/correlationoptimizer2-32-5ac93f83acfd31ce036381993eda303f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b b/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b new file mode 100644 index 0000000000000..556b77ecfc9eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-33-2b9eddc3452d8fc24ae9273e5d522e4b @@ -0,0 +1 @@ +9992 3531902962 9992 37 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 b/sql/hive/src/test/resources/golden/correlationoptimizer2-35-996af7ef09a07d38d1a238b00c80da03 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b b/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b new file mode 100644 index 0000000000000..556b77ecfc9eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-36-2b9eddc3452d8fc24ae9273e5d522e4b @@ -0,0 +1 @@ +9992 3531902962 9992 37 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 b/sql/hive/src/test/resources/golden/correlationoptimizer2-5-6eeb3aa27037ecf59c79c4252cb31f28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d b/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d new file mode 100644 index 0000000000000..c6243d7056353 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-6-d915fbdd493869aec42f548bdb66598d @@ -0,0 +1 @@ +3556 37 3556 15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-7-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 b/sql/hive/src/test/resources/golden/correlationoptimizer2-8-74078f19dfe424f3211e6ce26de52152 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 b/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 new file mode 100644 index 0000000000000..6c6dc691bbff2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer2-9-e6b368bfaea4d2838f8038b3bd29db06 @@ -0,0 +1 @@ +79136 500 3556 15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-10-e3d5ff08760b877d49c0f10f63776325 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-11-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-12-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 b/sql/hive/src/test/resources/golden/correlationoptimizer3-13-11a7c4a1256b5f56dbf8f2722e24831 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-14-1f13e03988991067d13a9f3f1b36fcf5 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 b/sql/hive/src/test/resources/golden/correlationoptimizer3-16-11a7c4a1256b5f56dbf8f2722e24831 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-17-1f13e03988991067d13a9f3f1b36fcf5 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-18-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-19-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace b/sql/hive/src/test/resources/golden/correlationoptimizer3-2-9a7769de1ce7fc0b0fc46f17da287ace new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe b/sql/hive/src/test/resources/golden/correlationoptimizer3-20-4025759f01fa2169a061070319ee8bfe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 b/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-21-1f13e03988991067d13a9f3f1b36fcf5 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-3-e3d5ff08760b877d49c0f10f63776325 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 b/sql/hive/src/test/resources/golden/correlationoptimizer3-5-3c5cec9256d6f9d24a64b36fe6256f91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 b/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 new file mode 100644 index 0000000000000..a1a6cbb91955e --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-6-e3d5ff08760b877d49c0f10f63776325 @@ -0,0 +1 @@ +9992 107 3531902962 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer3-8-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea b/sql/hive/src/test/resources/golden/correlationoptimizer3-9-c575bf5ba408caadb836d307b9971bea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 b/sql/hive/src/test/resources/golden/correlationoptimizer4-0-d157f058f9e8659c4367e01c4da13579 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/correlationoptimizer4-1-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e b/sql/hive/src/test/resources/golden/correlationoptimizer4-11-fb7cadb1f06690537178b2a04b1ee91e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e new file mode 100644 index 0000000000000..14e309fdcad89 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-12-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -0,0 +1 @@ +13 10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-14-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 b/sql/hive/src/test/resources/golden/correlationoptimizer4-15-8db0d44941d0ce086e95088ef579c136 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e new file mode 100644 index 0000000000000..14e309fdcad89 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-16-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -0,0 +1 @@ +13 10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-17-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-18-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb b/sql/hive/src/test/resources/golden/correlationoptimizer4-19-5e241577196c76217ed9615fcbc76dcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 b/sql/hive/src/test/resources/golden/correlationoptimizer4-2-46c5eef67c57677810028451dd2b4d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 b/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 new file mode 100644 index 0000000000000..1b8c137073e37 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-20-6ad79a473a876923a247f0cb57bb4208 @@ -0,0 +1 @@ +22 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-21-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 b/sql/hive/src/test/resources/golden/correlationoptimizer4-22-392b7eecdc93567cd0621e0efafbbaa8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 b/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 new file mode 100644 index 0000000000000..1b8c137073e37 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-23-6ad79a473a876923a247f0cb57bb4208 @@ -0,0 +1 @@ +22 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-24-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf b/sql/hive/src/test/resources/golden/correlationoptimizer4-25-18ec265534266497e1da68480cfe51cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b b/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b new file mode 100644 index 0000000000000..e6c95e600a2c0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-26-62a0fd05be48759c39f3c284458dde9b @@ -0,0 +1 @@ +13 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-27-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb b/sql/hive/src/test/resources/golden/correlationoptimizer4-28-f14be583b8c69df924ac3bc4cf1761fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf b/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf new file mode 100644 index 0000000000000..0248cc90cb2c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-29-6fa624d24bcc899e11dbe8b19b0d5cbf @@ -0,0 +1 @@ +21 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/correlationoptimizer4-3-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-30-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 b/sql/hive/src/test/resources/golden/correlationoptimizer4-31-7fea74857587a30456b095e20bc2bde1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf b/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf new file mode 100644 index 0000000000000..0248cc90cb2c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-32-6fa624d24bcc899e11dbe8b19b0d5cbf @@ -0,0 +1 @@ +21 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-33-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 b/sql/hive/src/test/resources/golden/correlationoptimizer4-34-fa4c05d7d8707d20d89338d744f51dd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 b/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 new file mode 100644 index 0000000000000..0248cc90cb2c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-35-64d8e7807af6fc5a0214675a7c6a3be8 @@ -0,0 +1 @@ +21 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-36-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 b/sql/hive/src/test/resources/golden/correlationoptimizer4-37-3cd84bc4d9e25a49dee3d1ab09f002c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 b/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 new file mode 100644 index 0000000000000..fd3a42ebe67e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-38-85fff71b240d0e26bab91d670c1349f2 @@ -0,0 +1 @@ +21 14 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-39-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 b/sql/hive/src/test/resources/golden/correlationoptimizer4-4-ee940bcfd73a883d16245ef746798e15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 b/sql/hive/src/test/resources/golden/correlationoptimizer4-40-ef5268865a18f57e15dc650a11527b11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 b/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 new file mode 100644 index 0000000000000..fd3a42ebe67e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-41-85fff71b240d0e26bab91d670c1349f2 @@ -0,0 +1 @@ +21 14 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 b/sql/hive/src/test/resources/golden/correlationoptimizer4-5-c44e5ccbff08c59a5dc1b74306835cd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-6-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-7-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 b/sql/hive/src/test/resources/golden/correlationoptimizer4-8-d926d0a2ecc3b2b7094cc7a094cb3d89 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e b/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e new file mode 100644 index 0000000000000..14e309fdcad89 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer4-9-340cf26bcac4ee29bdf8fd588ddc3c2e @@ -0,0 +1 @@ +13 10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-10-25b1fe48ef86952fc7766fb74b63bf21 new file mode 100644 index 0000000000000..19b8a2aea8f64 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-11-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-12-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 b/sql/hive/src/test/resources/golden/correlationoptimizer6-13-f16444a052505377b7747949ffc2c028 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db b/sql/hive/src/test/resources/golden/correlationoptimizer6-14-84463190baec77d61b287a071c8886db new file mode 100644 index 0000000000000..c4a418f59625b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 b/sql/hive/src/test/resources/golden/correlationoptimizer6-16-1cce9d13fab4b27d36c0fe705d239f68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db b/sql/hive/src/test/resources/golden/correlationoptimizer6-17-84463190baec77d61b287a071c8886db new file mode 100644 index 0000000000000..c4a418f59625b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-18-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 b/sql/hive/src/test/resources/golden/correlationoptimizer6-19-e7b8cde1cfc381901066753ecfa87ed4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 b/sql/hive/src/test/resources/golden/correlationoptimizer6-2-f196f01fc1d04f0e88881eb51e3b43c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/sql/hive/src/test/resources/golden/correlationoptimizer6-20-4b0a2d0d62b76bbd8a44ebed4a9cc4de new file mode 100644 index 0000000000000..9c8189500649e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-21-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 b/sql/hive/src/test/resources/golden/correlationoptimizer6-22-a5e1c26011ddfbe79a886044dc17ea67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de b/sql/hive/src/test/resources/golden/correlationoptimizer6-23-4b0a2d0d62b76bbd8a44ebed4a9cc4de new file mode 100644 index 0000000000000..9c8189500649e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-24-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-25-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 b/sql/hive/src/test/resources/golden/correlationoptimizer6-26-f3001b5ee3fe7b9b01c82b0c79c2df02 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-27-e149747103059314a9984235702b24b6 new file mode 100644 index 0000000000000..0b1ee37d7831c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-28-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd b/sql/hive/src/test/resources/golden/correlationoptimizer6-29-3bc6f1aef3516fd7ba8c7527f2865dcd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-3-25b1fe48ef86952fc7766fb74b63bf21 new file mode 100644 index 0000000000000..19b8a2aea8f64 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-30-e149747103059314a9984235702b24b6 new file mode 100644 index 0000000000000..0b1ee37d7831c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-31-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 b/sql/hive/src/test/resources/golden/correlationoptimizer6-32-17c94297caaed15ea3ea2ea72d3508b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 b/sql/hive/src/test/resources/golden/correlationoptimizer6-33-15d991127dc684513e2fff1aea3f1560 new file mode 100644 index 0000000000000..7c591d6146fd6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-34-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 b/sql/hive/src/test/resources/golden/correlationoptimizer6-35-833150221aa5d9fbb0cb626bd8ce0762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 b/sql/hive/src/test/resources/golden/correlationoptimizer6-36-15d991127dc684513e2fff1aea3f1560 new file mode 100644 index 0000000000000..7c591d6146fd6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-37-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb b/sql/hive/src/test/resources/golden/correlationoptimizer6-38-d1a903645384c97759e835649ce898fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 b/sql/hive/src/test/resources/golden/correlationoptimizer6-39-813d72763b5e9e3bed0f06232f55c8b8 new file mode 100644 index 0000000000000..7c591d6146fd6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-40-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-41-cea4eb2e1299a57e4732adc1439e7bb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 b/sql/hive/src/test/resources/golden/correlationoptimizer6-42-813d72763b5e9e3bed0f06232f55c8b8 new file mode 100644 index 0000000000000..7c591d6146fd6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-43-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb b/sql/hive/src/test/resources/golden/correlationoptimizer6-44-ed339f429b82397bffb7298a534d59fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 b/sql/hive/src/test/resources/golden/correlationoptimizer6-45-ff6c54b483726ef15c90a4c68dc659a0 new file mode 100644 index 0000000000000..8338433cd5a27 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-46-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 b/sql/hive/src/test/resources/golden/correlationoptimizer6-47-53940f5dd35ccbe9086533076c362bd4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 b/sql/hive/src/test/resources/golden/correlationoptimizer6-48-ff6c54b483726ef15c90a4c68dc659a0 new file mode 100644 index 0000000000000..8338433cd5a27 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-49-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 b/sql/hive/src/test/resources/golden/correlationoptimizer6-5-c58f05e3e6f63bf3af7cad564063e9d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 b/sql/hive/src/test/resources/golden/correlationoptimizer6-50-ade42af96a1436c5c4a3c86e3bc1a9f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-51-4746d944f4193018017984ca2df3c60d new file mode 100644 index 0000000000000..8b1d3cd388fa2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-52-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 b/sql/hive/src/test/resources/golden/correlationoptimizer6-53-b9552c6399ce578cde69e663dd43d870 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-54-4746d944f4193018017984ca2df3c60d new file mode 100644 index 0000000000000..8b1d3cd388fa2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-55-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-56-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 b/sql/hive/src/test/resources/golden/correlationoptimizer6-57-b9552c6399ce578cde69e663dd43d870 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d b/sql/hive/src/test/resources/golden/correlationoptimizer6-58-4746d944f4193018017984ca2df3c60d new file mode 100644 index 0000000000000..8b1d3cd388fa2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 b/sql/hive/src/test/resources/golden/correlationoptimizer6-6-25b1fe48ef86952fc7766fb74b63bf21 new file mode 100644 index 0000000000000..19b8a2aea8f64 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-7-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer6-8-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae b/sql/hive/src/test/resources/golden/correlationoptimizer6-9-fd372b979ca4b12d64c65cb24e7d82ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-10-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 b/sql/hive/src/test/resources/golden/correlationoptimizer7-11-fc438bb849eff3496559a916c7dee058 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-12-22d71fb589c53776dabb4696b38c4a42 new file mode 100644 index 0000000000000..747b650237b53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 b/sql/hive/src/test/resources/golden/correlationoptimizer7-14-5b2d5594522db2b4bbc6aeb8e33b68d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-15-22d71fb589c53776dabb4696b38c4a42 new file mode 100644 index 0000000000000..747b650237b53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 b/sql/hive/src/test/resources/golden/correlationoptimizer7-2-5b2d5594522db2b4bbc6aeb8e33b68d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-3-22d71fb589c53776dabb4696b38c4a42 new file mode 100644 index 0000000000000..747b650237b53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 b/sql/hive/src/test/resources/golden/correlationoptimizer7-5-5b2d5594522db2b4bbc6aeb8e33b68d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 b/sql/hive/src/test/resources/golden/correlationoptimizer7-6-22d71fb589c53776dabb4696b38c4a42 new file mode 100644 index 0000000000000..747b650237b53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-7-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-8-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d b/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer7-9-4d3e60a0e9bd8c12fdba4e010493537d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-0-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-1-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-10-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d b/sql/hive/src/test/resources/golden/correlationoptimizer8-11-45693930a19f7e1b86202a5f5f2d139d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f b/sql/hive/src/test/resources/golden/correlationoptimizer8-12-c1dfbe5cf77a97d195a3d0a65d1f1f new file mode 100644 index 0000000000000..7c0af7229d62a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-13-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea b/sql/hive/src/test/resources/golden/correlationoptimizer8-14-2b8991eb2f59bae3250c8d379b6d6aea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-15-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc b/sql/hive/src/test/resources/golden/correlationoptimizer8-16-8c9264ae28e72f3724a78412f26ddbcc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a b/sql/hive/src/test/resources/golden/correlationoptimizer8-2-34cad62f8ca6f6b53cf593960055c56a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 b/sql/hive/src/test/resources/golden/correlationoptimizer8-3-932db4b9935e4bc3d21dd33a8d12c275 new file mode 100644 index 0000000000000..efdd80c9f8b89 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-4-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 b/sql/hive/src/test/resources/golden/correlationoptimizer8-5-f5fa5cecdf6f7359e7d727716a7e5b78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 b/sql/hive/src/test/resources/golden/correlationoptimizer8-6-932db4b9935e4bc3d21dd33a8d12c275 new file mode 100644 index 0000000000000..efdd80c9f8b89 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/correlationoptimizer8-7-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 b/sql/hive/src/test/resources/golden/correlationoptimizer8-8-39a62b29e0ac91861c58c56fc654ba67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f b/sql/hive/src/test/resources/golden/correlationoptimizer8-9-c1dfbe5cf77a97d195a3d0a65d1f1f new file mode 100644 index 0000000000000..7c0af7229d62a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b b/sql/hive/src/test/resources/golden/count-0-dd24f2cc60a0cd0eba0a3f8ca8127a0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 b/sql/hive/src/test/resources/golden/count-1-3531872d964bc2b4f07d51b9d1629df2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 b/sql/hive/src/test/resources/golden/count-10-d0f804c7d06375db74a0fcf5f17603c6 new file mode 100644 index 0000000000000..4b020e0595d2c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/count-11-29aa086fe75b55482a91316c38498565 b/sql/hive/src/test/resources/golden/count-11-29aa086fe75b55482a91316c38498565 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 b/sql/hive/src/test/resources/golden/count-12-944f53db544c07a7b38a0544a21d8e13 new file mode 100644 index 0000000000000..54199fdb8166d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 b/sql/hive/src/test/resources/golden/count-2-461bad3feb7dbc25fb35d45c6876d698 new file mode 100644 index 0000000000000..2e9278da88858 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/count-3-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/count-4-590bf60b8d4dfa135f73dbb52180136f b/sql/hive/src/test/resources/golden/count-4-590bf60b8d4dfa135f73dbb52180136f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 b/sql/hive/src/test/resources/golden/count-5-d0f804c7d06375db74a0fcf5f17603c6 new file mode 100644 index 0000000000000..4b020e0595d2c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/count-6-29aa086fe75b55482a91316c38498565 b/sql/hive/src/test/resources/golden/count-6-29aa086fe75b55482a91316c38498565 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 b/sql/hive/src/test/resources/golden/count-7-944f53db544c07a7b38a0544a21d8e13 new file mode 100644 index 0000000000000..54199fdb8166d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/count-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/count-9-590bf60b8d4dfa135f73dbb52180136f b/sql/hive/src/test/resources/golden/count-9-590bf60b8d4dfa135f73dbb52180136f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 b/sql/hive/src/test/resources/golden/cp_mj_rc-0-645f784b04a91b8c8e5bb770f95c3766 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 b/sql/hive/src/test/resources/golden/cp_mj_rc-1-128c219e87f95f0425c8545a8c4bb32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e b/sql/hive/src/test/resources/golden/cp_mj_rc-2-17bbe2a542acac5960b755979dcb6d5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 b/sql/hive/src/test/resources/golden/cp_mj_rc-3-b1746cc42d6bbd2b1d314a9e5b18eb60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 b/sql/hive/src/test/resources/golden/cp_mj_rc-4-15beca26e35c885dff3bfe2336c26673 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 b/sql/hive/src/test/resources/golden/cp_mj_rc-5-27550c25058249ebfc6cda30dcf513f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac b/sql/hive/src/test/resources/golden/create table as with db name-0-a253b1ed35dbf503d1b8902dacbe23ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 b/sql/hive/src/test/resources/golden/create table as with db name-0-b7dfeb6a941b42f7def5fdceae99f425 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f b/sql/hive/src/test/resources/golden/create table as with db name-1-417609d2bb67ba26de38e92ad834008f new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 b/sql/hive/src/test/resources/golden/create table as with db name-1-febc8588a2a54ed5b00e47cb83dd85c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 b/sql/hive/src/test/resources/golden/create table as with db name-2-ce780d068b8d24786e639e361101a0c7 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c b/sql/hive/src/test/resources/golden/create table as with db name-3-afd6e46b6a289c3c24a8eec75a94043c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd b/sql/hive/src/test/resources/golden/create table as-0-fd42e3ffae73415391acb1012a3531bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 b/sql/hive/src/test/resources/golden/create table as-1-b9002c1d71895be765575b62656d1928 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_1-0-d57ed4bbfee1ffaffaeba0a4be84c31d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 b/sql/hive/src/test/resources/golden/create_1-1-c2351f011b8ea41ff7dfa8f195148da3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba b/sql/hive/src/test/resources/golden/create_1-10-b9c0b95624e601614ea2561b83aaf0ba new file mode 100644 index 0000000000000..66d40e52a4539 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 b/sql/hive/src/test/resources/golden/create_1-11-7daaeabd9c286e511e0628a32dc714d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 b/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 new file mode 100644 index 0000000000000..8e606f61a1c30 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_1-12-fe8680bc5ef6fe3e64c68d2638e10785 @@ -0,0 +1,2 @@ +a string None +b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a b/sql/hive/src/test/resources/golden/create_1-13-5715f85456733a71fb7c83f1b254b3a new file mode 100644 index 0000000000000..05b460270525d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_1-14-437bd1acbae61f48e851addb769d3379 b/sql/hive/src/test/resources/golden/create_1-14-437bd1acbae61f48e851addb769d3379 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 b/sql/hive/src/test/resources/golden/create_1-15-fd9415e340cf54a9473cc4bf86673816 new file mode 100644 index 0000000000000..eda3946588e3f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 b/sql/hive/src/test/resources/golden/create_1-16-c99c700ca511b68577aae8ae513a4a32 new file mode 100644 index 0000000000000..8fc60adf10167 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 new file mode 100644 index 0000000000000..8e606f61a1c30 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_1-2-ecd02bc3563cd6b60b8394956cb69084 @@ -0,0 +1,2 @@ +a string None +b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_1-3-c27702ff131e0ecfd71f1e1779fbe365 new file mode 100644 index 0000000000000..5e9bc70c3a1a3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_1-4-610b82bf7b0080d293977927e5ef780c b/sql/hive/src/test/resources/golden/create_1-4-610b82bf7b0080d293977927e5ef780c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_1-5-c77b018276b1558c1d9462e0625e152e b/sql/hive/src/test/resources/golden/create_1-5-c77b018276b1558c1d9462e0625e152e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a b/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a new file mode 100644 index 0000000000000..6ed3515ba6876 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_1-6-52dc9f900d7f7a559698aff9565f061a @@ -0,0 +1,2 @@ +a string None +b int None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 b/sql/hive/src/test/resources/golden/create_1-7-8564aa6cc2b0ee85292b3e8976fe9001 new file mode 100644 index 0000000000000..bec4c72e49857 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 b/sql/hive/src/test/resources/golden/create_1-8-19331fe6a2a35f1171babfe4e1c86f59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 b/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 new file mode 100644 index 0000000000000..8e606f61a1c30 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_1-9-f19e6b501d5d2fb284777c71f8f6c0c3 @@ -0,0 +1,2 @@ +a string None +b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d b/sql/hive/src/test/resources/golden/create_big_view-0-bdf3c2e20793ef833f336a40791091d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee b/sql/hive/src/test/resources/golden/create_big_view-1-fbd4b50d4d80d2a927a0db5037c62bee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e b/sql/hive/src/test/resources/golden/create_escape-0-3e860648a3f2f258d59a62fd0bbe1c3e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 new file mode 100644 index 0000000000000..8e606f61a1c30 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_escape-1-ecd02bc3563cd6b60b8394956cb69084 @@ -0,0 +1,2 @@ +a string None +b string None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_escape-2-c27702ff131e0ecfd71f1e1779fbe365 new file mode 100644 index 0000000000000..8ab6b24deab60 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 b/sql/hive/src/test/resources/golden/create_escape-3-9541399cde42210bd7ac1beb07ceb14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 b/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 new file mode 100644 index 0000000000000..bbe37f8e2a790 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_escape-4-7d84873a6ce03e0e408fa67ef5dd733 @@ -0,0 +1 @@ +86 \ \ \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 b/sql/hive/src/test/resources/golden/create_genericudaf-0-499d36ed8229cbf74a07b59914bdf717 new file mode 100644 index 0000000000000..cbcdfbe72e8c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b b/sql/hive/src/test/resources/golden/create_genericudaf-1-c7f934e9c76350a0d3caa694463a673b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 b/sql/hive/src/test/resources/golden/create_genericudaf-2-84f898ffd668a17fe2ef081866d1dcd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 b/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 new file mode 100644 index 0000000000000..2f958c483a9df --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_genericudaf-3-d3b1af3e7f037de800255d9411a690e8 @@ -0,0 +1 @@ +1.0 260.182 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 b/sql/hive/src/test/resources/golden/create_genericudaf-4-c64a3266b9a1c3383cc56bd883345c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 b/sql/hive/src/test/resources/golden/create_genericudf-0-dd23fb149bb6d6937b838334559ad8d1 new file mode 100644 index 0000000000000..0e9c06c882602 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a b/sql/hive/src/test/resources/golden/create_insert_outputformat-0-16167c581df48112004009fef228e29a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 b/sql/hive/src/test/resources/golden/create_insert_outputformat-1-1246ba69e870178971f5ae062641cf47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc b/sql/hive/src/test/resources/golden/create_insert_outputformat-2-8c2e4081b9d69297c35faab0a2ff86dc new file mode 100644 index 0000000000000..83f572c424926 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 b/sql/hive/src/test/resources/golden/create_insert_outputformat-3-cc4e4ae8e5be09017c6206b8ae5088c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 b/sql/hive/src/test/resources/golden/create_insert_outputformat-4-2453e9c2ed5c7c517dce66a20e402a57 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c b/sql/hive/src/test/resources/golden/create_insert_outputformat-5-8552731917a8260c25e6df79b83bf5c new file mode 100644 index 0000000000000..83f572c424926 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 b/sql/hive/src/test/resources/golden/create_insert_outputformat-6-2f52c0c383ad83fae1620a0db9f6c863 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e b/sql/hive/src/test/resources/golden/create_insert_outputformat-7-a755c7e39694261510421e262b5005e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 b/sql/hive/src/test/resources/golden/create_insert_outputformat-8-33a4312fd617c5c9f2a560dc6ce868a5 new file mode 100644 index 0000000000000..83f572c424926 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 b/sql/hive/src/test/resources/golden/create_like-0-c2351f011b8ea41ff7dfa8f195148da3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af b/sql/hive/src/test/resources/golden/create_like-1-30485a2507b60b96cad3d293527e6af new file mode 100644 index 0000000000000..b8ddba7f50b97 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 b/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 new file mode 100644 index 0000000000000..e87fb81a6f043 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like-10-7d84873a6ce03e0e408fa67ef5dd733 @@ -0,0 +1 @@ +86 val_86 diff --git a/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 b/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 new file mode 100644 index 0000000000000..375dcacb8db71 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like-11-ba64f0122b21f605d8b2928753784130 @@ -0,0 +1,2 @@ +100 val_100 +100 val_100 diff --git a/sql/hive/src/test/resources/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a b/sql/hive/src/test/resources/golden/create_like-2-a20451f152e68606cc5e373fe5fd86a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 b/sql/hive/src/test/resources/golden/create_like-3-eea111a209cf8895f31f64699669c705 new file mode 100644 index 0000000000000..52b637c27f98d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 b/sql/hive/src/test/resources/golden/create_like-4-39ead53334938635b60a5ffdaa2c9f86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like-5-dc9de26002604e9e436135bd4b40636d b/sql/hive/src/test/resources/golden/create_like-5-dc9de26002604e9e436135bd4b40636d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 b/sql/hive/src/test/resources/golden/create_like-6-83eb00d0ac6d26d398ed5f9ddb1d3919 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 b/sql/hive/src/test/resources/golden/create_like-7-231c8b6709a8dc0b6a3b3a9751191cd7 new file mode 100644 index 0000000000000..d9308798dcb1c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb b/sql/hive/src/test/resources/golden/create_like-8-639a13da6855b974fc2e170fd49b33cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb b/sql/hive/src/test/resources/golden/create_like-9-a0ce573e299b66b8ce31da2890b318cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b b/sql/hive/src/test/resources/golden/create_like_view-0-3b48eae3848493703396156bedb1e98b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a b/sql/hive/src/test/resources/golden/create_like_view-1-3c805fc10db9af83327e04d518f3753a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 b/sql/hive/src/test/resources/golden/create_like_view-10-eea111a209cf8895f31f64699669c705 new file mode 100644 index 0000000000000..3e2a912824ab3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 b/sql/hive/src/test/resources/golden/create_like_view-11-62e0dd08ff9214aa999d9a2f30704fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 b/sql/hive/src/test/resources/golden/create_like_view-12-cb343b6b463c2e9b5735fbdf82d24811 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 b/sql/hive/src/test/resources/golden/create_like_view-13-fd80fbff7a622abe6b25c9fff7c5d608 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 new file mode 100644 index 0000000000000..115d12fb70c81 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_view-14-f19e6b501d5d2fb284777c71f8f6c0c3 @@ -0,0 +1,2 @@ +a string None +b string None diff --git a/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 b/sql/hive/src/test/resources/golden/create_like_view-15-231c8b6709a8dc0b6a3b3a9751191cd7 new file mode 100644 index 0000000000000..61d34badb1a2d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb b/sql/hive/src/test/resources/golden/create_like_view-16-639a13da6855b974fc2e170fd49b33cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb b/sql/hive/src/test/resources/golden/create_like_view-17-a0ce573e299b66b8ce31da2890b318cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c b/sql/hive/src/test/resources/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c new file mode 100644 index 0000000000000..e87fb81a6f043 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_view-18-d77d78569d86802f7c097d3d02150c @@ -0,0 +1 @@ +86 val_86 diff --git a/sql/hive/src/test/resources/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f b/sql/hive/src/test/resources/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f new file mode 100644 index 0000000000000..375dcacb8db71 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_view-19-deabf2d92205da2cbce9bdff854a81f @@ -0,0 +1,2 @@ +100 val_100 +100 val_100 diff --git a/sql/hive/src/test/resources/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b b/sql/hive/src/test/resources/golden/create_like_view-2-78e7a2bb4c10776cbdfb37c73d92ad6b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 b/sql/hive/src/test/resources/golden/create_like_view-20-152e6a694276b2fc1bb4fe6b4543fef0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 b/sql/hive/src/test/resources/golden/create_like_view-21-f982cac17966dcb9d76c1b6b9d4a7914 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/sql/hive/src/test/resources/golden/create_like_view-22-26e7fe8b9b9769a8d6a8a95b9cfbdf91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b b/sql/hive/src/test/resources/golden/create_like_view-23-44c601e364e42a83babab4a342dfbd2b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 b/sql/hive/src/test/resources/golden/create_like_view-24-df2d18dbae578430bfc7b9d27d201505 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d b/sql/hive/src/test/resources/golden/create_like_view-25-87a663f8fd80110a9cee249535037c0d new file mode 100644 index 0000000000000..4c2967215fe66 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 b/sql/hive/src/test/resources/golden/create_like_view-26-152e6a694276b2fc1bb4fe6b4543fef0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/sql/hive/src/test/resources/golden/create_like_view-27-26e7fe8b9b9769a8d6a8a95b9cfbdf91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f b/sql/hive/src/test/resources/golden/create_like_view-3-559f17e8f827532749948b3b9e6c0f3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 b/sql/hive/src/test/resources/golden/create_like_view-4-c2351f011b8ea41ff7dfa8f195148da3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 new file mode 100644 index 0000000000000..115d12fb70c81 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_view-5-ecd02bc3563cd6b60b8394956cb69084 @@ -0,0 +1,2 @@ +a string None +b string None diff --git a/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af b/sql/hive/src/test/resources/golden/create_like_view-6-30485a2507b60b96cad3d293527e6af new file mode 100644 index 0000000000000..02dee147bca42 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 b/sql/hive/src/test/resources/golden/create_like_view-7-fcc954b645304d258611f21d3aed7b76 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 b/sql/hive/src/test/resources/golden/create_like_view-8-304a79a8a321b84aee91f907f756a7e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a new file mode 100644 index 0000000000000..115d12fb70c81 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_like_view-9-52dc9f900d7f7a559698aff9565f061a @@ -0,0 +1,2 @@ +a string None +b string None diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c b/sql/hive/src/test/resources/golden/create_merge_compressed-0-366a4de0343396b9df03277f1098722c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 b/sql/hive/src/test/resources/golden/create_merge_compressed-1-276fbe6fb296b13904516888ffa95342 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 b/sql/hive/src/test/resources/golden/create_merge_compressed-10-d6fee0f05fa9b04cb7c557862402c929 new file mode 100644 index 0000000000000..dbdf4585360ae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d b/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d new file mode 100644 index 0000000000000..9a037142aa3c1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_merge_compressed-11-614c34f9e88015f21bffc4b8930bc95d @@ -0,0 +1 @@ +10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 b/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 new file mode 100644 index 0000000000000..eb6c3f6aef813 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_merge_compressed-12-4d89cbe49f710527b54e6262472f0320 @@ -0,0 +1 @@ +508 -751895388 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f b/sql/hive/src/test/resources/golden/create_merge_compressed-13-440c6f8daa221613fe796d99b494e61f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 b/sql/hive/src/test/resources/golden/create_merge_compressed-14-32251c08304629a3153e0b471de060c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 b/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_merge_compressed-2-84b74227c9f1563f530cd3ac3b333e54 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 b/sql/hive/src/test/resources/golden/create_merge_compressed-3-d7cc5e5c2cc9e09e715afcf8a0fd34c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 b/sql/hive/src/test/resources/golden/create_merge_compressed-4-c2a7e48e37375fc59126d71b9965b6c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 b/sql/hive/src/test/resources/golden/create_merge_compressed-5-c2a7e48e37375fc59126d71b9965b6c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 b/sql/hive/src/test/resources/golden/create_merge_compressed-6-d6fee0f05fa9b04cb7c557862402c929 new file mode 100644 index 0000000000000..8a9d9d56a66d6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d b/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d new file mode 100644 index 0000000000000..9a037142aa3c1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_merge_compressed-7-614c34f9e88015f21bffc4b8930bc95d @@ -0,0 +1 @@ +10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 b/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 new file mode 100644 index 0000000000000..eb6c3f6aef813 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_merge_compressed-8-4d89cbe49f710527b54e6262472f0320 @@ -0,0 +1 @@ +508 -751895388 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 b/sql/hive/src/test/resources/golden/create_merge_compressed-9-16a420c1def17d14881bd75d35a4c0e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 b/sql/hive/src/test/resources/golden/create_nested_type-0-17320fbe4af5d2c6bf2d52425f70f968 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 b/sql/hive/src/test/resources/golden/create_nested_type-1-ecd02bc3563cd6b60b8394956cb69084 new file mode 100644 index 0000000000000..c49ad26c04d67 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 b/sql/hive/src/test/resources/golden/create_nested_type-2-c27702ff131e0ecfd71f1e1779fbe365 new file mode 100644 index 0000000000000..41c7202c8b886 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_nested_type-3-ac452c9279877935983c37113898e53c b/sql/hive/src/test/resources/golden/create_nested_type-3-ac452c9279877935983c37113898e53c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff b/sql/hive/src/test/resources/golden/create_nested_type-4-c26dff65f60f7c0c70183ce22f4529ff new file mode 100644 index 0000000000000..45bb8c610c677 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/create_or_replace_view-0-a14cfe3eff322066e61023ec06c7735d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea b/sql/hive/src/test/resources/golden/create_or_replace_view-1-a27131eb04bd5e071d3267c92d3f8dea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 b/sql/hive/src/test/resources/golden/create_or_replace_view-2-30dc3e80e3873af5115e4f5e39078a13 new file mode 100644 index 0000000000000..46869fc9e88eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d b/sql/hive/src/test/resources/golden/create_or_replace_view-3-5fd147edbe44a96782923a3ef6caa47d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de b/sql/hive/src/test/resources/golden/create_or_replace_view-4-b1880014afc9ad1f8db91ba3db3867de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb b/sql/hive/src/test/resources/golden/create_or_replace_view-5-b1d2deb8a13dde4bf18c9b34836e00fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_skewed_table1-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 b/sql/hive/src/test/resources/golden/create_skewed_table1-1-cc66bd64f1cdc97b953e20860305370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e b/sql/hive/src/test/resources/golden/create_skewed_table1-2-32a6d8b77b06bbd1f236d6c9d9a0c75e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 b/sql/hive/src/test/resources/golden/create_skewed_table1-3-4f1d4f68bacf5fc6af33ef9e5e89e159 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 b/sql/hive/src/test/resources/golden/create_skewed_table1-4-f3f1642674545762a4bff5cb75634e20 new file mode 100644 index 0000000000000..d7c386e17c096 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e b/sql/hive/src/test/resources/golden/create_skewed_table1-5-f5e6d62497ae174fdfeeb3fd6f899b2e new file mode 100644 index 0000000000000..0817efac83d79 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef b/sql/hive/src/test/resources/golden/create_skewed_table1-6-d7a147c6b0a3609663628b43457b2cef new file mode 100644 index 0000000000000..2986dd43b0fd6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 b/sql/hive/src/test/resources/golden/create_skewed_table1-7-81fc0a09afbcd674874961c09aa947d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd b/sql/hive/src/test/resources/golden/create_skewed_table1-8-2ae6d96ecee0081ccc474388d50675fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e b/sql/hive/src/test/resources/golden/create_skewed_table1-9-758987cfb7302bdb76898290de49a80e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 b/sql/hive/src/test/resources/golden/create_struct_table-0-d519c3ac0717b41167f8a40b472ad6b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 b/sql/hive/src/test/resources/golden/create_struct_table-1-2ca90a28a994405e6150c96f4a572294 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd b/sql/hive/src/test/resources/golden/create_struct_table-2-d51e74fd10cc16607137b7f715557ecd new file mode 100644 index 0000000000000..6a45cf1600994 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e b/sql/hive/src/test/resources/golden/create_udaf-0-a69c2b11dc912ef5444af32dce6aa33e new file mode 100644 index 0000000000000..8af6a0338d65a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 b/sql/hive/src/test/resources/golden/create_view-0-26e7fe8b9b9769a8d6a8a95b9cfbdf91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 b/sql/hive/src/test/resources/golden/create_view-1-c186ac1fe46117acb6fd452df15e0d92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd b/sql/hive/src/test/resources/golden/create_view-10-9f71514dffc747ddd49fbb1fafb6d3dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-11-2021c047203276dd2db51a56e672fcea b/sql/hive/src/test/resources/golden/create_view-11-2021c047203276dd2db51a56e672fcea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc b/sql/hive/src/test/resources/golden/create_view-12-420752b11848e29bce1c8ace7d3060fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a b/sql/hive/src/test/resources/golden/create_view-13-bff53e5df8356ac16f7b9b78b157e60a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 b/sql/hive/src/test/resources/golden/create_view-14-69162f2f22978113fea529d7fc7b78d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 b/sql/hive/src/test/resources/golden/create_view-15-ceebf4cb0dc23f517a444266bc8d2447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 b/sql/hive/src/test/resources/golden/create_view-16-cb434f5704ee3ed21d1f1521a2a654f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a b/sql/hive/src/test/resources/golden/create_view-17-b3c1023d56f3439833c246e8bfd4502a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 b/sql/hive/src/test/resources/golden/create_view-18-437d0a699b73c61044ebf8539feb14f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b b/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b new file mode 100644 index 0000000000000..e87fb81a6f043 --- /dev/null +++ b/sql/hive/src/test/resources/golden/create_view-19-df2da604b5a8f2b236519510b5e4d34b @@ -0,0 +1 @@ +86 val_86 diff --git a/sql/hive/src/test/resources/golden/create_view-2-d80dcd1271ab264292e9938f3162427c b/sql/hive/src/test/resources/golden/create_view-2-d80dcd1271ab264292e9938f3162427c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 b/sql/hive/src/test/resources/golden/create_view-20-56d203e4d0eb776bb4fa38409222b5b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 b/sql/hive/src/test/resources/golden/create_view-21-3609711e61b5b8d241d0e839557bfd64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 b/sql/hive/src/test/resources/golden/create_view-22-3bc364c0ee46900d2201d706d2d58d67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 b/sql/hive/src/test/resources/golden/create_view-3-25ffe475d52d6c399acaf120dc02afe8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 b/sql/hive/src/test/resources/golden/create_view-4-87ed262d455e99ad45c909a2265a61b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 b/sql/hive/src/test/resources/golden/create_view-5-391caf27ff1589ec68d5f3bc4a27e711 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 b/sql/hive/src/test/resources/golden/create_view-6-d8d0e830783c383e3c00e9de3919c409 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 b/sql/hive/src/test/resources/golden/create_view-7-50b35b8a1772becc96cff65bba1eaee7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 b/sql/hive/src/test/resources/golden/create_view-8-2ae18fc75eda9c3fe7d4e87829180805 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 b/sql/hive/src/test/resources/golden/create_view-9-ed7a1e8aeaed3beb95ac5aa3af216ab9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc b/sql/hive/src/test/resources/golden/create_view_partitioned-0-d98274f9b34c8968292ccd6c959491dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 b/sql/hive/src/test/resources/golden/create_view_partitioned-1-ff29c88fac9c21eaf36469a4ce0fce18 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 b/sql/hive/src/test/resources/golden/create_view_partitioned-2-45e7b89caadba56ec67638c341209f96 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 b/sql/hive/src/test/resources/golden/create_view_partitioned-3-cf44ff130f66de720a77888260ef8d16 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 b/sql/hive/src/test/resources/golden/create_view_partitioned-4-aa9fad452f806fd16fc0bdc7cdfdc4d5 new file mode 100644 index 0000000000000..70d2ecdbc8f78 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc b/sql/hive/src/test/resources/golden/create_view_partitioned-5-d7a7d8592fca266745725192d3f875fc new file mode 100644 index 0000000000000..43bc3de44f4df --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 b/sql/hive/src/test/resources/golden/ct_case_insensitive-0-a22bfdbaf9720a07e0b2c0923036ce93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 b/sql/hive/src/test/resources/golden/ct_case_insensitive-1-893c61ec6ea62362324c213f588d8030 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 b/sql/hive/src/test/resources/golden/ct_case_insensitive-2-9db45f87a7b1e69d7bf38ac6d5009122 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e b/sql/hive/src/test/resources/golden/ctas_varchar-0-311fdd725609cd47ea1b859f706da41e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 b/sql/hive/src/test/resources/golden/ctas_varchar-1-939814768fe997b27f01758d60fcd8f5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 b/sql/hive/src/test/resources/golden/ctas_varchar-2-3223504c97628a44b65736565c1dda32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b b/sql/hive/src/test/resources/golden/ctas_varchar-3-5d14fdc559b9790d81a8020bdcf4159b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 b/sql/hive/src/test/resources/golden/ctas_varchar-4-ccead78e4ec4583da3b48864e78cfd44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f b/sql/hive/src/test/resources/golden/ctas_varchar-5-ff9d92788c0a7d6d4fca8a5bf1095e7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 b/sql/hive/src/test/resources/golden/ctas_varchar-6-351fa7f09c2e809f9cc87d83e11c1539 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec b/sql/hive/src/test/resources/golden/ctas_varchar-7-8620d7f55849ab163b6b85f51abdaaec new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 b/sql/hive/src/test/resources/golden/ctas_varchar-8-c93df93c0e8688f9c7a6167589670d32 new file mode 100644 index 0000000000000..6839c16243bcd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 b/sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/custom_input_output_format-0-94f3da887aa34aed74715bd2051bf3c5 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd b/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/database.table table.attr-0-6cbb13c3a48f53fa6550dbba4d2c28fd @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e b/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/database.table-0-c657beb729b6a7882309a203fc6f298e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f b/sql/hive/src/test/resources/golden/database_drop-0-49f18014566b3e020dc19b1e61d25a4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 new file mode 100644 index 0000000000000..585786f6a8a41 --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_drop-1-549981e00a3d95f03dd5a9ef6044aa20 @@ -0,0 +1,2 @@ +db5 +default diff --git a/sql/hive/src/test/resources/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 b/sql/hive/src/test/resources/golden/database_drop-10-1b0a6cff3151cfa0ef0a6f78587973a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 b/sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_drop-11-2ea883422b74b701711e14e61472ba06 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941 b/sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_drop-12-e02a53f7e798d2741152526516f14941 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 b/sql/hive/src/test/resources/golden/database_drop-2-b28ed063cd2c2d1d370d6d422782a8f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 b/sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_drop-3-f21dd011aeb417043ed58c03fd5c3bf0 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 b/sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_drop-4-d419f4ff197d4291208c2028cd158909 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 b/sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_drop-5-b7cf74929eabe781b0db79ed1043dc24 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 b/sql/hive/src/test/resources/golden/database_drop-6-a47b1b070ec7c3b9ccabc34f41aebad9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba b/sql/hive/src/test/resources/golden/database_drop-7-693736836ccc99f6e2449b94efcfeeba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_drop-8-97101266791d2b2c662bcde549422318 b/sql/hive/src/test/resources/golden/database_drop-8-97101266791d2b2c662bcde549422318 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 b/sql/hive/src/test/resources/golden/database_drop-9-8db536f925bf0f5058f97897e145a661 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/database_location-0-5528e36b3b0f5b14313898cc45f9c23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 b/sql/hive/src/test/resources/golden/database_location-1-2b66b4554cf8ecf2ab6c325d4d89491 new file mode 100644 index 0000000000000..f05305df0c3d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 b/sql/hive/src/test/resources/golden/database_location-10-c5cd9c57a13da7f345563fbd75da4e45 new file mode 100644 index 0000000000000..e92c241d50092 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..7c77c06cda82b --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_location-11-9c36cac1372650b703400c60dd29042c @@ -0,0 +1 @@ +table_db2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 b/sql/hive/src/test/resources/golden/database_location-2-7650b86c86dd6b1a99c86ddc5a31bd63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 b/sql/hive/src/test/resources/golden/database_location-3-81383a2b9568f942cc8e758f9a6ee2f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb b/sql/hive/src/test/resources/golden/database_location-4-be65cf649363681d54e593c42a5ecffb new file mode 100644 index 0000000000000..cda1736e9ca6b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..4d8fc6a6d6fe0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_location-5-9c36cac1372650b703400c60dd29042c @@ -0,0 +1 @@ +table_db1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a b/sql/hive/src/test/resources/golden/database_location-6-6fa58170a7c2e78b06a250403f02091a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 b/sql/hive/src/test/resources/golden/database_location-7-5698ac10441da07dbe3a947143c999c2 new file mode 100644 index 0000000000000..3e23970adddcf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b b/sql/hive/src/test/resources/golden/database_location-8-6f2797b6f81943d3b53b8d247ae8512b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 b/sql/hive/src/test/resources/golden/database_location-9-92f087a5934481942995fc2aaf0d87e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_properties-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 b/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_properties-1-10de6a198e2b3f61974519ddd8623e68 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 b/sql/hive/src/test/resources/golden/database_properties-10-26c10ff2ec4a69b16589069ced427d23 new file mode 100644 index 0000000000000..2f5fbe26f4945 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 b/sql/hive/src/test/resources/golden/database_properties-2-a1074315e598ad16bce55860e6e43363 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 b/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 new file mode 100644 index 0000000000000..86362ead004d9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_properties-3-751417d45b8e80ee5cba2034458b5bc9 @@ -0,0 +1,2 @@ +db1 +default \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 b/sql/hive/src/test/resources/golden/database_properties-4-ddf44597db4fa15e89bee313f2dad371 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a b/sql/hive/src/test/resources/golden/database_properties-5-51c0974df1125b233936f25ce709ba4a new file mode 100644 index 0000000000000..ff89c3fe899d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 b/sql/hive/src/test/resources/golden/database_properties-6-26c10ff2ec4a69b16589069ced427d23 new file mode 100644 index 0000000000000..5827d2726d084 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_properties-7-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 b/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/database_properties-8-10de6a198e2b3f61974519ddd8623e68 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a b/sql/hive/src/test/resources/golden/database_properties-9-5a2bc556d3c66c5b33ab86e6cd37b54a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd b/sql/hive/src/test/resources/golden/date_2-0-8acfa0b538112534f57a3e051f0216bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 b/sql/hive/src/test/resources/golden/date_2-1-116032b973a2060b533e1cdc9dfda301 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_2-2-cab14d992c53c106ab257fae52001e04 b/sql/hive/src/test/resources/golden/date_2-2-cab14d992c53c106ab257fae52001e04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 b/sql/hive/src/test/resources/golden/date_3-0-c26de4559926ddb0127d2dc5ea154774 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 b/sql/hive/src/test/resources/golden/date_3-1-d9a07d08f5204ae8208fd88c9255d447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 b/sql/hive/src/test/resources/golden/date_3-2-a937c6e5a2c655930e0d3f80883ecc16 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c b/sql/hive/src/test/resources/golden/date_join1-0-70b9b49c55699fe94cfde069f5d197c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_join1-1-3a68de2112a212a07a3068916c608fb b/sql/hive/src/test/resources/golden/date_join1-1-3a68de2112a212a07a3068916c608fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 b/sql/hive/src/test/resources/golden/date_join1-2-894b6541812ac8b0abe2a24c966817d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 b/sql/hive/src/test/resources/golden/date_serde-0-ca88593bb7ec47fa782145d732100c07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-1-36e6041f53433482631018410bb62a99 b/sql/hive/src/test/resources/golden/date_serde-1-36e6041f53433482631018410bb62a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 b/sql/hive/src/test/resources/golden/date_serde-2-3ddfd8ecb28991aeed588f1ea852c427 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-3-e6167e27465514356c557a77d956ea46 b/sql/hive/src/test/resources/golden/date_serde-3-e6167e27465514356c557a77d956ea46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 b/sql/hive/src/test/resources/golden/date_serde-4-c1e17c93582656c12970c37bac153bf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c b/sql/hive/src/test/resources/golden/date_serde-5-4a17944b9ec8999bb20c5ba5d4cb877c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-6-e00facec2986bc55074868eff87ba22a b/sql/hive/src/test/resources/golden/date_serde-6-e00facec2986bc55074868eff87ba22a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d b/sql/hive/src/test/resources/golden/date_serde-7-a34279d8ebbadb78e925e8ed9c78947d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 b/sql/hive/src/test/resources/golden/ddltime-0-d81aa70a19a0e7428e8f7a76b60d3fc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 b/sql/hive/src/test/resources/golden/ddltime-1-426da52526f6f48c0ddeb0026fd566f1 new file mode 100644 index 0000000000000..c270c7cbdfa1f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/decimal_1-0-31ecaab3afa056fcc656d6e54f845cf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd b/sql/hive/src/test/resources/golden/decimal_1-1-f2e4dab0c9a4d9b3128aca89a424accd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 b/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-10-be179d261b9c42ed843dbf736b12e75 @@ -0,0 +1 @@ +17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 b/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-11-4c1fefa61e10a78f6406b526a60e4937 @@ -0,0 +1 @@ +17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 b/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 new file mode 100644 index 0000000000000..c4a17c1b14c88 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-12-75b9add5e32a963cc9913c6ef4f84989 @@ -0,0 +1 @@ +1969-12-31 16:00:17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 b/sql/hive/src/test/resources/golden/decimal_1-13-31ecaab3afa056fcc656d6e54f845cf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 b/sql/hive/src/test/resources/golden/decimal_1-2-80fc87cab17ceffea334afbb230a6653 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 b/sql/hive/src/test/resources/golden/decimal_1-3-2a3647b0a148236c45d8e3e9891c8ad5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 b/sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-4-f2bcc4f2b8db16b865d4ca396fbca575 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 b/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-5-cbe6b235663cf78e602673ed715a2f40 @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b b/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-6-91b7219bd8c67673e16cb970fcc7be1b @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a b/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-7-7b2fab0ebc0962f0a53f6c61da417a @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d b/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-8-cdd0932288d3cc43636334439805769d @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c b/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_1-9-1504e1b00092e2c58bafcc9936ad178c @@ -0,0 +1 @@ +17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee b/sql/hive/src/test/resources/golden/decimal_2-0-3c8ecb1468952afe028596c65d587bee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b b/sql/hive/src/test/resources/golden/decimal_2-1-868e124edc1581325bd0fd10235a126b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 b/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-10-f97d72aeb605ee18d34361c073552e92 @@ -0,0 +1 @@ +17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 b/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-11-58a090c30c59446d1e2b2a6c85fabf50 @@ -0,0 +1 @@ +17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 b/sql/hive/src/test/resources/golden/decimal_2-12-d63b5ea25e27852413132db4d9bfb035 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 b/sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-13-20ffe5115367abea9ea0ed1bda7a9439 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 b/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 new file mode 100644 index 0000000000000..b1bd38b62a080 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-14-3105d1029ad981af9cf1039ad9410fc0 @@ -0,0 +1 @@ +13 diff --git a/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 b/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 new file mode 100644 index 0000000000000..de7771ac23570 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-15-3266fde6f6ab80a8bc027de0d5574f02 @@ -0,0 +1 @@ +-3827 diff --git a/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 b/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 new file mode 100644 index 0000000000000..272791f402250 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-16-dc565c6c5bc24fd8b32729ce91999580 @@ -0,0 +1 @@ +3404045 diff --git a/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d b/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d new file mode 100644 index 0000000000000..272791f402250 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-17-a4a1ca9ae92642dd78d4bdf6af781c2d @@ -0,0 +1 @@ +3404045 diff --git a/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e b/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e new file mode 100644 index 0000000000000..deb8427800ee4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-18-f7c34d67fd579c82c636415172ec675e @@ -0,0 +1 @@ +3404045.5 diff --git a/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 b/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 new file mode 100644 index 0000000000000..6f31e8fe55034 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-19-f97d72aeb605ee18d34361c073552e92 @@ -0,0 +1 @@ +3404045.5044003 diff --git a/sql/hive/src/test/resources/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 b/sql/hive/src/test/resources/golden/decimal_2-2-6cc742523b3574e59ca21dad30f2d506 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 b/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 new file mode 100644 index 0000000000000..6f31e8fe55034 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-20-58a090c30c59446d1e2b2a6c85fabf50 @@ -0,0 +1 @@ +3404045.5044003 diff --git a/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 b/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 new file mode 100644 index 0000000000000..6324d401a069f --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-21-d72d68be1217c7b7a958f58456d85821 @@ -0,0 +1 @@ +3.14 diff --git a/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 b/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 new file mode 100644 index 0000000000000..6324d401a069f --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-22-648e694eea042c59e8db30d067cb5bc8 @@ -0,0 +1 @@ +3.14 diff --git a/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff b/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff new file mode 100644 index 0000000000000..603f18cc37bc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-23-a87b701c93a922b9e33ba16ae6a477ff @@ -0,0 +1 @@ +1355944339.1234567 diff --git a/sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e b/sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-24-8c2a8f1f3b792d5017be42078b15b94e @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36 b/sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-25-14face5c7104382196e65741a199c36 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 b/sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-26-4ea9fdaf7131c085df8f93ffb64956e5 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 b/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-27-a5ea3949eb5ab338916e4316c676c7f6 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 b/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-28-4a5410f96c6ef0843f12b0f593c104b1 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 b/sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-29-659f627f0ff5fe2296a8a0a7daed6298 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e b/sql/hive/src/test/resources/golden/decimal_2-3-17702ba7ecd04caad0158e2cd1f6f26e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a b/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a new file mode 100644 index 0000000000000..474c8b180aea9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-30-26a71d79e41353830b4ada96de6e2b8a @@ -0,0 +1 @@ +0.99999999999999999999 diff --git a/sql/hive/src/test/resources/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee b/sql/hive/src/test/resources/golden/decimal_2-31-3c8ecb1468952afe028596c65d587bee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 b/sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-4-20ffe5115367abea9ea0ed1bda7a9439 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 b/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-5-3105d1029ad981af9cf1039ad9410fc0 @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 b/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-6-3266fde6f6ab80a8bc027de0d5574f02 @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 b/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-7-dc565c6c5bc24fd8b32729ce91999580 @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d b/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d new file mode 100644 index 0000000000000..98d9bcb75a685 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-8-a4a1ca9ae92642dd78d4bdf6af781c2d @@ -0,0 +1 @@ +17 diff --git a/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e b/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e new file mode 100644 index 0000000000000..53aca7545dac7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_2-9-f7c34d67fd579c82c636415172ec675e @@ -0,0 +1 @@ +17.29 diff --git a/sql/hive/src/test/resources/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef b/sql/hive/src/test/resources/golden/decimal_3-0-90cd495a00051a0631b2021dbb9a4aef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 b/sql/hive/src/test/resources/golden/decimal_3-1-76f900dfe9ce95e8262a53939d33fb01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 b/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 new file mode 100644 index 0000000000000..8d8753f153d7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_3-10-420614bb0789115e008c96a7ad822624 @@ -0,0 +1,4 @@ +3.14 3 +3.14 3 +3.14 3 +3.14 4 diff --git a/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 b/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 new file mode 100644 index 0000000000000..8d8753f153d7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_3-11-63913753553b16d6c24e063fb49fdd15 @@ -0,0 +1,4 @@ +3.14 3 +3.14 3 +3.14 3 +3.14 4 diff --git a/sql/hive/src/test/resources/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 b/sql/hive/src/test/resources/golden/decimal_3-12-d495d7178707ba55dcc01b9bb3398792 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 b/sql/hive/src/test/resources/golden/decimal_3-2-4c5356497c8830c8e7cd8e5c02ad104 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b b/sql/hive/src/test/resources/golden/decimal_3-3-cb2d14de1d779cce9c19ba1a9690ca6b new file mode 100644 index 0000000000000..3e290231c27e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 b/sql/hive/src/test/resources/golden/decimal_3-4-b3f259a4b17b4fc585476ad4be8ed263 new file mode 100644 index 0000000000000..64fa7bca9a81b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b b/sql/hive/src/test/resources/golden/decimal_3-5-cb2d14de1d779cce9c19ba1a9690ca6b new file mode 100644 index 0000000000000..3e290231c27e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 b/sql/hive/src/test/resources/golden/decimal_3-6-127a3a8400cae591c282dd24f8951e55 new file mode 100644 index 0000000000000..24d34ee5d8c1b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 b/sql/hive/src/test/resources/golden/decimal_3-7-9d4f27d4a4819113c5083462baa72052 new file mode 100644 index 0000000000000..e08f588c89461 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 b/sql/hive/src/test/resources/golden/decimal_3-8-f65f4df6e3d971d575654ade4b4e4800 new file mode 100644 index 0000000000000..796707d06b0dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 b/sql/hive/src/test/resources/golden/decimal_3-9-b54243d38214362f9a9b1831548faac4 new file mode 100644 index 0000000000000..4217ad848170e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f b/sql/hive/src/test/resources/golden/decimal_4-0-98a58225355eb73036bb7b1144fa5a5f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 b/sql/hive/src/test/resources/golden/decimal_4-1-fa7e76c5dff605e58aa9d99182f865b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d b/sql/hive/src/test/resources/golden/decimal_4-2-945542ec888136afaece8d7a5e20d52d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e b/sql/hive/src/test/resources/golden/decimal_4-3-399140971a10a5a0cc6a8c97a4635e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c b/sql/hive/src/test/resources/golden/decimal_4-4-81b37675c09ed874497325ae13233e5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c b/sql/hive/src/test/resources/golden/decimal_4-5-7f2b3465d6a46b6f05cbb9bfe963f88c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e b/sql/hive/src/test/resources/golden/decimal_4-6-693c2e345731f9b2b547c3b75218458e new file mode 100644 index 0000000000000..3e290231c27e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 b/sql/hive/src/test/resources/golden/decimal_4-7-f1eb45492510cb76cf6b452121af8531 new file mode 100644 index 0000000000000..795a4b567ab7f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 b/sql/hive/src/test/resources/golden/decimal_4-8-79734272b75fb9076bdb64644bed6276 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 b/sql/hive/src/test/resources/golden/decimal_4-9-fe020b24cca92de056bddee2a1a3c5a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 b/sql/hive/src/test/resources/golden/decimal_join-0-c7659c1efa06c9eab1db28e916b750e4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e b/sql/hive/src/test/resources/golden/decimal_join-1-c1524f17ee815171055a67ddc2f9de4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 b/sql/hive/src/test/resources/golden/decimal_join-2-e966f01e702d4cc8f970dcdbc6007285 new file mode 100644 index 0000000000000..23ec633b6a337 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e b/sql/hive/src/test/resources/golden/decimal_precision-0-cae8ceb6b0ab342948041c511a867b8e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 b/sql/hive/src/test/resources/golden/decimal_precision-1-3f4119830536c92e5ccd76be0259e110 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 b/sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_precision-10-d8a597810b222e9e121a11a1f5658fb0 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 b/sql/hive/src/test/resources/golden/decimal_precision-11-673b15434ba47f11c71c3e8b2a575d83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e b/sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_precision-12-18906f5c6413065621430e3fe33c7e9e @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e b/sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_precision-13-2a65d450f57f8ba9f594063b96074f0e @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 b/sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_precision-14-34916eb904b8113a401ce78e6941a204 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce b/sql/hive/src/test/resources/golden/decimal_precision-15-5c49f041326bc5a9e936910094f190ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c b/sql/hive/src/test/resources/golden/decimal_precision-2-d5be00a0fa6e2e290b40458442bd036c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b b/sql/hive/src/test/resources/golden/decimal_precision-3-42cb35d680b3caeeb22e1c4865b8264b new file mode 100644 index 0000000000000..3d9e792183f3c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc b/sql/hive/src/test/resources/golden/decimal_precision-4-38aaeba3e587b4dac72e26c4b02029fc new file mode 100644 index 0000000000000..9853ce72ed8c3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef b/sql/hive/src/test/resources/golden/decimal_precision-5-bb27734245ecbd0511be91af21c3b9ef new file mode 100644 index 0000000000000..7cc75c789dee1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 b/sql/hive/src/test/resources/golden/decimal_precision-6-b2547e6ef33325b2da12ce91b57af21 new file mode 100644 index 0000000000000..c40875630d1b2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 b/sql/hive/src/test/resources/golden/decimal_precision-7-ee5b7767c7fbb8614bb4ef907e8737b7 new file mode 100644 index 0000000000000..bd23d17293f79 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e b/sql/hive/src/test/resources/golden/decimal_precision-8-6e6bd4655de124dc1fc482ce0d11930e new file mode 100644 index 0000000000000..c1e0db0174c63 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab b/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab new file mode 100644 index 0000000000000..81af0e4cd3ab8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/decimal_precision-9-e7b465fbeb49487b2a972a314e2c01ab @@ -0,0 +1 @@ +NULL NULL diff --git a/sql/hive/src/test/resources/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 b/sql/hive/src/test/resources/golden/default_partition_name-0-312a37c422883aa0d660018248157cf8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c b/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/default_partition_name-1-9de8e5f66c536d4ace89c61759db829c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 b/sql/hive/src/test/resources/golden/default_partition_name-2-8732fdce7fb224dd783da2b83a93c795 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be b/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be new file mode 100644 index 0000000000000..ded23df148827 --- /dev/null +++ b/sql/hive/src/test/resources/golden/default_partition_name-3-a7047012b4bce0158edaafe5cf0a57be @@ -0,0 +1 @@ +ds=__HIVE_DEFAULT_PARTITION__ \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba b/sql/hive/src/test/resources/golden/delimiter-0-ef9bd1f5e2dad851509d6d2907c0e6ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 b/sql/hive/src/test/resources/golden/delimiter-1-d9e405c4107da8af78fcacb83a667b41 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 b/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 new file mode 100644 index 0000000000000..f438072c76b5f --- /dev/null +++ b/sql/hive/src/test/resources/golden/delimiter-2-d7137294d2e53ea6edc259943e4c6069 @@ -0,0 +1,3 @@ +35 40 +48 32 +100100 40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 b/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 new file mode 100644 index 0000000000000..f438072c76b5f --- /dev/null +++ b/sql/hive/src/test/resources/golden/delimiter-3-176724f76343433a8f2e6131b12206d7 @@ -0,0 +1,3 @@ +35 40 +48 32 +100100 40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 b/sql/hive/src/test/resources/golden/delimiter-4-f17c3a91cdc84fbc6d14011b04f12a64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb b/sql/hive/src/test/resources/golden/describe_comment_indent-0-5b66d27453f15517fb266a5e1a0e3cbb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 b/sql/hive/src/test/resources/golden/describe_comment_indent-1-5536eb772d43014b971c6da3a0c44904 new file mode 100644 index 0000000000000..d980efc81b947 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 b/sql/hive/src/test/resources/golden/describe_comment_indent-2-91bffa09f4f2caaaa6864bf935c2ea94 new file mode 100644 index 0000000000000..01b9151074b22 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_database_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 b/sql/hive/src/test/resources/golden/describe_database_json-1-12cc2812ab067d58718c29ea6aa3d8a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 b/sql/hive/src/test/resources/golden/describe_database_json-10-1afddec8522bd69f496b15980600a6e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 b/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_database_json-11-b05391400dc31139998dc3abaaf86320 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a b/sql/hive/src/test/resources/golden/describe_database_json-2-8e7cfe3e6069e796124ca940125385a new file mode 100644 index 0000000000000..c56a79e4f322e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 b/sql/hive/src/test/resources/golden/describe_database_json-3-d097973152d91fa8072facb0f739e304 new file mode 100644 index 0000000000000..aa08c38c68d1d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 b/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 new file mode 100644 index 0000000000000..513aeaab1dc66 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_database_json-4-549981e00a3d95f03dd5a9ef6044aa20 @@ -0,0 +1 @@ +{"databases":["default","jsondb1"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c b/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c new file mode 100644 index 0000000000000..606069d6291b4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_database_json-5-a3ee372283f45479db3f2cd7ebeedc8c @@ -0,0 +1 @@ +{"databases":["jsondb1"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 b/sql/hive/src/test/resources/golden/describe_database_json-6-1afddec8522bd69f496b15980600a6e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 b/sql/hive/src/test/resources/golden/describe_database_json-7-7529ec337ca17cdf95d037f29e1cb793 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a b/sql/hive/src/test/resources/golden/describe_database_json-8-8e7cfe3e6069e796124ca940125385a new file mode 100644 index 0000000000000..bea7c01440c46 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 b/sql/hive/src/test/resources/golden/describe_database_json-9-d097973152d91fa8072facb0f739e304 new file mode 100644 index 0000000000000..bea7c01440c46 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-0-889714213a760ae9ab3ebe199eb30b62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-1-cbd03c487eba9e34d57a8decaa3a0dfa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-2-db8910ace81a5102495905a508ef5c28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-3-b80c7ae3530bfdbc8e865d49742da826 new file mode 100644 index 0000000000000..381821184d693 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned-4-889714213a760ae9ab3ebe199eb30b62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-1-889714213a760ae9ab3ebe199eb30b62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-2-cbd03c487eba9e34d57a8decaa3a0dfa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-3-db8910ace81a5102495905a508ef5c28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 new file mode 100644 index 0000000000000..7b51873776ad8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-4-b80c7ae3530bfdbc8e865d49742da826 @@ -0,0 +1 @@ +{"columns":[{"name":"key","type":"int"}]} diff --git a/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 b/sql/hive/src/test/resources/golden/describe_formatted_view_partitioned_json-5-889714213a760ae9ab3ebe199eb30b62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad b/sql/hive/src/test/resources/golden/describe_pretty-0-f34ca99310bf1d4793cf64423c024ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf b/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_pretty-1-dbfaa12f26f99277b8397379189172cf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf b/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_pretty-10-dbfaa12f26f99277b8397379189172cf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 b/sql/hive/src/test/resources/golden/describe_pretty-11-e98c50f4b5cdb6ba04df147c57e0b264 new file mode 100644 index 0000000000000..35fac1b6f2579 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 b/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_pretty-12-7d1e5e69d235a785fa3f0e099dd894c3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 b/sql/hive/src/test/resources/golden/describe_pretty-13-e98c50f4b5cdb6ba04df147c57e0b264 new file mode 100644 index 0000000000000..b57f8955ca397 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 b/sql/hive/src/test/resources/golden/describe_pretty-14-22db46f42dc0c1bf01a76ca360c20a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 b/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_pretty-15-db4abe06aba81e685c52db6b43ba7c03 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 b/sql/hive/src/test/resources/golden/describe_pretty-16-ada55b65b72e0d65563ad6161e005f22 new file mode 100644 index 0000000000000..3b7fe3c133089 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 b/sql/hive/src/test/resources/golden/describe_pretty-2-713712e0f6f18144d1f3a522e9b98861 new file mode 100644 index 0000000000000..49175da27357f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 b/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_pretty-3-87c1f2148ecdc0d77eecb72b0268d4b4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-4-1546db18568697fa68a7fc781802d255 new file mode 100644 index 0000000000000..c3e77e079a9d4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 b/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_pretty-5-ce1966d8626096821b404ab8745c4914 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-6-1546db18568697fa68a7fc781802d255 new file mode 100644 index 0000000000000..452f75a11fdd0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 b/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_pretty-7-db4abe06aba81e685c52db6b43ba7c03 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 b/sql/hive/src/test/resources/golden/describe_pretty-8-1546db18568697fa68a7fc781802d255 new file mode 100644 index 0000000000000..ee5a10c85057a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 b/sql/hive/src/test/resources/golden/describe_pretty-9-e382a994134aefcd2652b57af9195644 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/describe_syntax-0-5528e36b3b0f5b14313898cc45f9c23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 b/sql/hive/src/test/resources/golden/describe_syntax-1-4f3d3497418242124113538edab45df7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 b/sql/hive/src/test/resources/golden/describe_syntax-10-2d15bc50701f8f3c64ec48bd03a23ac5 new file mode 100644 index 0000000000000..4184ce21dc079 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f b/sql/hive/src/test/resources/golden/describe_syntax-11-ab161e38c5d66b6c344c8372160ac74f new file mode 100644 index 0000000000000..c94d6dcb90042 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 b/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_syntax-12-90c7890e1aa28e94520f35f5679560a4 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 b/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_syntax-13-7c1216f9852d454bf93256e5a2588758 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 b/sql/hive/src/test/resources/golden/describe_syntax-14-d167187f54ec60c25f5f7e1f2f2afee9 new file mode 100644 index 0000000000000..f3d242157dd98 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 b/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_syntax-15-e420b255509153b3326f00dcd25d11e4 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 b/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_syntax-16-5043ee273a313d28adeca74fd33739a7 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 b/sql/hive/src/test/resources/golden/describe_syntax-17-c97a9e691cc08199678ead7f79d58b58 new file mode 100644 index 0000000000000..f3d242157dd98 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 b/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_syntax-18-69cecdc1889b7feb40b46c95524aaad4 @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa b/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa new file mode 100644 index 0000000000000..0dea48c260ab2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_syntax-19-c1c3359705e256d7641bbffab00c43fa @@ -0,0 +1 @@ +key1 int from deserializer \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a b/sql/hive/src/test/resources/golden/describe_syntax-2-b198700c0129910d6205ef063ee83d5a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a b/sql/hive/src/test/resources/golden/describe_syntax-20-781dd449a9b26dc54f3bba8e5178cd8a new file mode 100644 index 0000000000000..f3d242157dd98 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 b/sql/hive/src/test/resources/golden/describe_syntax-21-5bac87eeb7e71928d01275b006720de3 new file mode 100644 index 0000000000000..4f76eaca6cd8b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f b/sql/hive/src/test/resources/golden/describe_syntax-22-719a15ffd0018bb2898f9045be886e0f new file mode 100644 index 0000000000000..aa25ca5a29dd3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 b/sql/hive/src/test/resources/golden/describe_syntax-23-2b2f2f068fe8e8fcbe10d11506804e49 new file mode 100644 index 0000000000000..311870f6ad6b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 b/sql/hive/src/test/resources/golden/describe_syntax-24-ee226b42db35b219702319858e925468 new file mode 100644 index 0000000000000..4f76eaca6cd8b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e b/sql/hive/src/test/resources/golden/describe_syntax-25-b6e10514fb473803c44bc793b9f9713e new file mode 100644 index 0000000000000..aa25ca5a29dd3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec b/sql/hive/src/test/resources/golden/describe_syntax-26-4851dc65e26ec33c605ab1ed65b59bec new file mode 100644 index 0000000000000..311870f6ad6b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 b/sql/hive/src/test/resources/golden/describe_syntax-3-458d6aaffeee94997f67a43b88382106 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 b/sql/hive/src/test/resources/golden/describe_syntax-4-67eeb4eddd6b4e905404dd32a9f1d9c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 b/sql/hive/src/test/resources/golden/describe_syntax-5-a305e3d663b2697acca5abb9e7d897c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 b/sql/hive/src/test/resources/golden/describe_syntax-6-74668fbe18bbf3d6323e052ef2b4ca85 new file mode 100644 index 0000000000000..4f76eaca6cd8b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d b/sql/hive/src/test/resources/golden/describe_syntax-7-d67ccae0a3893b4b91b0d4f1bd73b66d new file mode 100644 index 0000000000000..4184ce21dc079 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 b/sql/hive/src/test/resources/golden/describe_syntax-8-9e40f8077f34fa35fccfeae972e8c4 new file mode 100644 index 0000000000000..c94d6dcb90042 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b b/sql/hive/src/test/resources/golden/describe_syntax-9-fb744775fb30d92297534d29b6eafd6b new file mode 100644 index 0000000000000..4f76eaca6cd8b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b b/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_table_json-0-1110d5212ef44ba8c7ac357fb2f4fc7b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 b/sql/hive/src/test/resources/golden/describe_table_json-1-e48b5f50bb2ff56f9886004366cfd491 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..f8bc404bf7308 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_table_json-2-9c36cac1372650b703400c60dd29042c @@ -0,0 +1 @@ +{"tables":["jsontable","src","srcpart"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a b/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a new file mode 100644 index 0000000000000..5895645dbbb50 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_table_json-3-576670af142149302decb9bf8662e68a @@ -0,0 +1 @@ +{"tables":["jsontable"]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 b/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 new file mode 100644 index 0000000000000..353bf2df92f18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_table_json-4-4a57591b392bb8fd18238d068d191721 @@ -0,0 +1 @@ +{"tables":[]} \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb b/sql/hive/src/test/resources/golden/describe_table_json-5-865aeeea2647a71f7f25b03da4203ffb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 b/sql/hive/src/test/resources/golden/describe_table_json-6-ac49d26a0211b804fee89bbe0808f430 new file mode 100644 index 0000000000000..96c1178ae6eab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 b/sql/hive/src/test/resources/golden/describe_table_json-7-24552fd8c4b7d9d460a8ea25eb2d6e30 new file mode 100644 index 0000000000000..4cf10d1d762b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 b/sql/hive/src/test/resources/golden/describe_table_json-8-deaf826aef1c9a7f36d7824eafd20f11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 b/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/describe_table_json-9-b05391400dc31139998dc3abaaf86320 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad b/sql/hive/src/test/resources/golden/diff_part_input_formats-0-12652a5a33548c245772e8d0894af5ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa b/sql/hive/src/test/resources/golden/diff_part_input_formats-1-961f7cb386a6eacd391dcb189cbeddaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce b/sql/hive/src/test/resources/golden/diff_part_input_formats-2-28cd0f9b01baa8627a013339dc9508ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 b/sql/hive/src/test/resources/golden/diff_part_input_formats-3-c6eef43568e8ed96299720d30a6235e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b b/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/diff_part_input_formats-4-a4890f2b20715c75e05c674d9155a5b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c b/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/disable_file_format_check-0-bd9d00e3ffcaea450a3cc8d0ba6f865c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e b/sql/hive/src/test/resources/golden/disable_file_format_check-1-ec4d7e2ce5bd580b24119860e96f376e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 b/sql/hive/src/test/resources/golden/disable_file_format_check-2-d3e20a1484eabcd50e2039e55b4f549 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f b/sql/hive/src/test/resources/golden/disable_file_format_check-3-93063fb3476a400ecfec0bfd02cbc23f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb b/sql/hive/src/test/resources/golden/disable_file_format_check-4-fe6f402a026c882c2bc8f5251794dbbb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-0-a071dedef216e84d1cb2f0de6d34fd1a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b new file mode 100644 index 0000000000000..7ae602958428e --- /dev/null +++ b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-1-ce3797dc14a603cba2a5e58c8612de5b @@ -0,0 +1 @@ +238 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-2-f5340880d2be7b0643eb995673e89d11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-3-b4e3c4c6b7ae6e0fad9ab7728f2ace85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c new file mode 100644 index 0000000000000..ca21e093aa698 --- /dev/null +++ b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-4-34064fd15c28dba55865cb8f3c5ba68c @@ -0,0 +1 @@ +1 {"a1":"b1"} foo1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 b/sql/hive/src/test/resources/golden/disallow_incompatible_type_change_off-5-f40a07d7654573e1a8517770eb8529e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-0-b454ca2d55b61fd597540dbe38eb51ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 b/sql/hive/src/test/resources/golden/drop_database_removes_partition_dirs-1-ece80e0bd1236c547da7eceac114e602 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_function-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a b/sql/hive/src/test/resources/golden/drop_function-1-ea5871f0a80a41e19fd6a42bd29b693a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_index-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 b/sql/hive/src/test/resources/golden/drop_index-1-5875a80dd89498c8b61db5f6bf26898 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b b/sql/hive/src/test/resources/golden/drop_multi_partitions-0-c4449feb8f8e2c40c294ccf50680b47b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 b/sql/hive/src/test/resources/golden/drop_multi_partitions-1-d738aa3c169c10f5b1e5959453dffbd4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 new file mode 100644 index 0000000000000..c9d701778f9ab --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-10-9120e865aa132bac6e0a29c907f0b760 @@ -0,0 +1 @@ +b=2/c=2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 b/sql/hive/src/test/resources/golden/drop_multi_partitions-2-7554be9025c7683c67dce09177396357 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 b/sql/hive/src/test/resources/golden/drop_multi_partitions-3-bc92a4d9670709904a8d49ebe9ba8e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-4-9120e865aa132bac6e0a29c907f0b760 new file mode 100644 index 0000000000000..53e09b6e34202 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 b/sql/hive/src/test/resources/golden/drop_multi_partitions-5-53b0c9e5b0c8c37e75c1750280cf2aa0 new file mode 100644 index 0000000000000..31b543e8b4122 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 b/sql/hive/src/test/resources/golden/drop_multi_partitions-6-70ad97221d2be48259ea556f9d8e5353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 b/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 new file mode 100644 index 0000000000000..c9d701778f9ab --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-7-9120e865aa132bac6e0a29c907f0b760 @@ -0,0 +1 @@ +b=2/c=2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 b/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_multi_partitions-8-46a4f646bbc04f70e7ae5ed992f102f1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 b/sql/hive/src/test/resources/golden/drop_multi_partitions-9-3d4108847515b6386cd28b8862dcab53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 b/sql/hive/src/test/resources/golden/drop_partitions_filter-0-6863a128b9a05e5c251ec0092e6124d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter-1-e3eda6672f6602a1b9bc93ef789662f6 new file mode 100644 index 0000000000000..d1e9fd1cd0a21 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b b/sql/hive/src/test/resources/golden/drop_partitions_filter-10-119b0fe0597fa478e1533a412e2d444b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-11-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..3377ef5cf498f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 b/sql/hive/src/test/resources/golden/drop_partitions_filter-12-e0e995c7bcc6b6d801b68878b6166835 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-13-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..40e71fb79ad0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 b/sql/hive/src/test/resources/golden/drop_partitions_filter-14-f47427726dd7546c3c59a2ec53891bb3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-15-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..c03d86a551c29 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b b/sql/hive/src/test/resources/golden/drop_partitions_filter-16-fae7d99452ab708daa96c30f0b25c03b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-17-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..133c0256f898f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd b/sql/hive/src/test/resources/golden/drop_partitions_filter-18-4553ba96e8c121b2e98966a67220a0fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-19-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..98e52eedc2b03 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 b/sql/hive/src/test/resources/golden/drop_partitions_filter-2-16027a4bed34a7610bbea1e11e83c3f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c b/sql/hive/src/test/resources/golden/drop_partitions_filter-20-a08a2d37122fd5770f4197ec1f0ebd1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..687f7dd7e13a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-21-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1 @@ +c=France/d=4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 b/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-22-46a4f646bbc04f70e7ae5ed992f102f1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e b/sql/hive/src/test/resources/golden/drop_partitions_filter-23-45bb3dea1b8d4bd353885cd68729698e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..687f7dd7e13a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_partitions_filter-24-83e3e422cdf9403523fa60d75376d7d7 @@ -0,0 +1 @@ +c=France/d=4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c b/sql/hive/src/test/resources/golden/drop_partitions_filter-25-67d75c72ea2d3982c55f3a850d93f83c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c b/sql/hive/src/test/resources/golden/drop_partitions_filter-3-a499bb560b6e228b924387759214bc3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d b/sql/hive/src/test/resources/golden/drop_partitions_filter-4-8fde1e5c12230f4b8081950dfd68b18d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 b/sql/hive/src/test/resources/golden/drop_partitions_filter-5-d3dc443408a20518c42b839fba218216 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e b/sql/hive/src/test/resources/golden/drop_partitions_filter-6-8cf34fc1e9004b8fdf43770f7893506e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 b/sql/hive/src/test/resources/golden/drop_partitions_filter-7-7f8f890e3104e36ff8f5747d9a287b39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 b/sql/hive/src/test/resources/golden/drop_partitions_filter-8-12ca7d0f34ab5127416bdb66d9e4a698 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 b/sql/hive/src/test/resources/golden/drop_partitions_filter-9-972bcd28a9e24b4cac70ef74524f1696 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-0-322b270dab4032668de9002e9e8bc7c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-1-e3eda6672f6602a1b9bc93ef789662f6 new file mode 100644 index 0000000000000..1329d173d6a21 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-10-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..08051a26d24cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a b/sql/hive/src/test/resources/golden/drop_partitions_filter2-11-8b9e0542bfbf07e9ceabb1ce599d856a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-12-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..7f6e4ae8abf83 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c b/sql/hive/src/test/resources/golden/drop_partitions_filter2-13-67d75c72ea2d3982c55f3a850d93f83c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-2-e83e7a8f276f890c4eb29e192d684730 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-3-31bf5686028f845040ae39acf642701 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-4-a2c778112718207a10070596cb4595d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a b/sql/hive/src/test/resources/golden/drop_partitions_filter2-5-bc5cbd3b953ca86467c4a5fccd262f0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-6-bbe938b3b20589283cc4541f3e417268 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-7-74ed9df854eae5a025077b7012ef7b97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter2-8-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..226ef460b53a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a b/sql/hive/src/test/resources/golden/drop_partitions_filter2-9-185122a935af4fbe8466d7e39fc7648a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-0-e182f5fbf99c4d9be3fa8c496e0a5994 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-1-e3eda6672f6602a1b9bc93ef789662f6 new file mode 100644 index 0000000000000..01562f65d807f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-10-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..08051a26d24cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-11-57ca78d49ce16d1ebbbc759bad7adfa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-12-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..7f6e4ae8abf83 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c b/sql/hive/src/test/resources/golden/drop_partitions_filter3-13-67d75c72ea2d3982c55f3a850d93f83c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-2-3e8e821dd63112223649b5d06febf7d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec b/sql/hive/src/test/resources/golden/drop_partitions_filter3-3-431228f63002f8b7d1364aa2a07f92ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a b/sql/hive/src/test/resources/golden/drop_partitions_filter3-4-ce613fad87e72c2519c0d01b64f4a99a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-5-9a22b61cd027d740791ba847abc7e072 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae b/sql/hive/src/test/resources/golden/drop_partitions_filter3-6-6127604e4c55b13778cc56c0068ce6ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-7-688620ee5d61cce432e6c2d590b31404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-8-83e3e422cdf9403523fa60d75376d7d7 new file mode 100644 index 0000000000000..226ef460b53a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 b/sql/hive/src/test/resources/golden/drop_partitions_filter3-9-d1a5c03e520fbfa2249e0a32b824a275 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-0-11d8788cb803a84dd4ca3b4103a8bcb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-1-864c8c804db30687f4265ba081ca6368 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-2-446c2380269a60eb4710d7dbeb7c2ec6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-3-312aa26fdea6da7907e3a91f75e36f1 new file mode 100644 index 0000000000000..8caab1c99b27d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-4-11f9ef9dd235f44b29c77abf7ca27881 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 b/sql/hive/src/test/resources/golden/drop_partitions_ignore_protection-5-1283e970d6dc39e7a86e86af904bf116 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_table-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 b/sql/hive/src/test/resources/golden/drop_table-1-afec243db5bd3a1b65d961e2325c6a57 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 b/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_table2-0-ac5bb9a5583f2d8968f2aaef3385b3f2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa b/sql/hive/src/test/resources/golden/drop_table2-1-35229351a48427cf25b42ac8a61200fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 b/sql/hive/src/test/resources/golden/drop_table2-2-cb72d751d94b6d8518c8d69017b6a293 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f b/sql/hive/src/test/resources/golden/drop_table2-3-5f94efc9e658389a6d63553abd7a517f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 b/sql/hive/src/test/resources/golden/drop_table2-4-1c852531c1e75093c27911b45315ed62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 b/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 new file mode 100644 index 0000000000000..fbe12dca4efc8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_table2-5-1fdd850f6c301619f91eb58c890f2ad4 @@ -0,0 +1,3 @@ +p=p1 +p=p2 +p=p3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 b/sql/hive/src/test/resources/golden/drop_table2-6-120f319d6031395a86c726e43d4ef678 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa b/sql/hive/src/test/resources/golden/drop_table2-7-35229351a48427cf25b42ac8a61200fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 b/sql/hive/src/test/resources/golden/drop_table2-8-1fdd850f6c301619f91eb58c890f2ad4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 b/sql/hive/src/test/resources/golden/drop_table2-9-120f319d6031395a86c726e43d4ef678 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 b/sql/hive/src/test/resources/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/drop_view-0-9a0a48e6e2e5edffb4bcca349c49fa48 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 b/sql/hive/src/test/resources/golden/drop_view-1-70a24b7e47d8527298241bcbec922cf5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-0-2bba07855af8c11899cc6b89f8c0ee02 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-2-d71f115b7d42f6c67de701bf69c617a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-3-b7f2a424f616cfb015937e9ef980277 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-4-d8c93b46de2a09a4af12017c6ba196f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-5-e394cdeb88f69b4d4b08450680f779b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-6-725ba4225501c1279f593b9c72eaca28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-7-e707e693aa61edf87768fb71f6e936e1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-8-725ba4225501c1279f593b9c72eaca28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 b/sql/hive/src/test/resources/golden/dynamic_partition_skip_default-9-3b57aa58995f862f2713624b50db5b65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 b/sql/hive/src/test/resources/golden/enforce_order-0-9b9af6de0a12a47694e6f93264c2ebf9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d b/sql/hive/src/test/resources/golden/enforce_order-1-633034e3d966737cecf2befc5df1e35d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/enforce_order-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f b/sql/hive/src/test/resources/golden/enforce_order-3-794fc9cf9fb97001efa85f24fde0cd4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 b/sql/hive/src/test/resources/golden/enforce_order-4-3136edd49e681ea21aa35d0836eab65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 b/sql/hive/src/test/resources/golden/enforce_order-5-15d62540b57faa68f58c5c83c3a296c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 b/sql/hive/src/test/resources/golden/enforce_order-6-277e01aa70e41e8cce47236fcbbb36c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf b/sql/hive/src/test/resources/golden/enforce_order-7-5fb418612e7c2201790d6f318c715ccf new file mode 100644 index 0000000000000..8273b7ed19da6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 b/sql/hive/src/test/resources/golden/enforce_order-8-a7cb2e48b7b4fc5b008be3c9a3485314 new file mode 100644 index 0000000000000..045906a29a1da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/escape1-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 b/sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/escape1-1-683124e29877d2c5a96b95c8ddba97b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 b/sql/hive/src/test/resources/golden/escape1-2-395d5a528c5e7235a48b4ac90938e2d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape1-3-4267651148da591da38737028fdbd80 b/sql/hive/src/test/resources/golden/escape1-3-4267651148da591da38737028fdbd80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 b/sql/hive/src/test/resources/golden/escape1-4-9745f8775c680d094a2c7cbeeb9bdf62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape1-5-70729c3d79ded87e884c176138174645 b/sql/hive/src/test/resources/golden/escape1-5-70729c3d79ded87e884c176138174645 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 b/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 new file mode 100644 index 0000000000000..a949a93dfcca6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/escape1-6-134b596abc363f0bfa7f770732ebb960 @@ -0,0 +1 @@ +128 diff --git a/sql/hive/src/test/resources/golden/escape1-7-486585cbb4de5bc908dde4c601dd7c17 b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 b/sql/hive/src/test/resources/golden/escape1-8-910536a438eec89c78bd611b3c4bb7e0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/escape2-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 b/sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/escape2-1-683124e29877d2c5a96b95c8ddba97b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e b/sql/hive/src/test/resources/golden/escape2-10-13884d58efe80bd24862b3c54cb57c6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/escape2-2-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 b/sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/escape2-3-1774adb1085f4ee6782a8dac0735399 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe b/sql/hive/src/test/resources/golden/escape2-4-bf9010ab0f7bc30bd6db771c5b3d0abe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 b/sql/hive/src/test/resources/golden/escape2-5-fd0c88ad6ad131a16d1b78adbea65800 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 b/sql/hive/src/test/resources/golden/escape2-6-9745f8775c680d094a2c7cbeeb9bdf62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape2-7-70729c3d79ded87e884c176138174645 b/sql/hive/src/test/resources/golden/escape2-7-70729c3d79ded87e884c176138174645 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 b/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 new file mode 100644 index 0000000000000..a949a93dfcca6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/escape2-8-134b596abc363f0bfa7f770732ebb960 @@ -0,0 +1 @@ +128 diff --git a/sql/hive/src/test/resources/golden/escape2-9-486585cbb4de5bc908dde4c601dd7c17 b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf b/sql/hive/src/test/resources/golden/escape_clusterby1-0-e34202f0d36c00a68722c802139d17cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 b/sql/hive/src/test/resources/golden/escape_clusterby1-1-914091aa635a64c707c69e296eb097a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb b/sql/hive/src/test/resources/golden/escape_distributeby1-0-1f178ec5a7ea27b365012c751f3a9cdb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 b/sql/hive/src/test/resources/golden/escape_distributeby1-1-f17d649f307c8c827bf2d136c5d02e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 b/sql/hive/src/test/resources/golden/escape_orderby1-0-4057beace528a415308f7ca332f29941 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 b/sql/hive/src/test/resources/golden/escape_orderby1-1-9c36a3f0e645466b4ebaf2b0f83bc568 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 b/sql/hive/src/test/resources/golden/escape_sortby1-0-e9ca3a2551a33c710e1759517af3d5b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 b/sql/hive/src/test/resources/golden/escape_sortby1-1-bb5ad94d261df75e195d3051a4634d99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-4-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_00_nonpart_empty-5-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_01_nonpart-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_01_nonpart-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_01_nonpart-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/exim_01_nonpart-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_01_nonpart-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_01_nonpart-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_01_nonpart-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-4-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_00_part_empty-5-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_02_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_02_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_part-5-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_02_part-6-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_03_nonpart_over_compat-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_all_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_all_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_all_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_04_all_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_04_all_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_04_all_part-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_04_all_part-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_04_all_part-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_all_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_all_part-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-10-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-11-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-3-c148cf39c4f8f02d44964cfd6919fa29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-4-3c27502d4f6977b959e0928755b43be3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-5-2d68fb88796f07b4bbe2b6895464ee62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-6-10c086eb97e0cae9a2d4b79d90925e85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-7-bff74501ebaea5bd2227d029003dbe08 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-8-9c8594d9438bbceaa3e6c6f98278cf60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 b/sql/hive/src/test/resources/golden/exim_04_evolved_parts-9-18a8affd3b07841b7cbe64003470a9f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_05_some_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_05_some_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_05_some_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_05_some_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_05_some_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_05_some_part-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_05_some_part-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_05_some_part-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_05_some_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_05_some_part-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_06_one_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_06_one_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_06_one_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_06_one_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_06_one_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_06_one_part-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_06_one_part-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_06_one_part-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_06_one_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_06_one_part-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_07_all_part_over_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-2-ad7877a96aba7cd6e29edc19f4f0b394 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_08_nonpart_rename-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_09_part_spec_nonoverlap-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_10_external_managed-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_10_external_managed-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_10_external_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d b/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_10_external_managed-3-be31972099603addb71187f19f7cd25d @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 b/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_10_external_managed-4-46667daf88f9c8b9b758ced6a6b28ef1 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_11_managed_external-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_11_managed_external-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_11_managed_external-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/exim_11_managed_external-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_11_managed_external-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_11_managed_external-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_11_managed_external-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_12_external_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_12_external_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_12_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/exim_12_external_location-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_12_external_location-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_12_external_location-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_12_external_location-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_13_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_13_managed_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_13_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/exim_13_managed_location-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_13_managed_location-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_13_managed_location-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_13_managed_location-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-3-4f20db97105fb03ad21ffbf3edab7b77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_14_managed_location_over_existing-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_15_external_part-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_15_external_part-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_15_external_part-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_15_external_part-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_15_external_part-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_15_external_part-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_15_external_part-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_15_external_part-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_15_external_part-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_15_external_part-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_16_part_external-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_16_part_external-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_16_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_16_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_16_part_external-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_16_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_16_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_16_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_16_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_16_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_17_part_managed-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_17_part_managed-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_17_part_managed-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_17_part_managed-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_17_part_managed-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_17_part_managed-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_17_part_managed-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_17_part_managed-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_17_part_managed-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_17_part_managed-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_18_part_external-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_18_part_external-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_18_part_external-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_18_part_external-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_18_part_external-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_18_part_external-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_18_part_external-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_18_part_external-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_18_part_external-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_18_part_external-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-5-75f428bb2aa8624ac08095cdfd7a6993 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-6-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_00_part_external_location-7-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_part_external_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_part_external_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_part_external_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_19_part_external_location-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_19_part_external_location-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_part_external_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_19_part_external_location-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-5-93aba23b0fa5247d2ed67e5fa976bc0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-6-a14fc179cf3755a0aa7e63d4a514d394 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-7-308a4e8e07efb2b777d9c7de5abab1d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-8-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_20_part_managed_location-9-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-2-f5d86ed3cbc46bb0c7619703081d5873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-3-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-4-7368973ec1870dd3b237c37eb3857b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-5-760e902318ec521eed07cf23e0f256a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-6-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_21_export_authsuccess-7-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-5-75eed21390055f8e397c81ab9d253a32 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_22_import_exist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-3-3430d89fb70985e8a62fb19aa280f2e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-4-88b581725ecdd603117a1706ab9c34dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-5-677ddd4b14eb6f19cfcf0c3d57f54e22 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_23_import_part_authsuccess-6-a9f93b4185a714e4f6d14171d10a6c07 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-1-baeaf0da490037e7ada642d23013075a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-2-e6e650bf4c6291ee2d78e5af5b60e906 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-3-f5d86ed3cbc46bb0c7619703081d5873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-4-9fb7c47b98513bf3355e077ee9732cdd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-5-d2ec90909f243a767be1aa299720f45d @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/exim_24_import_nonexist_authsuccess-6-2a161bb8d69da9d6e3679ca3677a0de5 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 b/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/fetch_aggregation-0-739356d010a919138911f295fac81607 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 b/sql/hive/src/test/resources/golden/fetch_aggregation-1-8dc96d77daa18d053fab3c134028788 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 b/sql/hive/src/test/resources/golden/fetch_aggregation-2-f015c961b6c34ac56cb8fc52fb36d7c7 new file mode 100644 index 0000000000000..fb12634ea81c1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 b/sql/hive/src/test/resources/golden/fileformat_mix-0-c6dff7eb0a793f9cd555164d23eda699 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b b/sql/hive/src/test/resources/golden/fileformat_mix-1-9fa0ea19c0cb6ccef1b4bf9519d8a01b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e b/sql/hive/src/test/resources/golden/fileformat_mix-2-701660c0ea117b11d12de54dc661bc3e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 b/sql/hive/src/test/resources/golden/fileformat_mix-3-2b2316f235737a3f9a30fb05a082e132 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 b/sql/hive/src/test/resources/golden/fileformat_mix-4-fcda187f1366ff93a113cbe670335198 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 b/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/fileformat_mix-5-c2d0da9a0f01736a2163c99fc667f279 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 b/sql/hive/src/test/resources/golden/fileformat_mix-6-4b658b3222b7a09ef41d023215e5b818 new file mode 100644 index 0000000000000..d23e05acf7ba5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 b/sql/hive/src/test/resources/golden/filter_join_breaktask-0-5090cca7feb54de047bc535e234bd4a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 b/sql/hive/src/test/resources/golden/filter_join_breaktask-1-3e3504c67d1ef47c71ea661f647ac4a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 b/sql/hive/src/test/resources/golden/filter_join_breaktask-2-aa61f4a2dd974ae0df026de640ed5802 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e b/sql/hive/src/test/resources/golden/filter_join_breaktask-3-550e9b759fb088a81feddeff2e4be64e new file mode 100644 index 0000000000000..cc545367b951b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-0-db1a6e42b4f880b00b389ae21c7658e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd b/sql/hive/src/test/resources/golden/filter_join_breaktask2-1-4a7480781402d0ac0a856a46ca3883fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd b/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd new file mode 100644 index 0000000000000..140c6590a27fc --- /dev/null +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-10-8dbe2f8cda7ab38b38fc75d0d2413efd @@ -0,0 +1 @@ +4 5 0 2010-04-17 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-11-5cc7e24257a1cb4ad0f2fe41c7177370 new file mode 100644 index 0000000000000..51f5701ceae46 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 new file mode 100644 index 0000000000000..69dcc336a73f2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/filter_join_breaktask2-12-c608f51882260caec7bb9c57a0ba3179 @@ -0,0 +1 @@ +5 5 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c b/sql/hive/src/test/resources/golden/filter_join_breaktask2-2-b8486987aee5bac5f5b7301952e67d0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-3-1139b5e7f76614bc03bf0db677ed7d73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-4-7accb0b0e00dcfd6468a6ff6058bb4e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba b/sql/hive/src/test/resources/golden/filter_join_breaktask2-5-25f891cf5e9138c14ba8c385c83230ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-6-4aaf479be27e3f8b38465ea946b530e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-7-571467e86d08833eecf5d3e67b41bba6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e b/sql/hive/src/test/resources/golden/filter_join_breaktask2-8-604cf64d16b9d438ee187a10d9f6352e new file mode 100644 index 0000000000000..c0f577c3cd2fd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 b/sql/hive/src/test/resources/golden/filter_join_breaktask2-9-6d551990a8a745dde1cb5244947b81a1 new file mode 100644 index 0000000000000..e426b4879bcb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1-2-f90acd59ace31e16059bae52583188cc b/sql/hive/src/test/resources/golden/groupby1-2-f90acd59ace31e16059bae52583188cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d b/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1-3-d57ed4bbfee1ffaffaeba0a4be84c31d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 b/sql/hive/src/test/resources/golden/groupby1-4-19094f229f8af852ef8dad406333ae08 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 b/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1-5-dd7bf298b8c921355edd8665c6b0c168 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea b/sql/hive/src/test/resources/golden/groupby1-6-977359ea0d2f429a9a35bbd8599cddea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 b/sql/hive/src/test/resources/golden/groupby1-7-c2c54378ffce53ade73a9dda783903e7 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby11-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby11-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 b/sql/hive/src/test/resources/golden/groupby11-2-f06bf6a360c7f9425cffa237f00445d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 b/sql/hive/src/test/resources/golden/groupby11-3-f11e9a986df49215fb8aa17aaccc7087 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 b/sql/hive/src/test/resources/golden/groupby11-4-1ae3e153888f9ca44e92ef43aea19092 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f b/sql/hive/src/test/resources/golden/groupby11-5-a6d0a37db950e5d309ef2b89e9cffe0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd b/sql/hive/src/test/resources/golden/groupby11-6-686ee4a5b2e24e51ba1d41b42215accd new file mode 100644 index 0000000000000..8939d9f8af186 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d b/sql/hive/src/test/resources/golden/groupby11-7-149d359546ab38226ffeb023d7414b3d new file mode 100644 index 0000000000000..82a0329b4d459 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_limit-0-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 b/sql/hive/src/test/resources/golden/groupby1_limit-1-647cd470ff311f1879243a6e7f1e7bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 b/sql/hive/src/test/resources/golden/groupby1_limit-2-e8f9567aa6dd201dd22db10fe7e8e082 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 b/sql/hive/src/test/resources/golden/groupby1_limit-3-deff149e2f91d6d605a3dccf1ca8bdd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 b/sql/hive/src/test/resources/golden/groupby1_limit-4-7a5ee36a377f0869bcb8c29c2391caa2 new file mode 100644 index 0000000000000..9cf9606d751ac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 b/sql/hive/src/test/resources/golden/groupby1_map-3-647cd470ff311f1879243a6e7f1e7bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe b/sql/hive/src/test/resources/golden/groupby1_map-4-330a40b8e19028bbb370adf219f469fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 b/sql/hive/src/test/resources/golden/groupby1_map-5-40f5168bfd9d124165bd207543b68a28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map_nomap-2-fb2713a701e7e9c3fa36b5459d772f4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-3-647cd470ff311f1879243a6e7f1e7bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe b/sql/hive/src/test/resources/golden/groupby1_map_nomap-4-330a40b8e19028bbb370adf219f469fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 b/sql/hive/src/test/resources/golden/groupby1_map_nomap-5-40f5168bfd9d124165bd207543b68a28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map_nomap-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 b/sql/hive/src/test/resources/golden/groupby1_map_skew-3-647cd470ff311f1879243a6e7f1e7bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe b/sql/hive/src/test/resources/golden/groupby1_map_skew-4-330a40b8e19028bbb370adf219f469fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 b/sql/hive/src/test/resources/golden/groupby1_map_skew-5-40f5168bfd9d124165bd207543b68a28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby1_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby1_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc b/sql/hive/src/test/resources/golden/groupby1_noskew-3-f90acd59ace31e16059bae52583188cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 b/sql/hive/src/test/resources/golden/groupby1_noskew-4-19094f229f8af852ef8dad406333ae08 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea b/sql/hive/src/test/resources/golden/groupby1_noskew-5-977359ea0d2f429a9a35bbd8599cddea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 b/sql/hive/src/test/resources/golden/groupby1_noskew-6-c2c54378ffce53ade73a9dda783903e7 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2-1-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2-2-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b b/sql/hive/src/test/resources/golden/groupby2-3-6b5d354a5a81c0171c3cc8d553bfdb9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 b/sql/hive/src/test/resources/golden/groupby2-4-67a2618eb44e68e6d8cf8792ded802f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 b/sql/hive/src/test/resources/golden/groupby2-5-c0660310e5d882732d07cb76bc0a7939 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2-6-41dfb7b036cae4972b275a0262ea2e4c new file mode 100644 index 0000000000000..3210a3048ff48 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_limit-0-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 b/sql/hive/src/test/resources/golden/groupby2_limit-1-d6db5e2b44be5a3927eab70e4cf60c70 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 b/sql/hive/src/test/resources/golden/groupby2_limit-2-3460130ddc7cd91c0b1d5ce2ff98d0b9 new file mode 100644 index 0000000000000..9cf9606d751ac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 b/sql/hive/src/test/resources/golden/groupby2_map-3-1c0aa77190a5b3e1895e58cfbe7467a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 b/sql/hive/src/test/resources/golden/groupby2_map-4-e193b8c0a9e1731dd46b145d166c78a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 b/sql/hive/src/test/resources/golden/groupby2_map-5-ce0965adac15c4da6526d433d17ebc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..3210a3048ff48 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-3-3aa4057488720c0f514696154f2070b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-4-485f8328cdc8639bcdea5fb9f58c8695 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-5-23478129b7b6e377dcfb7caaa17dfb48 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..922f4bfc83e44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 b/sql/hive/src/test/resources/golden/groupby2_map_skew-3-1c0aa77190a5b3e1895e58cfbe7467a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 b/sql/hive/src/test/resources/golden/groupby2_map_skew-4-e193b8c0a9e1731dd46b145d166c78a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 b/sql/hive/src/test/resources/golden/groupby2_map_skew-5-ce0965adac15c4da6526d433d17ebc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby2_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..3210a3048ff48 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b b/sql/hive/src/test/resources/golden/groupby2_noskew-3-6b5d354a5a81c0171c3cc8d553bfdb9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 b/sql/hive/src/test/resources/golden/groupby2_noskew-4-67a2618eb44e68e6d8cf8792ded802f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 b/sql/hive/src/test/resources/golden/groupby2_noskew-5-c0660310e5d882732d07cb76bc0a7939 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2_noskew-6-41dfb7b036cae4972b275a0262ea2e4c new file mode 100644 index 0000000000000..3210a3048ff48 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-3-f64b982c4bf34931f03447e635ae33d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-4-2115f3e7d207621ce2b07b6e33563844 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-5-92891db0de9b8cd00892f0a790aff494 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c b/sql/hive/src/test/resources/golden/groupby2_noskew_multi_distinct-6-41dfb7b036cae4972b275a0262ea2e4c new file mode 100644 index 0000000000000..922f4bfc83e44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/groupby4-2-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 b/sql/hive/src/test/resources/golden/groupby4-3-132eabb30b8d011c848c531a6ab54529 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 b/sql/hive/src/test/resources/golden/groupby4-4-ac19a9a7f4a16763bfe7998179257933 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 b/sql/hive/src/test/resources/golden/groupby4_map-3-840759680be77463c13e4b19b2c1eb04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b b/sql/hive/src/test/resources/golden/groupby4_map-4-19182d5780c4632c5bf8effdd5c7f36b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 b/sql/hive/src/test/resources/golden/groupby4_map-5-c0117072e2d392e3f860456d0226b7b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 b/sql/hive/src/test/resources/golden/groupby4_map_skew-3-840759680be77463c13e4b19b2c1eb04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b b/sql/hive/src/test/resources/golden/groupby4_map_skew-4-19182d5780c4632c5bf8effdd5c7f36b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 b/sql/hive/src/test/resources/golden/groupby4_map_skew-5-c0117072e2d392e3f860456d0226b7b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby4_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/groupby4_noskew-3-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 b/sql/hive/src/test/resources/golden/groupby4_noskew-4-132eabb30b8d011c848c531a6ab54529 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 b/sql/hive/src/test/resources/golden/groupby4_noskew-5-ac19a9a7f4a16763bfe7998179257933 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby4_noskew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/groupby5-2-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 b/sql/hive/src/test/resources/golden/groupby5-3-a773aeb40af0516f2527f8e9d6907420 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e b/sql/hive/src/test/resources/golden/groupby5-4-c4570c2676d599793e1e9ece32aa596e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 b/sql/hive/src/test/resources/golden/groupby5_map-3-840759680be77463c13e4b19b2c1eb04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f b/sql/hive/src/test/resources/golden/groupby5_map-4-7b0346bd80d9833f2eccf8365b53d80f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 b/sql/hive/src/test/resources/golden/groupby5_map-5-586dc4493f66ea612a1d5b3cda89d725 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..349d8b75d942b --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_map-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +130091 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 b/sql/hive/src/test/resources/golden/groupby5_map_skew-3-840759680be77463c13e4b19b2c1eb04 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f b/sql/hive/src/test/resources/golden/groupby5_map_skew-4-7b0346bd80d9833f2eccf8365b53d80f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 b/sql/hive/src/test/resources/golden/groupby5_map_skew-5-586dc4493f66ea612a1d5b3cda89d725 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..349d8b75d942b --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +130091 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby5_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/groupby5_noskew-3-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 b/sql/hive/src/test/resources/golden/groupby5_noskew-4-a773aeb40af0516f2527f8e9d6907420 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e b/sql/hive/src/test/resources/golden/groupby5_noskew-5-c4570c2676d599793e1e9ece32aa596e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby5_noskew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/groupby6-2-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac b/sql/hive/src/test/resources/golden/groupby6-3-ae61517a9997b80d512a9089cdb71fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 b/sql/hive/src/test/resources/golden/groupby6-4-3f468a119e7975110b4063adb42c7dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/groupby6_map-3-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac b/sql/hive/src/test/resources/golden/groupby6_map-4-ae61517a9997b80d512a9089cdb71fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 b/sql/hive/src/test/resources/golden/groupby6_map-5-3f468a119e7975110b4063adb42c7dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_map-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/groupby6_map_skew-3-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac b/sql/hive/src/test/resources/golden/groupby6_map_skew-4-ae61517a9997b80d512a9089cdb71fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 b/sql/hive/src/test/resources/golden/groupby6_map_skew-5-3f468a119e7975110b4063adb42c7dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_map_skew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby6_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/groupby6_noskew-3-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac b/sql/hive/src/test/resources/golden/groupby6_noskew-4-ae61517a9997b80d512a9089cdb71fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 b/sql/hive/src/test/resources/golden/groupby6_noskew-5-3f468a119e7975110b4063adb42c7dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby6_noskew-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f55b5c9eef39f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby7-2-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby7-3-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7-4-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7-5-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 b/sql/hive/src/test/resources/golden/groupby7-6-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7-8-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-1-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map-11-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-2-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-3-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby7_map-4-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby7_map-5-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map-7-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd b/sql/hive/src/test/resources/golden/groupby7_map-8-409f355bf35dfaa8b9e643510c58fabd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 b/sql/hive/src/test/resources/golden/groupby7_map-9-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-10-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-7-409f355bf35dfaa8b9e643510c58fabd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-8-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map_multi_single_reducer-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_map_skew-10-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby7_map_skew-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby7_map_skew-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_map_skew-6-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd b/sql/hive/src/test/resources/golden/groupby7_map_skew-7-409f355bf35dfaa8b9e643510c58fabd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 b/sql/hive/src/test/resources/golden/groupby7_map_skew-8-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_map_skew-9-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-1-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby7_noskew-10-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby7_noskew-11-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..339756be98e73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-2-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-3-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby7_noskew-4-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby7_noskew-5-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-6-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew-7-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd b/sql/hive/src/test/resources/golden/groupby7_noskew-8-409f355bf35dfaa8b9e643510c58fabd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 b/sql/hive/src/test/resources/golden/groupby7_noskew-9-b2af91348c5fa9605702be50983c3bd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-10-96d0598a2a4c17041a31b908d1f065e5 new file mode 100644 index 0000000000000..dfca4e169cbe8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-5-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-6-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-7-186e2b8dc9a393a8bd8c47a303f7f471 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-8-e404ba29e723df8bd8662d4f48129c7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 b/sql/hive/src/test/resources/golden/groupby7_noskew_multi_single_reducer-9-652510b8dc20117c65511f06e6e73d73 new file mode 100644 index 0000000000000..dfca4e169cbe8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f b/sql/hive/src/test/resources/golden/groupby8-10-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8-11-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8-12-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby8-2-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby8-3-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d b/sql/hive/src/test/resources/golden/groupby8-4-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f b/sql/hive/src/test/resources/golden/groupby8-5-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8-6-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8-7-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8-8-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d b/sql/hive/src/test/resources/golden/groupby8-9-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby8_map-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby8_map-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d b/sql/hive/src/test/resources/golden/groupby8_map-5-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f b/sql/hive/src/test/resources/golden/groupby8_map-6-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_map-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_map-8-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_map_skew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby8_map_skew-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby8_map_skew-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d b/sql/hive/src/test/resources/golden/groupby8_map_skew-5-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f b/sql/hive/src/test/resources/golden/groupby8_map_skew-6-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_map_skew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_map_skew-8-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby8_noskew-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby8_noskew-3-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a b/sql/hive/src/test/resources/golden/groupby8_noskew-4-95474af63c0f92717ed49c3a0f37b10a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d b/sql/hive/src/test/resources/golden/groupby8_noskew-5-a0a22ec83d6632cba3f17d79dbb9494d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f b/sql/hive/src/test/resources/golden/groupby8_noskew-6-83296fd5b5fc22af0c51514c4e67c95f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby8_noskew-7-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby8_noskew-8-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 b/sql/hive/src/test/resources/golden/groupby9-0-3678fb67b5c739bd87d4907630da1208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 b/sql/hive/src/test/resources/golden/groupby9-1-6ffcd6fad9ca3e9934f521673f5039a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby9-10-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f b/sql/hive/src/test/resources/golden/groupby9-11-a4e1a4b250c160f9b90b12a2e7c5e82f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 b/sql/hive/src/test/resources/golden/groupby9-12-9f4c2e7d95494bcdc7c4ed19e0434de6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-13-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-14-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..0e40f3f1a49a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed b/sql/hive/src/test/resources/golden/groupby9-15-c3d5e2da9bbf7d66aa106eb13183dbed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b b/sql/hive/src/test/resources/golden/groupby9-16-c05b1bc66a607e43633dc457ecf48f3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-17-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-18-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..cf0c065dc6532 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e b/sql/hive/src/test/resources/golden/groupby9-19-55781df7ed3ff9e37001fbd7739b9c2e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f b/sql/hive/src/test/resources/golden/groupby9-2-a4e1a4b250c160f9b90b12a2e7c5e82f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 b/sql/hive/src/test/resources/golden/groupby9-20-ccc5914317422f569e8b7171a3b2b243 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-21-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-22-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..4ef88d57aa72b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 b/sql/hive/src/test/resources/golden/groupby9-3-9f4c2e7d95494bcdc7c4ed19e0434de6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-5-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..0e40f3f1a49a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e b/sql/hive/src/test/resources/golden/groupby9-6-55781df7ed3ff9e37001fbd7739b9c2e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 b/sql/hive/src/test/resources/golden/groupby9-7-ccc5914317422f569e8b7171a3b2b243 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/groupby9-8-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..326493a1d8cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/groupby9-9-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..4ef88d57aa72b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 b/sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_bigdata-1-1100bb0c115c024998d35888ae5bbd71 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_bigdata-2-3688b45adbdb190d58799c0b6d601055 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 b/sql/hive/src/test/resources/golden/groupby_complex_types-0-de39f8b5f4305136d945da94d5222283 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 b/sql/hive/src/test/resources/golden/groupby_complex_types-1-10fe6134247226ab2b309bb62460d080 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a b/sql/hive/src/test/resources/golden/groupby_complex_types-2-2500cd8e85b71222253a05a979442a4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 b/sql/hive/src/test/resources/golden/groupby_complex_types-3-85850072f0f9604d4e55a08fb9c45ba6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 b/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-1-de39f8b5f4305136d945da94d5222283 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 b/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-2-10fe6134247226ab2b309bb62460d080 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 b/sql/hive/src/test/resources/golden/groupby_complex_types_multi_single_reducer-3-28264b197e3997f7c5fe88c1c2f7d5c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-0-63603572a0f759ea81f4649ae2210ef8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-1-a3f3e279ab0be5093f4a926e265c0211 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-2-59ee4550803e419b3adb1e9dd4220113 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-3-12b0749f4fb915f3b2e317ed4fbc9cb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-4-d88c962262d3524bfc881b7309688e38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad new file mode 100644 index 0000000000000..917d19a8f8cd8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-5-78c0b7be08118a14e0337ff552fcb9ad @@ -0,0 +1,4 @@ +0 0 +2 2 +4 4 +5 5 diff --git a/sql/hive/src/test/resources/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 new file mode 100644 index 0000000000000..917d19a8f8cd8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-6-edcbea736edb6104a73f0dc670418ee5 @@ -0,0 +1,4 @@ +0 0 +2 2 +4 4 +5 5 diff --git a/sql/hive/src/test/resources/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 b/sql/hive/src/test/resources/golden/groupby_distinct_samekey-7-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 b/sql/hive/src/test/resources/golden/groupby_map_ppr-3-1c0aa77190a5b3e1895e58cfbe7467a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 b/sql/hive/src/test/resources/golden/groupby_map_ppr-4-e5121a2b8210b2e4f8b1bfbf0a044486 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a b/sql/hive/src/test/resources/golden/groupby_map_ppr-5-d4faa22fc1ec8cfc8ab21474841d415a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_map_ppr-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..ae9bbc3e2c2f7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-2-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-3-a04c523002e79c588e350486c815e785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-4-d6c134387c0c9343ec6ea88b5086dbe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-5-2efbb90d3df011282b70623e0905c390 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_map_ppr_multi_distinct-6-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..64bb7c62c1885 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-1-388618d4d475de38d5c280fd7038730b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-10-6aa66df624cd32601218eee200977ce6 new file mode 100644 index 0000000000000..6b6a788e382db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-11-59a77127c166fef57504c011ccb427a6 new file mode 100644 index 0000000000000..c33eed60c8d5b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-2-2e1779fc58da364612a1c84d563ea7d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-3-febd68f62dd71550dd3db8335d1f93f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-4-c2781ed9ce9a29f484f7648ce6e06a9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-5-6aa66df624cd32601218eee200977ce6 new file mode 100644 index 0000000000000..6b6a788e382db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-6-59a77127c166fef57504c011ccb427a6 new file mode 100644 index 0000000000000..c33eed60c8d5b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-7-293182ac89effb268855f5ac53e1ec11 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-8-51d6bc83dcb6610b5b4f350cbaf25d29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e b/sql/hive/src/test/resources/golden/groupby_multi_insert_common_distinct-9-c2781ed9ce9a29f484f7648ce6e06a9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-0-b31bf66c43bb9f7ddc09b138b7bf36e0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-1-f64b982c4bf34931f03447e635ae33d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-10-3b3ef4d975716744c85af560aa240abd new file mode 100644 index 0000000000000..922f4bfc83e44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-11-4e95946ec07f04479da42ba5cbfa531b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-12-51fe5b5a17ddc56fb8712340b38773b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-13-38f492067df78144c272bb212633cc5e new file mode 100644 index 0000000000000..106132fc993d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-14-d7b3676444a9e95811184637dd0b3231 new file mode 100644 index 0000000000000..65235356ea425 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-15-3b3ef4d975716744c85af560aa240abd new file mode 100644 index 0000000000000..922f4bfc83e44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-16-b21ae88a5fc7f9b5c25ca954889ee421 new file mode 100644 index 0000000000000..017878bc9bee5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-17-7f344c2f066d74ecd135c43d39658bae new file mode 100644 index 0000000000000..f21a658e3c68f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-18-4938ddc6b516cf67779be0d7dc29e7ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-19-180b611e08d4080fa098ed69169c2478 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-2-392062177be62090adedf1ab6c0a0b78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-20-db2149b42cdbf998187034233fc846cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-21-322f23866cf3ca62d4ba93cf904c520 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-22-21fcbcafc3b5b22572ddd5c87df1c06f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-3-82b65775ae9c6dcb85ca87804dbb0288 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-4-7df7c16e4063683d0ca40360da460799 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-5-c19e7bf0732e7a103966ebc43eb0bcea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-6-3be7ae6a87b9091bc61d221de8bdd55b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-7-ad39c0f14b6e0752489479138516bd57 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-8-38f492067df78144c272bb212633cc5e new file mode 100644 index 0000000000000..106132fc993d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer-9-d7b3676444a9e95811184637dd0b3231 new file mode 100644 index 0000000000000..65235356ea425 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-0-b31bf66c43bb9f7ddc09b138b7bf36e0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-1-660d3ff0ca588c2da097b5f3ca753ada new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-2-7372ea92a15e7beed3be5e2dd2cbac47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-3-eeaa92b3f8e6667d3395f8dc7ea6d89e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-4-427627f4cf6d91e5314c85e0c5aa2f84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-5-376542befbcab97d864e874251720c40 new file mode 100644 index 0000000000000..dbc7aebdbbae5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-6-8566e4c94d34b0fc4a71d34e89529c74 new file mode 100644 index 0000000000000..ebf9d6978dec6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-7-4938ddc6b516cf67779be0d7dc29e7ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer2-8-180b611e08d4080fa098ed69169c2478 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-0-36cc74ebca5adb026757c5cd8df5a0dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-1-9e61989d717403353689cbbb2816210d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-10-73819ea1a7c0653a61652b3766afb003 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-11-8603afa3bf3deeda532fc69b7df49e09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-12-2ab5cc10c0b37e7cf3c0d33fdc39b628 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..a5ae9e2a62227 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-13-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,2 @@ +100 2 +200 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..16c7a647a2344 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-14-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +400 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-15-36e81eba0a6a42532d8ee147086d668a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-16-59b2c947e136092e7ca5019c96a9994b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-17-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..67f077e51a647 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..b06ad20135fbe --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-18-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +val_200 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-2-8603afa3bf3deeda532fc69b7df49e09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-3-2ab5cc10c0b37e7cf3c0d33fdc39b628 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..a5ae9e2a62227 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-4-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1,2 @@ +100 2 +200 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..16c7a647a2344 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-5-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +400 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-6-36e81eba0a6a42532d8ee147086d668a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-7-59b2c947e136092e7ca5019c96a9994b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-8-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..67f077e51a647 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..b06ad20135fbe --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_multi_single_reducer3-9-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +val_200 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb b/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb new file mode 100644 index 0000000000000..efe5cc7795c65 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_neg_float-0-dd44874fbefeee1ed0a0dd8bfbab96eb @@ -0,0 +1 @@ +-30.33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 b/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 new file mode 100644 index 0000000000000..efe5cc7795c65 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_neg_float-1-4a435e268a327404f75725b82a32ee03 @@ -0,0 +1 @@ +-30.33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 b/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_position-0-422c2068a838f59324c1d9861225c824 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 b/sql/hive/src/test/resources/golden/groupby_position-1-6b06902de5c0ca13cebe03018d86f447 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 b/sql/hive/src/test/resources/golden/groupby_position-10-b4c6c0aedcc924e2af65549b87b3f3f7 new file mode 100644 index 0000000000000..f2a91fe3bfab0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 b/sql/hive/src/test/resources/golden/groupby_position-11-75a28e558d2fb7a78f43f55b0479c646 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f b/sql/hive/src/test/resources/golden/groupby_position-12-5583e5cfcf8083d45a3bd80434c1829f new file mode 100644 index 0000000000000..c8e666cb01e8e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 b/sql/hive/src/test/resources/golden/groupby_position-13-9ee2150594ad2eece6ee14424155d396 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb b/sql/hive/src/test/resources/golden/groupby_position-14-6f54558123eb5d63eeeb3e261c1dc5fb new file mode 100644 index 0000000000000..96a824a81c589 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 b/sql/hive/src/test/resources/golden/groupby_position-2-627bb7be9c0edb5ba4c677912800d364 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac b/sql/hive/src/test/resources/golden/groupby_position-3-c39bd6c3c24658ec082bef9876d6e2ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c b/sql/hive/src/test/resources/golden/groupby_position-4-19461cbd2be1f2f3f3b65478e3eeb53c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d b/sql/hive/src/test/resources/golden/groupby_position-5-e2470670c5d709001fb17ecbc6e4f85d new file mode 100644 index 0000000000000..a79396dac079a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 b/sql/hive/src/test/resources/golden/groupby_position-6-b4c6c0aedcc924e2af65549b87b3f3f7 new file mode 100644 index 0000000000000..f2a91fe3bfab0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 b/sql/hive/src/test/resources/golden/groupby_position-7-5b32a45af11e04b46f8566bd27a28014 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c b/sql/hive/src/test/resources/golden/groupby_position-8-3ff6c255dda69ec117c6a7ec93db2f1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d b/sql/hive/src/test/resources/golden/groupby_position-9-e2470670c5d709001fb17ecbc6e4f85d new file mode 100644 index 0000000000000..a79396dac079a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_ppr-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_ppr-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 b/sql/hive/src/test/resources/golden/groupby_ppr-2-1c0aa77190a5b3e1895e58cfbe7467a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 b/sql/hive/src/test/resources/golden/groupby_ppr-3-e5121a2b8210b2e4f8b1bfbf0a044486 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a b/sql/hive/src/test/resources/golden/groupby_ppr-4-d4faa22fc1ec8cfc8ab21474841d415a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_ppr-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..ae9bbc3e2c2f7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-0-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-2-3aa4057488720c0f514696154f2070b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-3-ace7b2624b125764e9f1f6b5559f023d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-4-61ae2f9b3a9a2f60a307aa25e42425d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/groupby_ppr_multi_distinct-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..64bb7c62c1885 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-10-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 b/sql/hive/src/test/resources/golden/groupby_sort_1-11-c166396bbdb62490f56ab0bc84aaa7d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c b/sql/hive/src/test/resources/golden/groupby_sort_1-12-c8a51e8a269da4c4ae98ac105a573f3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 b/sql/hive/src/test/resources/golden/groupby_sort_1-13-46c4a3675c8de0510b648856a193f3e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 b/sql/hive/src/test/resources/golden/groupby_sort_1-14-bbdd53118f788d7bb679d094c41243c8 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea b/sql/hive/src/test/resources/golden/groupby_sort_1-15-54f3a2a97939d3eca8a601b74ef30dea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 b/sql/hive/src/test/resources/golden/groupby_sort_1-16-236d51792f4913b5858f367e3cff2c60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-17-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 b/sql/hive/src/test/resources/golden/groupby_sort_1-18-1e77dcdd6f54334dbae6a4d11ad6ff64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 b/sql/hive/src/test/resources/golden/groupby_sort_1-19-90c84358c50e51be5ce210bd7dec9bc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-20-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 b/sql/hive/src/test/resources/golden/groupby_sort_1-21-27e2e662d517f32952145cc2a51bf564 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 b/sql/hive/src/test/resources/golden/groupby_sort_1-22-6775cb6aee040e22508cf3cac0b55f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 b/sql/hive/src/test/resources/golden/groupby_sort_1-23-eb2b625279d8794390d7e2dc8f2dc907 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_1-24-d53196339980a00a619788bd799a32e7 new file mode 100644 index 0000000000000..c1cc4ee204773 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 b/sql/hive/src/test/resources/golden/groupby_sort_1-25-138e93f004f7bd16e63bbf8d9090af21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 b/sql/hive/src/test/resources/golden/groupby_sort_1-26-93153fd67c1d19bb9ad64f98294e4981 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f b/sql/hive/src/test/resources/golden/groupby_sort_1-27-ca369ab23c32070e2d42ba8df036175f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-28-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 b/sql/hive/src/test/resources/golden/groupby_sort_1-29-4095695e88e23dd42acb1749a83bdbb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 b/sql/hive/src/test/resources/golden/groupby_sort_1-30-d9ba55c20c3f5df262e81cbf5dab5387 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_1-31-d53196339980a00a619788bd799a32e7 new file mode 100644 index 0000000000000..f0192040e147b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f b/sql/hive/src/test/resources/golden/groupby_sort_1-32-c88ee457dee7bb141a198a2ae39d787f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef b/sql/hive/src/test/resources/golden/groupby_sort_1-33-c04167e880fe3e942ce77e75d660f1ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-34-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..b6c2eb98e5e49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 b/sql/hive/src/test/resources/golden/groupby_sort_1-35-deb3f1793f51d1edf011a8405abf4968 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 b/sql/hive/src/test/resources/golden/groupby_sort_1-36-7871722f392f801a868e0e2fb372c610 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-37-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..8e7ee8a2b47bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d b/sql/hive/src/test/resources/golden/groupby_sort_1-38-b71bdaa2b7c4b5c51a9773c123e5306d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b b/sql/hive/src/test/resources/golden/groupby_sort_1-39-89aa7bab4272546e06cd7e504380d96b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad b/sql/hive/src/test/resources/golden/groupby_sort_1-4-e906be6d27c9dfcffd4af171541639ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-40-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..0b2b54cd94e4f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 b/sql/hive/src/test/resources/golden/groupby_sort_1-41-348b0126cb1d214fea58d4af9d3dbf67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c b/sql/hive/src/test/resources/golden/groupby_sort_1-42-83889e7dc73d796cc869160b6b35102c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-43-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..3d4708b7c9d64 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a b/sql/hive/src/test/resources/golden/groupby_sort_1-44-41462b2e60bf44571a7b1fb435374d6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 b/sql/hive/src/test/resources/golden/groupby_sort_1-45-8aea6edf0481e2a10f14879acba62648 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 b/sql/hive/src/test/resources/golden/groupby_sort_1-46-4999db9eb86d9455c1d75e97b052f279 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa b/sql/hive/src/test/resources/golden/groupby_sort_1-47-fecb9f2203aeb2ac4b693a97badde6fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 b/sql/hive/src/test/resources/golden/groupby_sort_1-48-f0f18d5fa2824735799edc4bdeb1afb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_1-49-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_1-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 b/sql/hive/src/test/resources/golden/groupby_sort_1-50-461847b174096e7a255fb07cb35ab434 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 b/sql/hive/src/test/resources/golden/groupby_sort_1-51-8da967e7c2210ad044ba8b08d1685065 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-52-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 b/sql/hive/src/test/resources/golden/groupby_sort_1-53-fdc295aaefba56548a22dfcddc2a94f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c b/sql/hive/src/test/resources/golden/groupby_sort_1-54-74bfe3fbf9d68a75013fba1c3c7bbd7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 b/sql/hive/src/test/resources/golden/groupby_sort_1-55-1013d1ad014aa203b1dce26085b09c01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 b/sql/hive/src/test/resources/golden/groupby_sort_1-56-5373c5449884d95bc7db9dab55239a49 new file mode 100644 index 0000000000000..7ca6b0b28a960 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 b/sql/hive/src/test/resources/golden/groupby_sort_1-57-825135844e8ac6d8d5254cc961ec8fd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 b/sql/hive/src/test/resources/golden/groupby_sort_1-58-e671e63f6b70094048563a9c33748c97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-59-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/groupby_sort_1-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 b/sql/hive/src/test/resources/golden/groupby_sort_1-60-51824b04f2a008f63e1469695e60d9c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 b/sql/hive/src/test/resources/golden/groupby_sort_1-61-18b0757f6d9e29808061ca3763b8b6d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_1-62-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..58e16ef3c0ef3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-63-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-64-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-65-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 b/sql/hive/src/test/resources/golden/groupby_sort_1-66-b4c5b3eeb74085711866a2eec27bcb37 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 b/sql/hive/src/test/resources/golden/groupby_sort_1-67-b4fec0996399be2239961594897d6715 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-68-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-69-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/groupby_sort_1-7-c0ea81b686236d661166912040a16ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc b/sql/hive/src/test/resources/golden/groupby_sort_1-70-2d5403943a3efdf9fd3eccb6927499cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e b/sql/hive/src/test/resources/golden/groupby_sort_1-71-ca16ff548ebb9bab4b76f2e339064a9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_1-72-d2fa5e7bdd6b7934d10d5905cacd5715 new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd b/sql/hive/src/test/resources/golden/groupby_sort_1-73-6296dde4e71acf7e7f42ee58cf3b5acd new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 b/sql/hive/src/test/resources/golden/groupby_sort_1-74-6f7caef1c773268350c9cf94ad85be01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 b/sql/hive/src/test/resources/golden/groupby_sort_1-75-4931c5a72a5231f67317d27ca025bb97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 new file mode 100644 index 0000000000000..d15db8c5d079f --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_1-76-d2fa5e7bdd6b7934d10d5905cacd5715 @@ -0,0 +1 @@ +8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 b/sql/hive/src/test/resources/golden/groupby_sort_1-77-6d6eac558569563cc47a657daffa8ba7 new file mode 100644 index 0000000000000..3d1609d961673 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 b/sql/hive/src/test/resources/golden/groupby_sort_1-8-f0ee61903aeacb758e2eada242e5e14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 b/sql/hive/src/test/resources/golden/groupby_sort_1-9-ebc7ac3b2dfdb958d161cd7c8f947a72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 b/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 new file mode 100644 index 0000000000000..c6bb9dbfd6497 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-10-b4e225bc4787d7718bae6f00d8addfe2 @@ -0,0 +1,2 @@ +0 +11 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 b/sql/hive/src/test/resources/golden/groupby_sort_10-11-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e b/sql/hive/src/test/resources/golden/groupby_sort_10-4-475d50465b23adfb70e67122425ede9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 b/sql/hive/src/test/resources/golden/groupby_sort_10-5-91f5326fe696124d862dfcfb72de2bf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 b/sql/hive/src/test/resources/golden/groupby_sort_10-6-7da6ce8e3115f054ce532c26fb19bb44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 b/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 new file mode 100644 index 0000000000000..c6bb9dbfd6497 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_10-7-b4e225bc4787d7718bae6f00d8addfe2 @@ -0,0 +1,2 @@ +0 +11 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 b/sql/hive/src/test/resources/golden/groupby_sort_10-8-2c37a0e69aca38f2ce9db4c0aaf94db7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c b/sql/hive/src/test/resources/golden/groupby_sort_10-9-aed4b33d1aeb64e88cf36e29f8d8b6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d b/sql/hive/src/test/resources/golden/groupby_sort_11-10-dc4ffd56a4be5e2c52df2f70c614720d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 b/sql/hive/src/test/resources/golden/groupby_sort_11-11-d9bf7e80b71121935ed4b008ae916cb1 new file mode 100644 index 0000000000000..ded2854cdf564 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 b/sql/hive/src/test/resources/golden/groupby_sort_11-12-56f9f2a13698e71c4f00c93c48dffc30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 b/sql/hive/src/test/resources/golden/groupby_sort_11-13-c7d70702783eb90d4f53028a63c318f8 new file mode 100644 index 0000000000000..487b4c4a5cc6f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 b/sql/hive/src/test/resources/golden/groupby_sort_11-14-d403afd709251edba6ae13b8cc14e0b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 b/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-15-9fd5f5fce6f3821f2d7801eb0e83a015 @@ -0,0 +1 @@ +6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 b/sql/hive/src/test/resources/golden/groupby_sort_11-16-166ae99e823bc1bcbee39a466f9e3551 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 b/sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-17-5298cc011d107ad06c365d132cab6fc9 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-18-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 b/sql/hive/src/test/resources/golden/groupby_sort_11-19-46ca52d697918f2327b2293d9fd57d15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-20-bd99462ed878bf4bec74b3cb9132908d @@ -0,0 +1 @@ +6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e b/sql/hive/src/test/resources/golden/groupby_sort_11-4-475d50465b23adfb70e67122425ede9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 b/sql/hive/src/test/resources/golden/groupby_sort_11-5-7ca5ebad57578206b8830da6746fb952 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 b/sql/hive/src/test/resources/golden/groupby_sort_11-6-1c033f86ad59eb6ab59008d12cd00e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-7-bd99462ed878bf4bec74b3cb9132908d @@ -0,0 +1 @@ +6 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c b/sql/hive/src/test/resources/golden/groupby_sort_11-8-4bf6c5e938afa4f937b69d2a095c675c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 b/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 new file mode 100644 index 0000000000000..6a5fe2835fc56 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_11-9-9be51f5537a03d7dbf56693d9fdc1688 @@ -0,0 +1 @@ +6 10 10 28.0 diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 b/sql/hive/src/test/resources/golden/groupby_sort_2-10-5032dd0941fab4871eefd79f7e4a5c86 new file mode 100644 index 0000000000000..c5b99ed941efc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_2-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 b/sql/hive/src/test/resources/golden/groupby_sort_2-4-7dc6959cca820ea825e6567e1b152088 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_2-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/groupby_sort_2-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 b/sql/hive/src/test/resources/golden/groupby_sort_2-7-2238ae1cfb52dfd4f9e7b1d0e2b2c0f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 b/sql/hive/src/test/resources/golden/groupby_sort_2-8-4d64b6bb15f6b31e47d52df53a1d9414 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed b/sql/hive/src/test/resources/golden/groupby_sort_2-9-30377eb1022ca1f4f5201f8897dff9ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_3-10-4bf8dba6e66e733423a3408d61897926 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 b/sql/hive/src/test/resources/golden/groupby_sort_3-11-2437764cdf86cd2d67430bd323346086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e b/sql/hive/src/test/resources/golden/groupby_sort_3-12-bc5b02bd034ead563d39d2685087005e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd b/sql/hive/src/test/resources/golden/groupby_sort_3-13-1f1638d495fd67ab40bbd2a03ee4ddd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f b/sql/hive/src/test/resources/golden/groupby_sort_3-14-1c8def3eb5fc590046d9cdd02d1cbf3f new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_3-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 b/sql/hive/src/test/resources/golden/groupby_sort_3-4-74ba824287893d3aaa1cdd957d472729 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_3-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/groupby_sort_3-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 b/sql/hive/src/test/resources/golden/groupby_sort_3-7-fa27a0ebfb1024248031f281b5e320d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 b/sql/hive/src/test/resources/golden/groupby_sort_3-8-5ce0d81bbee5bbab19194535b4b05b6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd b/sql/hive/src/test/resources/golden/groupby_sort_3-9-37a53e292752d1a4caff70e64c5cdfbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_4-10-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 b/sql/hive/src/test/resources/golden/groupby_sort_4-11-dfd54cb0f947152bcf66af1eaa221eb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f b/sql/hive/src/test/resources/golden/groupby_sort_4-12-1f082ad7217ad620063b58887b9b922f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 b/sql/hive/src/test/resources/golden/groupby_sort_4-13-46c4a3675c8de0510b648856a193f3e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 b/sql/hive/src/test/resources/golden/groupby_sort_4-14-75d59344b6114c0bb20d5eac301c2170 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_4-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f b/sql/hive/src/test/resources/golden/groupby_sort_4-4-40891a8b7f896b11de173835c51aca4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_4-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/groupby_sort_4-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e b/sql/hive/src/test/resources/golden/groupby_sort_4-7-8f552bff84cdc75d7a7e1e12f67c240e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef b/sql/hive/src/test/resources/golden/groupby_sort_4-8-91aa8f7c6e9b0d8fa11277061c00f6ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 b/sql/hive/src/test/resources/golden/groupby_sort_4-9-ebc7ac3b2dfdb958d161cd7c8f947a72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_5-10-4bf8dba6e66e733423a3408d61897926 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 b/sql/hive/src/test/resources/golden/groupby_sort_5-11-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f b/sql/hive/src/test/resources/golden/groupby_sort_5-12-42f4d08dc197e04234d95e08f6ed1e2f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_5-13-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/groupby_sort_5-14-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 b/sql/hive/src/test/resources/golden/groupby_sort_5-15-e9baee9e98db00b9277dae33097aab82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd b/sql/hive/src/test/resources/golden/groupby_sort_5-16-37a53e292752d1a4caff70e64c5cdfbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_5-17-4bf8dba6e66e733423a3408d61897926 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 b/sql/hive/src/test/resources/golden/groupby_sort_5-18-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf b/sql/hive/src/test/resources/golden/groupby_sort_5-19-2e441f92547a956942f4451e943bf5cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_5-20-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/groupby_sort_5-21-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d b/sql/hive/src/test/resources/golden/groupby_sort_5-22-70a6c959960c299db2b961dffaa2628d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f b/sql/hive/src/test/resources/golden/groupby_sort_5-23-8dce8ffad6523bbe076b968e243f094f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd b/sql/hive/src/test/resources/golden/groupby_sort_5-24-1f1638d495fd67ab40bbd2a03ee4ddd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f b/sql/hive/src/test/resources/golden/groupby_sort_5-25-1c8def3eb5fc590046d9cdd02d1cbf3f new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 b/sql/hive/src/test/resources/golden/groupby_sort_5-26-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_5-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 b/sql/hive/src/test/resources/golden/groupby_sort_5-4-f3697ac93bcda24a9d7593b703b312e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_5-5-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/groupby_sort_5-6-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 b/sql/hive/src/test/resources/golden/groupby_sort_5-7-5219a87f995d294a0c68ae6499dba7d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 b/sql/hive/src/test/resources/golden/groupby_sort_5-8-e9baee9e98db00b9277dae33097aab82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd b/sql/hive/src/test/resources/golden/groupby_sort_5-9-37a53e292752d1a4caff70e64c5cdfbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 b/sql/hive/src/test/resources/golden/groupby_sort_6-10-788b62269990c43aa3cb1847f99b3343 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 b/sql/hive/src/test/resources/golden/groupby_sort_6-11-8c28fe3adff3cd106e88d7984ef5fe52 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_6-12-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 b/sql/hive/src/test/resources/golden/groupby_sort_6-13-73805f987b3b2384352c5dd5201f1b29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 b/sql/hive/src/test/resources/golden/groupby_sort_6-14-5574207f68aac30a893785c50c735864 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_6-15-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_6-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e b/sql/hive/src/test/resources/golden/groupby_sort_6-4-659bc2cd87fd74fef5ed50d795e8aa1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/groupby_sort_6-5-c0ea81b686236d661166912040a16ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 b/sql/hive/src/test/resources/golden/groupby_sort_6-6-788b62269990c43aa3cb1847f99b3343 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 b/sql/hive/src/test/resources/golden/groupby_sort_6-7-8c28fe3adff3cd106e88d7984ef5fe52 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_6-8-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 b/sql/hive/src/test/resources/golden/groupby_sort_6-9-7b84dbb0895a623d460c70678bd74a64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 b/sql/hive/src/test/resources/golden/groupby_sort_7-10-4bf8dba6e66e733423a3408d61897926 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 b/sql/hive/src/test/resources/golden/groupby_sort_7-11-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_7-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 b/sql/hive/src/test/resources/golden/groupby_sort_7-4-253f1f4f2e4153a4a9358d268f8352e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/groupby_sort_7-5-8b5d511014e1776743cacaf77f68d5fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 b/sql/hive/src/test/resources/golden/groupby_sort_7-6-137e81fded2b36969bf71279d7ffee34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 b/sql/hive/src/test/resources/golden/groupby_sort_7-7-5219a87f995d294a0c68ae6499dba7d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b b/sql/hive/src/test/resources/golden/groupby_sort_7-8-289632719165e6e8182ebd8f5f766b7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 b/sql/hive/src/test/resources/golden/groupby_sort_7-9-6e36549828003492627aa5e96a63d3a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 b/sql/hive/src/test/resources/golden/groupby_sort_8-10-d3a2d251675f7bd7a196d2b8345b36d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-11-bd99462ed878bf4bec74b3cb9132908d @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 b/sql/hive/src/test/resources/golden/groupby_sort_8-12-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f b/sql/hive/src/test/resources/golden/groupby_sort_8-4-14283ed438d96ad881025b969c8fb69f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/groupby_sort_8-5-8b5d511014e1776743cacaf77f68d5fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 b/sql/hive/src/test/resources/golden/groupby_sort_8-6-137e81fded2b36969bf71279d7ffee34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d b/sql/hive/src/test/resources/golden/groupby_sort_8-7-8188c7bcb9ead25f8c27af7def87218d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d b/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-8-bd99462ed878bf4bec74b3cb9132908d @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a b/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_8-9-8e06b51e940e956f14a8c7679c3d423a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 b/sql/hive/src/test/resources/golden/groupby_sort_9-10-4d9341036906853bb9a1641f5e0179b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_9-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f b/sql/hive/src/test/resources/golden/groupby_sort_9-4-14283ed438d96ad881025b969c8fb69f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb b/sql/hive/src/test/resources/golden/groupby_sort_9-5-8b5d511014e1776743cacaf77f68d5fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 b/sql/hive/src/test/resources/golden/groupby_sort_9-6-137e81fded2b36969bf71279d7ffee34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c b/sql/hive/src/test/resources/golden/groupby_sort_9-7-c762061ecb1eea7ac218809b9e49900c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 b/sql/hive/src/test/resources/golden/groupby_sort_9-8-fba2dc1329046ee81e2dbf16b92abc27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c b/sql/hive/src/test/resources/golden/groupby_sort_9-9-feec69facdc973a0ff78455f766845c new file mode 100644 index 0000000000000..3d4708b7c9d64 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-10-ebc7ac3b2dfdb958d161cd7c8f947a72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-11-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-12-c166396bbdb62490f56ab0bc84aaa7d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-13-c8a51e8a269da4c4ae98ac105a573f3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-14-46c4a3675c8de0510b648856a193f3e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-15-bbdd53118f788d7bb679d094c41243c8 new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-16-54f3a2a97939d3eca8a601b74ef30dea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-17-236d51792f4913b5858f367e3cff2c60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-18-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-19-1e77dcdd6f54334dbae6a4d11ad6ff64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-20-90c84358c50e51be5ce210bd7dec9bc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-21-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-22-27e2e662d517f32952145cc2a51bf564 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-23-6775cb6aee040e22508cf3cac0b55f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-24-eb2b625279d8794390d7e2dc8f2dc907 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-25-d53196339980a00a619788bd799a32e7 new file mode 100644 index 0000000000000..c1cc4ee204773 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-26-138e93f004f7bd16e63bbf8d9090af21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-27-93153fd67c1d19bb9ad64f98294e4981 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-28-ca369ab23c32070e2d42ba8df036175f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-29-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-30-4095695e88e23dd42acb1749a83bdbb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-31-d9ba55c20c3f5df262e81cbf5dab5387 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-32-d53196339980a00a619788bd799a32e7 new file mode 100644 index 0000000000000..f0192040e147b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-33-c88ee457dee7bb141a198a2ae39d787f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-34-c04167e880fe3e942ce77e75d660f1ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-35-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..b6c2eb98e5e49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-36-deb3f1793f51d1edf011a8405abf4968 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-37-7871722f392f801a868e0e2fb372c610 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-38-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..8e7ee8a2b47bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-39-b71bdaa2b7c4b5c51a9773c123e5306d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-4-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-40-89aa7bab4272546e06cd7e504380d96b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-41-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..0b2b54cd94e4f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-42-348b0126cb1d214fea58d4af9d3dbf67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-43-83889e7dc73d796cc869160b6b35102c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-44-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..3d4708b7c9d64 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-45-41462b2e60bf44571a7b1fb435374d6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-46-8aea6edf0481e2a10f14879acba62648 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-47-4999db9eb86d9455c1d75e97b052f279 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-48-fecb9f2203aeb2ac4b693a97badde6fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-49-f0f18d5fa2824735799edc4bdeb1afb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-5-e906be6d27c9dfcffd4af171541639ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-50-f3f94c4814c3bff60a0b06edf0c884bd new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-51-461847b174096e7a255fb07cb35ab434 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-52-8da967e7c2210ad044ba8b08d1685065 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-53-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-54-fdc295aaefba56548a22dfcddc2a94f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-55-74bfe3fbf9d68a75013fba1c3c7bbd7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-56-1013d1ad014aa203b1dce26085b09c01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-57-5373c5449884d95bc7db9dab55239a49 new file mode 100644 index 0000000000000..7ca6b0b28a960 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-58-825135844e8ac6d8d5254cc961ec8fd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-59-e671e63f6b70094048563a9c33748c97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-60-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..97a3b8c2f5977 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-61-51824b04f2a008f63e1469695e60d9c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-62-18b0757f6d9e29808061ca3763b8b6d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-63-c4ec0433a832ef551d70254957e3afca new file mode 100644 index 0000000000000..58e16ef3c0ef3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-64-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-65-7f98b724df05f51b3ec1f087a8da414e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-66-83c59d378571a6e487aa20217bd87817 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-67-b4c5b3eeb74085711866a2eec27bcb37 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-68-b4fec0996399be2239961594897d6715 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-69-9d01ff3d1fde3ed2ab55ea9d7079fd5c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-7-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-70-ed76c0068780120a6f23feefee303403 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-71-2d5403943a3efdf9fd3eccb6927499cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-72-ca16ff548ebb9bab4b76f2e339064a9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-73-d2fa5e7bdd6b7934d10d5905cacd5715 new file mode 100644 index 0000000000000..4e31460a412ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-74-6296dde4e71acf7e7f42ee58cf3b5acd new file mode 100644 index 0000000000000..10f4a1f5ff34c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-75-6f7caef1c773268350c9cf94ad85be01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-76-4931c5a72a5231f67317d27ca025bb97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 new file mode 100644 index 0000000000000..d15db8c5d079f --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-77-d2fa5e7bdd6b7934d10d5905cacd5715 @@ -0,0 +1 @@ +8 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-78-6d6eac558569563cc47a657daffa8ba7 new file mode 100644 index 0000000000000..3d1609d961673 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-8-c0ea81b686236d661166912040a16ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 b/sql/hive/src/test/resources/golden/groupby_sort_skew_1-9-f0ee61903aeacb758e2eada242e5e14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e b/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-2-fc30020d09151dc29be807795ad9475e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-3-fffea659b633b1f269b38556a7f54634 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a b/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/groupby_sort_test_1-4-8e06b51e940e956f14a8c7679c3d423a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad b/sql/hive/src/test/resources/golden/groupby_sort_test_1-5-e906be6d27c9dfcffd4af171541639ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/groupby_sort_test_1-6-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b b/sql/hive/src/test/resources/golden/groupby_sort_test_1-7-b6c452a800ff333aacb863bb3243c15b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-8-c0ea81b686236d661166912040a16ea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 b/sql/hive/src/test/resources/golden/groupby_sort_test_1-9-4d3e8128fb29c232b984f41ed4e78794 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f b/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f new file mode 100644 index 0000000000000..a99cee758fe57 --- /dev/null +++ b/sql/hive/src/test/resources/golden/hash-0-a658b129316d666d4b01c1581eed1c1f @@ -0,0 +1 @@ +3556498 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf b/sql/hive/src/test/resources/golden/hook_context_cs-0-e319c8574a6cd8739e5fd5984ceed3cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 b/sql/hive/src/test/resources/golden/hook_context_cs-1-de3aa1c4674fb166b825b1e2f58d1950 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 b/sql/hive/src/test/resources/golden/implicit_cast1-0-e3d2bd2cc5a4d5d794f9bf29927f2296 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a b/sql/hive/src/test/resources/golden/implicit_cast1-1-66f185b6fdccddba498c463641e7dc7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 b/sql/hive/src/test/resources/golden/implicit_cast1-2-6871be041d36ea813e1b2f331e3747f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 b/sql/hive/src/test/resources/golden/index_creation-0-f880114c33c99a5f23c1465fd88f0db3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 b/sql/hive/src/test/resources/golden/index_creation-1-a8bc76559014d9cdf07184208d582d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 b/sql/hive/src/test/resources/golden/index_creation-10-4bd6c86cd3f6a94964b3d13235e8e261 new file mode 100644 index 0000000000000..d7c6f236687d8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 b/sql/hive/src/test/resources/golden/index_creation-11-b5b7e4f7af5186033be12a4393dc3bb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 b/sql/hive/src/test/resources/golden/index_creation-12-9cc02e06c6051810c50e225bb2c66669 new file mode 100644 index 0000000000000..4c6ec0ba34bb8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 b/sql/hive/src/test/resources/golden/index_creation-13-9c0ec67e79a0d50b46bd5c944c710fc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 b/sql/hive/src/test/resources/golden/index_creation-14-a8c1ed9913d9fbcd1b3299a2f8bb2165 new file mode 100644 index 0000000000000..0c6af94247b85 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 b/sql/hive/src/test/resources/golden/index_creation-15-7d7e1a479e7dcd8f9d4199775e05bc1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d b/sql/hive/src/test/resources/golden/index_creation-16-ffa6d3fcef97b7322dd3759d4a70881d new file mode 100644 index 0000000000000..e4a5816e6f1cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad b/sql/hive/src/test/resources/golden/index_creation-17-18ed0b70b0b6d076b96c9f73bfa721ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 b/sql/hive/src/test/resources/golden/index_creation-18-bcdb19db031341c4a50264ccf49328e4 new file mode 100644 index 0000000000000..3d8751ca47049 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f b/sql/hive/src/test/resources/golden/index_creation-19-98dbf83283f9e073e88ba770ec5a707f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 b/sql/hive/src/test/resources/golden/index_creation-2-9d5d11cb38f2e097f16d2db5693f4f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 b/sql/hive/src/test/resources/golden/index_creation-20-68c5c98581c683b17ceaf1c0fd192871 new file mode 100644 index 0000000000000..0e4852b319052 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e b/sql/hive/src/test/resources/golden/index_creation-21-4c8f6b48c437bf0be109fc0be1dc840e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 b/sql/hive/src/test/resources/golden/index_creation-22-f36cb2eed39691ca949b25182e2dd31 new file mode 100644 index 0000000000000..41a5492fc5331 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 b/sql/hive/src/test/resources/golden/index_creation-23-e7f21f556d3aa2bedb0717a167720741 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf b/sql/hive/src/test/resources/golden/index_creation-24-8cb102bd5addf7fece0e2691468bc3bf new file mode 100644 index 0000000000000..8212bf8b7d2fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca b/sql/hive/src/test/resources/golden/index_creation-25-33c48966230b934ae8ddf74ff18bb9ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 b/sql/hive/src/test/resources/golden/index_creation-26-f85db55b3f63ae186a1b6d5cec545939 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 b/sql/hive/src/test/resources/golden/index_creation-27-e4856f13692e63d61f72aaf75e50e5f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe b/sql/hive/src/test/resources/golden/index_creation-28-bd20d4b59e6489082a92fcbfcc5f8dbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f b/sql/hive/src/test/resources/golden/index_creation-29-ee8d287111069805c41c9c0032adc46f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 b/sql/hive/src/test/resources/golden/index_creation-3-14b999fc6dfb10a3632afe14e08003e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 b/sql/hive/src/test/resources/golden/index_creation-30-f880114c33c99a5f23c1465fd88f0db3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 b/sql/hive/src/test/resources/golden/index_creation-31-a8bc76559014d9cdf07184208d582d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 b/sql/hive/src/test/resources/golden/index_creation-32-9d5d11cb38f2e097f16d2db5693f4f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 b/sql/hive/src/test/resources/golden/index_creation-33-14b999fc6dfb10a3632afe14e08003e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/sql/hive/src/test/resources/golden/index_creation-34-c8c5d4c45e59d041dcbbdfc5426e5fa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/sql/hive/src/test/resources/golden/index_creation-35-e78eb4d61c0ddb272fd94c5f7a8c0e84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/index_creation-36-21bcf37075b02097f16c8fc8130a83b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b b/sql/hive/src/test/resources/golden/index_creation-37-9334418431eca405f13206bd8db42a1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 b/sql/hive/src/test/resources/golden/index_creation-38-f1f56119aede4f42221a68f6aaa42a26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d b/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d new file mode 100644 index 0000000000000..e8310385c56dc --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_creation-39-489b4ceb2f4301a7132628303f99240d @@ -0,0 +1 @@ +src \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 b/sql/hive/src/test/resources/golden/index_creation-4-c8c5d4c45e59d041dcbbdfc5426e5fa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 b/sql/hive/src/test/resources/golden/index_creation-5-e78eb4d61c0ddb272fd94c5f7a8c0e84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 b/sql/hive/src/test/resources/golden/index_creation-6-21bcf37075b02097f16c8fc8130a83b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b b/sql/hive/src/test/resources/golden/index_creation-7-9334418431eca405f13206bd8db42a1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 b/sql/hive/src/test/resources/golden/index_creation-8-f1f56119aede4f42221a68f6aaa42a26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c b/sql/hive/src/test/resources/golden/index_creation-9-bf40d4d50d050f2f8342c07f5a9dcf0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 b/sql/hive/src/test/resources/golden/index_serde-0-6560d12b69d55e5297a145ebc4bb0cb3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 b/sql/hive/src/test/resources/golden/index_serde-1-f92d6c66d21791c11d2a822df04c1b63 new file mode 100644 index 0000000000000..48522980f81a8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b b/sql/hive/src/test/resources/golden/index_serde-10-123301a057d4a46072d0431e00e20c4b new file mode 100644 index 0000000000000..4ed570f9070eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 b/sql/hive/src/test/resources/golden/index_serde-11-309e916d683a1a12ab62565697cb0046 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 b/sql/hive/src/test/resources/golden/index_serde-12-d590fd7cb9d433143de490d75686dd4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c b/sql/hive/src/test/resources/golden/index_serde-2-fd1b220f4eafb0ba9b519a156e3c87c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 b/sql/hive/src/test/resources/golden/index_serde-3-afcf2a156ccd4f79a0489b4593908d79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 b/sql/hive/src/test/resources/golden/index_serde-4-d7547751c37375a9238043bbe250e716 new file mode 100644 index 0000000000000..d2ca633d0ae01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 b/sql/hive/src/test/resources/golden/index_serde-5-e6ff4b23b7f102e359afb4d53a1dedc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae b/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_serde-6-c9d7dcde469d3b9a66965a64dd15e4ae @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_serde-7-3b03210f94ec40db9ab02620645014d1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 b/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/index_serde-8-35f48c7d6fa164bb84643657bc9280a8 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 b/sql/hive/src/test/resources/golden/index_serde-9-c85e061ea9c5b90ca69b7450faad14b6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e b/sql/hive/src/test/resources/golden/infer_const_type-0-e231c5154b18cbc0baa082a7461dd13e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 b/sql/hive/src/test/resources/golden/infer_const_type-1-c836a98522549d2a3fd43998afd8ae94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd b/sql/hive/src/test/resources/golden/infer_const_type-2-d8590c7336ae771b7a685bb544e8d2bd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/innerjoin-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e b/sql/hive/src/test/resources/golden/innerjoin-1-f1293ebf768eb04f2f0bfe6297c4509e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 b/sql/hive/src/test/resources/golden/innerjoin-2-6c0cafe1d94c0acfe2d09afd0289df75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 b/sql/hive/src/test/resources/golden/innerjoin-3-dec6031cdf20e07dd1cd70e3741dc330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 b/sql/hive/src/test/resources/golden/innerjoin-4-35c7611180562dcd9dab834f41654095 new file mode 100644 index 0000000000000..3b7cf42f96358 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a b/sql/hive/src/test/resources/golden/innerjoin-5-5eb553ac988b409ad2d96cf6cd395d9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce b/sql/hive/src/test/resources/golden/innerjoin-6-17c49c593f6160e096b30dfee7b643ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 b/sql/hive/src/test/resources/golden/innerjoin-7-b42ecd0ee90bd28441c41fab4e36fe78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 b/sql/hive/src/test/resources/golden/innerjoin-8-b125c27acd6a6cacbde1f7587df2fce9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b b/sql/hive/src/test/resources/golden/innerjoin-9-326eeac56676d78fe489c464cddb526b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 b/sql/hive/src/test/resources/golden/inoutdriver-0-47981488ab51ed3cc6c335f5cf703908 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 b/sql/hive/src/test/resources/golden/inoutdriver-1-b2f337566a5075f3e3e81335008d95d3 new file mode 100644 index 0000000000000..010e999c36749 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 b/sql/hive/src/test/resources/golden/input-0-42022446607cfbe8a64d8fd7b9898ce7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 b/sql/hive/src/test/resources/golden/input-1-6558e385bb08991302a72076dd7b7ff5 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 b/sql/hive/src/test/resources/golden/input0-0-fb47f8a8a8e3213f3fe47825a556c593 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f b/sql/hive/src/test/resources/golden/input0-1-efefd4364cd2790447fb0f908e87501f new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 b/sql/hive/src/test/resources/golden/input1-0-28c9f2913833d6911f22e2e2e8c60f68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 b/sql/hive/src/test/resources/golden/input1-1-8aaad4ee49c9bdf7b34642cc75f6a1a0 new file mode 100644 index 0000000000000..e00bf4ec2c7e5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 new file mode 100644 index 0000000000000..743be67e8d1c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input1-2-d3aa54d5436b7b59ff5c7091b7ca6145 @@ -0,0 +1,2 @@ +a int None +b double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input10-0-acdd72fda64c5463ce29f31020012d42 b/sql/hive/src/test/resources/golden/input10-0-acdd72fda64c5463ce29f31020012d42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 b/sql/hive/src/test/resources/golden/input10-1-6970b6d2d451612b59fccbfd7ec68f74 new file mode 100644 index 0000000000000..175d371fd09c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 b/sql/hive/src/test/resources/golden/input10-2-73f00da5cfc254745d1d80f913eb6449 new file mode 100644 index 0000000000000..8dcdf43e31be3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input11-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb b/sql/hive/src/test/resources/golden/input11-1-3e66f12ae780a85721fa21c42fb3c8fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 b/sql/hive/src/test/resources/golden/input11-2-24fe55bae88ad4a8e240376d012bc491 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input11-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..37dd922fd0787 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input11_limit-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 b/sql/hive/src/test/resources/golden/input11_limit-1-f2c43fcadcfca0c8c5cdd0b29c47a921 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 b/sql/hive/src/test/resources/golden/input11_limit-2-47fdba9764a8851379e2ed7e16c54583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 b/sql/hive/src/test/resources/golden/input11_limit-3-8a0c68a4f7386ff214db5d9eed0876d5 new file mode 100644 index 0000000000000..fdf93911ee847 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input12-0-9b141c1e5917ca82c6bc36a9a2950a1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input12-1-2b9ccaa793eae0e73bf76335d3d6880 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c b/sql/hive/src/test/resources/golden/input12-10-4d9eb316259a8e7ed6627bc27a639f7c new file mode 100644 index 0000000000000..4220cf5f30392 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input12-2-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 b/sql/hive/src/test/resources/golden/input12-3-5186c4f54bcc1d9a2afb953c8dd5d8d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 b/sql/hive/src/test/resources/golden/input12-4-f69ce1f5b3e0e77d1c487877580e6a23 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf b/sql/hive/src/test/resources/golden/input12-5-d83ffe497d7f7a8f72a6844dc6dad3cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 b/sql/hive/src/test/resources/golden/input12-6-f6ae011490e0c8c037927767ad966ce4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12-7-16da1c1293626b943343443368679e9c b/sql/hive/src/test/resources/golden/input12-7-16da1c1293626b943343443368679e9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input12-8-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..37dd922fd0787 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd b/sql/hive/src/test/resources/golden/input12-9-3d08dc27c1a133c2497fc554c0d169bd new file mode 100644 index 0000000000000..d6e0c29932b9b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/input12_hadoop20-0-db1cd54a4cb36de2087605f32e41824f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/input12_hadoop20-1-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 b/sql/hive/src/test/resources/golden/input12_hadoop20-2-743c24cd29161d31a16ec446bd709d92 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 b/sql/hive/src/test/resources/golden/input12_hadoop20-3-f69ce1f5b3e0e77d1c487877580e6a23 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf b/sql/hive/src/test/resources/golden/input12_hadoop20-4-d83ffe497d7f7a8f72a6844dc6dad3cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 b/sql/hive/src/test/resources/golden/input12_hadoop20-5-f6ae011490e0c8c037927767ad966ce4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c b/sql/hive/src/test/resources/golden/input12_hadoop20-6-16da1c1293626b943343443368679e9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input12_hadoop20-7-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..46057aa0a8fca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd b/sql/hive/src/test/resources/golden/input12_hadoop20-8-3d08dc27c1a133c2497fc554c0d169bd new file mode 100644 index 0000000000000..9ee31317478d5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c b/sql/hive/src/test/resources/golden/input12_hadoop20-9-4d9eb316259a8e7ed6627bc27a639f7c new file mode 100644 index 0000000000000..a66a07386eef8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input14-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a b/sql/hive/src/test/resources/golden/input14-1-d0a1caeeeb79580713b7ecc28543f74a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 b/sql/hive/src/test/resources/golden/input14-2-1d791beabaa5288ea2fcf6b3675eda26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input14-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..703a4eef24f3f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input14_limit-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf b/sql/hive/src/test/resources/golden/input14_limit-1-be9934fc5e6ecb9854eb7531a5929dcf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 b/sql/hive/src/test/resources/golden/input14_limit-2-780cdc89e0e736790124b6bdac827951 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input14_limit-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..4335dce6a9929 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/input16_cc-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 b/sql/hive/src/test/resources/golden/input16_cc-1-5180e975a6babd51752706f1799e7df5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 b/sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/input16_cc-2-9ea7bc94b5383e71adb678f1dac1c619 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input17-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 b/sql/hive/src/test/resources/golden/input17-1-be9cde5e769f171f60f61a7739de8f17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 b/sql/hive/src/test/resources/golden/input17-2-21166e268096f6ec67f4f57ec333e901 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input17-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..1c48b3680a3ac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a b/sql/hive/src/test/resources/golden/input19-0-a8eaa79d3a2762cdb263bc822d62df3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc b/sql/hive/src/test/resources/golden/input19-1-f2832e249ab28bb3fb8e472012c5ffc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 b/sql/hive/src/test/resources/golden/input19-2-5a804e02e4419e785d15e7f39d6c5730 new file mode 100644 index 0000000000000..795dbe3a976eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/input1_limit-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 b/sql/hive/src/test/resources/golden/input1_limit-1-f69ce1f5b3e0e77d1c487877580e6a23 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 b/sql/hive/src/test/resources/golden/input1_limit-2-c1d996fc96189d323f6af180708abc12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e b/sql/hive/src/test/resources/golden/input1_limit-3-a4a7e1c4b4482e1063c00c32d11cf1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 b/sql/hive/src/test/resources/golden/input1_limit-4-8a0c68a4f7386ff214db5d9eed0876d5 new file mode 100644 index 0000000000000..fdf93911ee847 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 b/sql/hive/src/test/resources/golden/input1_limit-5-eaaf713833e28a803c798562c7d6cd23 new file mode 100644 index 0000000000000..8a8f1a1b8bbe1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c b/sql/hive/src/test/resources/golden/input2-0-eaf4ec8905774e4be6ea3fa4f32a456c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 new file mode 100644 index 0000000000000..743be67e8d1c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input2-1-e0efeda558cd0194f4764a5735147b16 @@ -0,0 +1,2 @@ +a int None +b double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd new file mode 100644 index 0000000000000..743be67e8d1c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input2-2-aa9ab0598e0cb7a12c719f9b3d98dbfd @@ -0,0 +1,2 @@ +a int None +b double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input2-3-45b9f786f64afa5e039d1856fe926649 b/sql/hive/src/test/resources/golden/input2-3-45b9f786f64afa5e039d1856fe926649 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b b/sql/hive/src/test/resources/golden/input2-4-235f92683416fab031e6e7490487b15b new file mode 100644 index 0000000000000..ca0726f517eeb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..2c861553f9aa2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input2-5-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,4 @@ +src +srcpart +test2a +test2b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input2-6-4a6b9674c126337f71834f11613f996d b/sql/hive/src/test/resources/golden/input2-6-4a6b9674c126337f71834f11613f996d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..3e40a0c866d4d --- /dev/null +++ b/sql/hive/src/test/resources/golden/input2-7-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,3 @@ +src +srcpart +test2b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 b/sql/hive/src/test/resources/golden/input2-8-28c3fc507cd2e02a3a39331fc9c95334 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 b/sql/hive/src/test/resources/golden/input2-9-48bf8c06ed0a264d0863613fe79795e1 new file mode 100644 index 0000000000000..5337f342fedd8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input21-0-7f40c525398953892ffd6283f54ba427 b/sql/hive/src/test/resources/golden/input21-0-7f40c525398953892ffd6283f54ba427 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 b/sql/hive/src/test/resources/golden/input21-1-70b803742328eacc69eb1ed044a5c6b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 b/sql/hive/src/test/resources/golden/input21-2-a4b3aeb45ae0cf38777e126faed0eff7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe b/sql/hive/src/test/resources/golden/input21-3-9809b74435cbaedef0dc6e6b88b180fe new file mode 100644 index 0000000000000..c6c298df200ca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input22-0-b9cd4041ca510639838a97376747b498 b/sql/hive/src/test/resources/golden/input22-0-b9cd4041ca510639838a97376747b498 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 b/sql/hive/src/test/resources/golden/input22-1-b7f46eccd104e6ed1b29e2de45089f66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 b/sql/hive/src/test/resources/golden/input22-2-10e46b53620b6b05019da4e58f53e6c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 b/sql/hive/src/test/resources/golden/input22-3-8285c1934441b12f6c016e13cb410e79 new file mode 100644 index 0000000000000..336ba0545635d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf b/sql/hive/src/test/resources/golden/input23-0-c983ab8049996ad5e1e0296b51ae81cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb b/sql/hive/src/test/resources/golden/input23-1-c8e32187d09172eb32b0480fcd28cccb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 b/sql/hive/src/test/resources/golden/input24-0-f8116598acadb7589ae02b13b65e1ad8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input24-1-f57485de80ad64f9931bf92514fa1 b/sql/hive/src/test/resources/golden/input24-1-f57485de80ad64f9931bf92514fa1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 b/sql/hive/src/test/resources/golden/input24-2-3fc20939000a5324a928feeb673a1ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 b/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input24-3-3189f3b2990de94619b9cb583d9dd3c5 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 b/sql/hive/src/test/resources/golden/input25-0-f8116598acadb7589ae02b13b65e1ad8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input25-1-f57485de80ad64f9931bf92514fa1 b/sql/hive/src/test/resources/golden/input25-1-f57485de80ad64f9931bf92514fa1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 b/sql/hive/src/test/resources/golden/input25-2-74a4b9d59ee26b650f567c07e9103b66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 b/sql/hive/src/test/resources/golden/input25-3-d4940dd537ae72d234ffea23f8f0b103 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input25-4-72c74c55dd7d85d194448e9c58488938 b/sql/hive/src/test/resources/golden/input25-4-72c74c55dd7d85d194448e9c58488938 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 b/sql/hive/src/test/resources/golden/input26-0-1b6e9e73dd7ee60644c0997b21f48430 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 b/sql/hive/src/test/resources/golden/input26-1-8272225744e83ba4cbe158a5c113fce3 new file mode 100644 index 0000000000000..b8fd0ab6545d6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a b/sql/hive/src/test/resources/golden/input28-0-b85d5edb4640c4b154e91e9a1beeef8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input28-1-f57485de80ad64f9931bf92514fa1 b/sql/hive/src/test/resources/golden/input28-1-f57485de80ad64f9931bf92514fa1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c b/sql/hive/src/test/resources/golden/input28-2-e88e8d960eeb128fbc6642274780ce4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 b/sql/hive/src/test/resources/golden/input28-3-a3da4a5ba73e312a8adafc1bc8a0f930 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f b/sql/hive/src/test/resources/golden/input2_limit-0-44591b07e007def036cea6c61a3f0a7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 b/sql/hive/src/test/resources/golden/input2_limit-1-fed7e0bb996623da7dd17793e835f785 new file mode 100644 index 0000000000000..899417ee77ad8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff b/sql/hive/src/test/resources/golden/input3-0-2c80ec90d4d2c9c7446c05651bb76bff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d b/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d new file mode 100644 index 0000000000000..743be67e8d1c3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input3-1-6ec8e282bd39883a57aecd9e4c8cdf1d @@ -0,0 +1,2 @@ +a int None +b double None \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 b/sql/hive/src/test/resources/golden/input3-10-10a1a8a97f6417c3da16829f7e519475 new file mode 100644 index 0000000000000..594b29ca1410f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ac382c7369264 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input3-11-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,4 @@ +src +srcpart +test3a +test3c \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 b/sql/hive/src/test/resources/golden/input3-12-a22d09de72e5067a0a94113cdecdaa95 new file mode 100644 index 0000000000000..cb17be511e875 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a b/sql/hive/src/test/resources/golden/input3-13-23bbec31affef0d758bc4a40490e0b9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 b/sql/hive/src/test/resources/golden/input3-14-efee6816e20fe61595a4a2a991071219 new file mode 100644 index 0000000000000..b906fd3c2e775 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 b/sql/hive/src/test/resources/golden/input3-2-fa2aceba8cdcb869262e8ad6d431f491 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-3-1c5990b1aed2be48311810dae3019994 new file mode 100644 index 0000000000000..ca0726f517eeb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..6b1ce270d97e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input3-4-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,4 @@ +src +srcpart +test3a +test3b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 b/sql/hive/src/test/resources/golden/input3-5-f40b7cc4ac38c0121ccab9ef4e7e9fd2 new file mode 100644 index 0000000000000..92c0ed68f8a7b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input3-6-ba8c440158c2519353d02471bfb05694 b/sql/hive/src/test/resources/golden/input3-6-ba8c440158c2519353d02471bfb05694 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 b/sql/hive/src/test/resources/golden/input3-7-1c5990b1aed2be48311810dae3019994 new file mode 100644 index 0000000000000..594b29ca1410f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be b/sql/hive/src/test/resources/golden/input3-8-4dc0fefca4d158fd2ab40551ae9e35be new file mode 100644 index 0000000000000..09bbc29377720 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e b/sql/hive/src/test/resources/golden/input3-9-5076c1c35053b09173f6acdf1b5e9d6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c b/sql/hive/src/test/resources/golden/input30-0-582c5fcbe2fe12cc8e7b21225583d96c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab b/sql/hive/src/test/resources/golden/input30-1-90c0d1a75de78c405413fd627caea4ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input30-2-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input30-3-c21dba410fb07a098f93430a9d21df79 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 b/sql/hive/src/test/resources/golden/input30-4-f0ebd08e7675b19ae831824ef4d9e223 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 b/sql/hive/src/test/resources/golden/input30-5-38734677c27b5f90a8df5da6e6351c76 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input30-6-f120ac8c87db9eebb5da7ac99b48600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f b/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f new file mode 100644 index 0000000000000..25bf17fc5aaab --- /dev/null +++ b/sql/hive/src/test/resources/golden/input30-7-95d10d459c088d5fbefd00bdd8d44c3f @@ -0,0 +1 @@ +18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/input31-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/input31-1-c21dba410fb07a098f93430a9d21df79 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe b/sql/hive/src/test/resources/golden/input31-2-705764f8f7cab9378964af30b83f7fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f b/sql/hive/src/test/resources/golden/input31-3-50c905261882f7fd8539fdd91e68151f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 b/sql/hive/src/test/resources/golden/input31-4-2f886fa357df9342733551fa1b53f913 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/input32-0-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79 b/sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/input32-1-c21dba410fb07a098f93430a9d21df79 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 b/sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/input32-2-1ba7748b3d2f8908c2e81771ab229316 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d b/sql/hive/src/test/resources/golden/input32-3-b0070890240c15d647af59f41b77ba3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 b/sql/hive/src/test/resources/golden/input32-4-d0e1378a30e21e0198e47d9b668ee1f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e b/sql/hive/src/test/resources/golden/input32-5-8789d32fc5b784fe2d171566732c573e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 b/sql/hive/src/test/resources/golden/input37-0-86e2e274650fb56651607ea10d356fc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 b/sql/hive/src/test/resources/golden/input37-1-6bb557a5cfe7ceaa2f749494ea32b9e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 b/sql/hive/src/test/resources/golden/input39-0-7bd12162381231be9d578797818957a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/sql/hive/src/test/resources/golden/input39-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 b/sql/hive/src/test/resources/golden/input39-10-28bf1b34c04a048da339eddd4c1fd779 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 b/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 new file mode 100644 index 0000000000000..25bf17fc5aaab --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39-11-6d0814cbb72eb96bfc75c95d06f1e528 @@ -0,0 +1 @@ +18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39-12-f120ac8c87db9eebb5da7ac99b48600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 b/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39-13-c9c5b254ce9b439f09e72a0dce5ec8f0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d b/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39-14-bcc1d5fd287f81bac1092a913b09956d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 b/sql/hive/src/test/resources/golden/input39-2-ee667c6ab003bb83f6bf1c72153eba39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/sql/hive/src/test/resources/golden/input39-3-9e1699caf2caa347fa8ee6e9b6e7da6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 b/sql/hive/src/test/resources/golden/input39-4-53453776bf062f28d371fc7336b7eae2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39-5-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39-6-763ab5853bff619e6525c01e46b2a923 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39-7-9b141c1e5917ca82c6bc36a9a2950a1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39-8-2b9ccaa793eae0e73bf76335d3d6880 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39-9-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe b/sql/hive/src/test/resources/golden/input39_hadoop20-0-4c33233bafd910d69c2b8845d35abffe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 b/sql/hive/src/test/resources/golden/input39_hadoop20-1-92a6ee4486a9fc8fc7bc567e42b9e2a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 b/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 new file mode 100644 index 0000000000000..25bf17fc5aaab --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39_hadoop20-10-6d0814cbb72eb96bfc75c95d06f1e528 @@ -0,0 +1 @@ +18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 b/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39_hadoop20-11-f120ac8c87db9eebb5da7ac99b48600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 b/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39_hadoop20-12-a6a77ae749a7e7f8022d71c03322fc21 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 b/sql/hive/src/test/resources/golden/input39_hadoop20-2-ee667c6ab003bb83f6bf1c72153eba39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e b/sql/hive/src/test/resources/golden/input39_hadoop20-3-9e1699caf2caa347fa8ee6e9b6e7da6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 b/sql/hive/src/test/resources/golden/input39_hadoop20-4-53453776bf062f28d371fc7336b7eae2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee b/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39_hadoop20-5-823920925ca9c8a2ca9016f52c0f4ee @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39_hadoop20-6-763ab5853bff619e6525c01e46b2a923 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39_hadoop20-7-db1cd54a4cb36de2087605f32e41824f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input39_hadoop20-8-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 b/sql/hive/src/test/resources/golden/input39_hadoop20-9-28bf1b34c04a048da339eddd4c1fd779 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 b/sql/hive/src/test/resources/golden/input3_limit-0-27fc8f7d7456a761e1d0c2c075b84dc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 b/sql/hive/src/test/resources/golden/input3_limit-1-7b46b8baf9c7628da9c190c96b917057 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d b/sql/hive/src/test/resources/golden/input3_limit-2-c6583bdb759c8a050238a32a6ce8273d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece b/sql/hive/src/test/resources/golden/input3_limit-3-2a87d8faa18a6311376812bd0453fece new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d b/sql/hive/src/test/resources/golden/input3_limit-4-70dad45d534146923fce88b2ffb99b0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 b/sql/hive/src/test/resources/golden/input3_limit-5-3664b564747487df13a5d109837219b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 b/sql/hive/src/test/resources/golden/input3_limit-6-7fec232bd656e1c1cf6cd731afc55d67 new file mode 100644 index 0000000000000..6731b0cb0baaa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input4-0-b9cd4041ca510639838a97376747b498 b/sql/hive/src/test/resources/golden/input4-0-b9cd4041ca510639838a97376747b498 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 b/sql/hive/src/test/resources/golden/input4-1-c139adc70f9942e527142e3be7fd2b87 new file mode 100644 index 0000000000000..65a457b52b0a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 b/sql/hive/src/test/resources/golden/input4-2-b7f46eccd104e6ed1b29e2de45089f66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 b/sql/hive/src/test/resources/golden/input4-3-d9280be6c9dd1e2263f8d50f0f43a486 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 b/sql/hive/src/test/resources/golden/input4-4-271b04e0fbee2ee81bae21dcb46d55e4 new file mode 100644 index 0000000000000..218c223b050b7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input40-0-37e74908fd43254141ae3664f13a48d b/sql/hive/src/test/resources/golden/input40-0-37e74908fd43254141ae3664f13a48d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 b/sql/hive/src/test/resources/golden/input40-1-acb61ae95ffabcb4a8ea3444d704e6b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 b/sql/hive/src/test/resources/golden/input40-2-e7ab3c9244fcfda76061b4153d796e93 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input40-3-12f634800ffae17942ddd789a01af227 b/sql/hive/src/test/resources/golden/input40-3-12f634800ffae17942ddd789a01af227 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e b/sql/hive/src/test/resources/golden/input40-4-f241eafbb8d5da3f9c1737aed7b4f94e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 b/sql/hive/src/test/resources/golden/input40-5-ccdc9c87a8fdbeee0fa48927f9700361 new file mode 100644 index 0000000000000..6bc66cd8fe19b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 b/sql/hive/src/test/resources/golden/input40-6-93a0c37189dfe2318ba6ad84616f0d64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 b/sql/hive/src/test/resources/golden/input40-7-ccdc9c87a8fdbeee0fa48927f9700361 new file mode 100644 index 0000000000000..ee3dddd8cff71 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input41-0-763ab5853bff619e6525c01e46b2a923 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input41-1-8112b3a278e8337dbd6f017ae178658b b/sql/hive/src/test/resources/golden/input41-1-8112b3a278e8337dbd6f017ae178658b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 b/sql/hive/src/test/resources/golden/input41-2-61812b3093413d09f0fd2372c61f7d53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee b/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee new file mode 100644 index 0000000000000..579784a58a66c --- /dev/null +++ b/sql/hive/src/test/resources/golden/input41-3-526399455dc5ecd4ea9f676b09fafeee @@ -0,0 +1,2 @@ +0 +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a b/sql/hive/src/test/resources/golden/input43-0-2baba8070f3585debc14b6bb3c83607a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 b/sql/hive/src/test/resources/golden/input46-0-b0cdbecce0321ac452c8e13e1bfc6924 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f b/sql/hive/src/test/resources/golden/input46-1-1efdd5ebfa732abdedeb10467ca71f7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca b/sql/hive/src/test/resources/golden/input46-2-4b3b3bedcb5765c5cfaa5d8e8bfb69ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 b/sql/hive/src/test/resources/golden/input46-3-c185163787977498a4b84f39f983c431 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 b/sql/hive/src/test/resources/golden/input46-4-7f05cde078d90c25780a7d5491b20c27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab b/sql/hive/src/test/resources/golden/input46-5-f5c502e88a3dc3edb37b04af7d7955ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input49-0-2bd546beeb607da5b925ffbea20a741c b/sql/hive/src/test/resources/golden/input49-0-2bd546beeb607da5b925ffbea20a741c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d b/sql/hive/src/test/resources/golden/input49-1-e2b5acbef6ef09cb3eecdb0ce2729e8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 b/sql/hive/src/test/resources/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 new file mode 100644 index 0000000000000..0d3a5cb1af472 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input49-2-d56860e68d333bb6132ef5ed0327eb1 @@ -0,0 +1 @@ +false 2.9098519E7 1410.0 996 40408519555 test_string 12 diff --git a/sql/hive/src/test/resources/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 b/sql/hive/src/test/resources/golden/input49-3-4b0fff4bd5c63666ccc22b07a521b7b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 b/sql/hive/src/test/resources/golden/input4_cb_delim-0-f8e975c0fc126982e8e70cc30d2b6367 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 b/sql/hive/src/test/resources/golden/input4_cb_delim-1-5692d0e91dd0114729b8eb3aee388b72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 b/sql/hive/src/test/resources/golden/input4_cb_delim-2-e52787bf798a941c854eb09d75efe617 new file mode 100644 index 0000000000000..218c223b050b7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d b/sql/hive/src/test/resources/golden/input4_limit-0-4f0124854141b8be1defa7a6d0877d8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af b/sql/hive/src/test/resources/golden/input4_limit-1-c634fc723fb3aac3ce007069bdcb2af new file mode 100644 index 0000000000000..217a1915f8826 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea b/sql/hive/src/test/resources/golden/input5-0-659e06570690cceeb3f37e10e855d2ea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 b/sql/hive/src/test/resources/golden/input5-1-b1062ddf6be670dbfc66a6e7dc0e7a56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad b/sql/hive/src/test/resources/golden/input5-2-3d6eb15b4fe23d0a1aa303da818d97ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input5-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f8aa003a65bb4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input6-0-659e06570690cceeb3f37e10e855d2ea b/sql/hive/src/test/resources/golden/input6-0-659e06570690cceeb3f37e10e855d2ea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input6-1-21149162906e31964a328b6cb2d5286e b/sql/hive/src/test/resources/golden/input6-1-21149162906e31964a328b6cb2d5286e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c b/sql/hive/src/test/resources/golden/input6-2-9601ace6dce45dc8d6281cc248c9e28c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input6-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..b4dc488518f24 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 b/sql/hive/src/test/resources/golden/input7-0-8daa7823607b82336736a9d4f9a2ce45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 b/sql/hive/src/test/resources/golden/input7-1-bb25de1b9a7791ec5609303bab350da0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input7-2-c740098a289f9a4267d26fc10347a0dc b/sql/hive/src/test/resources/golden/input7-2-c740098a289f9a4267d26fc10347a0dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input7-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..0b8a8960a992b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa b/sql/hive/src/test/resources/golden/input8-0-584fc8ceef39e1e1333ebaeec3e9e7fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa b/sql/hive/src/test/resources/golden/input8-1-c9f9239e73b04fc5c0e0219f438ceefa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba b/sql/hive/src/test/resources/golden/input8-2-4a4cb89ba45d26bb3dd73e44620578ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input8-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..c87107a2f1168 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 b/sql/hive/src/test/resources/golden/input9-0-8cc1539c21d92af8ea8c1431b0e3d85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input9-1-b30105391cb53915997a84ca3137bd0a b/sql/hive/src/test/resources/golden/input9-1-b30105391cb53915997a84ca3137bd0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input9-2-171ea277f05a642699a0f91bacffccd8 b/sql/hive/src/test/resources/golden/input9-2-171ea277f05a642699a0f91bacffccd8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input9-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e b/sql/hive/src/test/resources/golden/input_columnarserde-0-df919fd41f281bf7b45a2340d0c9d43e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 b/sql/hive/src/test/resources/golden/input_columnarserde-1-2db9e6115d209fabcb0c06e5e666fa3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be b/sql/hive/src/test/resources/golden/input_columnarserde-2-ac60752a883d3204c215fa01811701be new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 b/sql/hive/src/test/resources/golden/input_columnarserde-3-3455e6f385b0f60be5d0c842ade0f1d7 new file mode 100644 index 0000000000000..e782acd4d1e7f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 b/sql/hive/src/test/resources/golden/input_columnarserde-4-c471b057cdb4f3c3405b311ea2b92aa3 new file mode 100644 index 0000000000000..6038b8aa32884 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d b/sql/hive/src/test/resources/golden/input_dynamicserde-0-92c95af00fd419aa106571f72fcad67d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc b/sql/hive/src/test/resources/golden/input_dynamicserde-1-39bc8a2aea379563a62a465cc54aecbc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 b/sql/hive/src/test/resources/golden/input_dynamicserde-2-7bee1cc403a04b53d0a0324819e1d768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_dynamicserde-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e782acd4d1e7f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 b/sql/hive/src/test/resources/golden/input_dynamicserde-4-8bf42e02f82b0ac58e7d0b525a993b31 new file mode 100644 index 0000000000000..6038b8aa32884 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d b/sql/hive/src/test/resources/golden/input_lazyserde-0-92c95af00fd419aa106571f72fcad67d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 b/sql/hive/src/test/resources/golden/input_lazyserde-1-bf8ce1d1366256d5c07fc4b55dde7ba7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 b/sql/hive/src/test/resources/golden/input_lazyserde-10-d915efbc5233a49f48e81e74eef2a3c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c b/sql/hive/src/test/resources/golden/input_lazyserde-11-6aaa88142e86a9a39e980caed145e32c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c b/sql/hive/src/test/resources/golden/input_lazyserde-12-abde20a4a37ed330bc7128576dd18d7c new file mode 100644 index 0000000000000..6af528eab23a8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc b/sql/hive/src/test/resources/golden/input_lazyserde-2-7a165139976654640c86db8e4e5871cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 b/sql/hive/src/test/resources/golden/input_lazyserde-3-bdf93641b97ab6347ece67e2fb636e97 new file mode 100644 index 0000000000000..e782acd4d1e7f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db b/sql/hive/src/test/resources/golden/input_lazyserde-4-8cc058fb7986f59965976cad813267db new file mode 100644 index 0000000000000..6038b8aa32884 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/input_lazyserde-5-460dffb0f1ab0ac0ebc4fd545809aa9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 b/sql/hive/src/test/resources/golden/input_lazyserde-6-42e2838ee69484bf5301475905cee12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 b/sql/hive/src/test/resources/golden/input_lazyserde-7-b44a44827ad0dce044badd6b258eabd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c b/sql/hive/src/test/resources/golden/input_lazyserde-8-abde20a4a37ed330bc7128576dd18d7c new file mode 100644 index 0000000000000..1bb008b44d6ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/input_lazyserde-9-460dffb0f1ab0ac0ebc4fd545809aa9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 b/sql/hive/src/test/resources/golden/input_limit-0-b2b4b45519484c8bac49d07debf678b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 b/sql/hive/src/test/resources/golden/input_limit-1-77d7012bc901b0a9bcf9dae500ae2034 new file mode 100644 index 0000000000000..17f24d2991d14 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 b/sql/hive/src/test/resources/golden/input_part0-0-5f2f3118b9f61ac610b2d76f470e3e27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb b/sql/hive/src/test/resources/golden/input_part0-1-1aa1486a207bedc275035acc3b37cbdb new file mode 100644 index 0000000000000..499e6b8ab6b7e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 b/sql/hive/src/test/resources/golden/input_part1-0-55f89c73a00f0f23ee04239ad9fc909 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 b/sql/hive/src/test/resources/golden/input_part1-1-593e11f2ca748f3ae6b8fdf5da435229 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc b/sql/hive/src/test/resources/golden/input_part1-2-4153379704a43162abf9dbdbd79a92bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_part1-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..3e5ae10e4670a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 b/sql/hive/src/test/resources/golden/input_part10-0-77d7f7c938d7827a2135d9391376a518 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part10-1-35f67c0112026170d015b5f80a254bde b/sql/hive/src/test/resources/golden/input_part10-1-35f67c0112026170d015b5f80a254bde new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part10-2-40069c199502c2724ac2a2733f964248 b/sql/hive/src/test/resources/golden/input_part10-2-40069c199502c2724ac2a2733f964248 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 b/sql/hive/src/test/resources/golden/input_part10-3-48b242bc305c9bf879e083fa11edc967 new file mode 100644 index 0000000000000..82116102c1f54 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 b/sql/hive/src/test/resources/golden/input_part10-4-d0ba28297a8b73569d93605aa890aa09 new file mode 100644 index 0000000000000..c8d0d55930069 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf b/sql/hive/src/test/resources/golden/input_part10_win-0-93c438f3bbfa5c46514f0ab6b83a59bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde b/sql/hive/src/test/resources/golden/input_part10_win-1-35f67c0112026170d015b5f80a254bde new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 b/sql/hive/src/test/resources/golden/input_part10_win-2-40069c199502c2724ac2a2733f964248 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 b/sql/hive/src/test/resources/golden/input_part10_win-3-48b242bc305c9bf879e083fa11edc967 new file mode 100644 index 0000000000000..869eb58e70d82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 b/sql/hive/src/test/resources/golden/input_part10_win-4-d0ba28297a8b73569d93605aa890aa09 new file mode 100644 index 0000000000000..c8d0d55930069 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 b/sql/hive/src/test/resources/golden/input_part2-0-55f89c73a00f0f23ee04239ad9fc909 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e b/sql/hive/src/test/resources/golden/input_part2-1-5fde770f3a672878b62d7c2e9e9a8f8e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd b/sql/hive/src/test/resources/golden/input_part2-2-a8684c5574f90c3db2fc8a269a7556cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 b/sql/hive/src/test/resources/golden/input_part2-3-fcb06513e757d4bf929ff681cb9f02b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 b/sql/hive/src/test/resources/golden/input_part2-4-93c97e1760e0d41b3791d6f08010a665 new file mode 100644 index 0000000000000..fd945cc15d9ca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 b/sql/hive/src/test/resources/golden/input_part2-5-9d0992a91951e6e4242b5b4c38d4e861 new file mode 100644 index 0000000000000..f23877361f33b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff b/sql/hive/src/test/resources/golden/input_part3-0-aed975ebb2a3dfee3f8747c00ea2c4ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 b/sql/hive/src/test/resources/golden/input_part3-1-ba5256285fb22a43b491253a2d519730 new file mode 100644 index 0000000000000..f4026a591a958 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef b/sql/hive/src/test/resources/golden/input_part4-0-6c73b8b3f885b8fe0e61a7111f5cc4ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 b/sql/hive/src/test/resources/golden/input_part4-1-88be7d93e9a03ea76085111c18d437a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b b/sql/hive/src/test/resources/golden/input_part5-0-679bf51b7de8df94fd2928744e887e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c b/sql/hive/src/test/resources/golden/input_part5-1-705301f0bf4fe9758e9f919d129dbc7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part5-2-d678533566fba4b922f01284538ca484 b/sql/hive/src/test/resources/golden/input_part5-2-d678533566fba4b922f01284538ca484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 b/sql/hive/src/test/resources/golden/input_part5-3-e4419c33287ca1f48a43f61cca5b5928 new file mode 100644 index 0000000000000..853c3bc8df7f0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d b/sql/hive/src/test/resources/golden/input_part6-0-6ec3d5a634d8c011f32a0277e654ae6d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 b/sql/hive/src/test/resources/golden/input_part6-1-9ffee7a86f484eab399266efa6e086f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part7-0-14448a12627b8bccf6828032f938f39e b/sql/hive/src/test/resources/golden/input_part7-0-14448a12627b8bccf6828032f938f39e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 b/sql/hive/src/test/resources/golden/input_part7-1-affad4cedcd29bb136bc477fc07e6ea0 new file mode 100644 index 0000000000000..540ca86636f3c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part8-0-f60652796af548135f9e104b456840e3 b/sql/hive/src/test/resources/golden/input_part8-0-f60652796af548135f9e104b456840e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 b/sql/hive/src/test/resources/golden/input_part8-1-60b409a520999ba50e8b7c6e30de3474 new file mode 100644 index 0000000000000..a1728c82f0b35 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 b/sql/hive/src/test/resources/golden/input_part9-0-726907a7cf62f5a79466caa0fa6eca01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 b/sql/hive/src/test/resources/golden/input_part9-1-e60c60afc073367464898b8396e8f643 new file mode 100644 index 0000000000000..438355d7b06f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 b/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-0-68975193b30cb34102b380e647d8d5f4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e b/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-1-1c0f3be2d837dee49312e0a80440447e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf b/sql/hive/src/test/resources/golden/input_testsequencefile-2-186e4009966778e765b18d6476cf5abf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 b/sql/hive/src/test/resources/golden/input_testsequencefile-3-d0fade04b658b0dc0caf3fe4b2d5d432 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 b/sql/hive/src/test/resources/golden/input_testsequencefile-4-8ebe479eca517e80bf0348a1b3a89f42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c b/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input_testsequencefile-5-3708198aac609695b22e19e89306034c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 b/sql/hive/src/test/resources/golden/input_testsequencefile-6-6716fa5aec125f0f8e9520e4318f71b9 new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 b/sql/hive/src/test/resources/golden/input_testxpath-0-3c8a098a179d578119f75e5d7b214bd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f b/sql/hive/src/test/resources/golden/input_testxpath-1-ac18d29e8bd8aeed00296992edc17c0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 b/sql/hive/src/test/resources/golden/input_testxpath-2-86b0a60940ffc4bdaafcc4f5a8c0972 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_testxpath-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..c1a6abba038e7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 b/sql/hive/src/test/resources/golden/input_testxpath2-0-a1baa22f15f745a2dfe27ce52d363704 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 b/sql/hive/src/test/resources/golden/input_testxpath2-1-27b77465d23aebf66a50c8074a75b755 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd b/sql/hive/src/test/resources/golden/input_testxpath2-2-6b8b2daaeaa985a7de0e377ffc4436dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/input_testxpath2-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..51645b2a07a39 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac b/sql/hive/src/test/resources/golden/input_testxpath3-0-f05134d54292acd1f2067027889a4fac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 b/sql/hive/src/test/resources/golden/input_testxpath3-1-807b097ac2f785f774db03069ebbde11 new file mode 100644 index 0000000000000..373a573714f4e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input_testxpath4-0-73819ea1a7c0653a61652b3766afb003 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 b/sql/hive/src/test/resources/golden/input_testxpath4-1-f746888141a38ba707fad01d86d41960 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 b/sql/hive/src/test/resources/golden/input_testxpath4-2-e4beab1294dcac60ff36e4f2561810b9 new file mode 100644 index 0000000000000..7490d2d44d71a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/input_testxpath4-3-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 b/sql/hive/src/test/resources/golden/input_testxpath4-4-f746888141a38ba707fad01d86d41960 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 b/sql/hive/src/test/resources/golden/input_testxpath4-5-e4beab1294dcac60ff36e4f2561810b9 new file mode 100644 index 0000000000000..7490d2d44d71a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 b/sql/hive/src/test/resources/golden/inputddl4-0-2f5570984afde1a30ff2f794d63d6ab3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 b/sql/hive/src/test/resources/golden/inputddl4-1-dd94576788fa50ebcf950cdf837fbcf6 new file mode 100644 index 0000000000000..d5a489a9a5ed5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 b/sql/hive/src/test/resources/golden/inputddl4-2-7fdf00ff5c22ae284728e0f035396865 new file mode 100644 index 0000000000000..e0bedb0512cfc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 b/sql/hive/src/test/resources/golden/inputddl6-0-baa412d895c4fc57d40b338944a0ecd8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf b/sql/hive/src/test/resources/golden/inputddl6-1-8b96b4fba4cf54c399a008d0f3be1edf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c b/sql/hive/src/test/resources/golden/inputddl6-2-c4c902d39d8dd9568f1d95ac3a8e5c6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c b/sql/hive/src/test/resources/golden/inputddl6-3-e81b962986706e1c16f059b407e3f05c new file mode 100644 index 0000000000000..f1427896e8330 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 b/sql/hive/src/test/resources/golden/inputddl6-4-5855e2998e26f63e927854afa86c1f03 new file mode 100644 index 0000000000000..822897217e867 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 b/sql/hive/src/test/resources/golden/inputddl6-5-47e6a8d33932cb014830812e1f7b1f94 new file mode 100644 index 0000000000000..94bcaaee2408c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 b/sql/hive/src/test/resources/golden/inputddl6-6-6eac06a114fe33c1278d47ad8652fe68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 b/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 new file mode 100644 index 0000000000000..b12a9f82cd90a --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl6-7-47e6a8d33932cb014830812e1f7b1f94 @@ -0,0 +1 @@ +ds=2008-04-09 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 b/sql/hive/src/test/resources/golden/inputddl6-8-f9e6ee98eb448f9ab68fa77bea027aa5 new file mode 100644 index 0000000000000..a6c282ab6f573 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 b/sql/hive/src/test/resources/golden/inputddl6-9-2164df0b7eab73cd8c4a553a762028e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f b/sql/hive/src/test/resources/golden/inputddl7-0-6a30e2d3fc23ec0c18513265bd8c987f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 b/sql/hive/src/test/resources/golden/inputddl7-1-7b46b8baf9c7628da9c190c96b917057 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d b/sql/hive/src/test/resources/golden/inputddl7-10-4eec8cef401b6bba00457dfbec58cc2d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 b/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl7-11-6e1f1709d770ed76aee6ff5f76856e63 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 b/sql/hive/src/test/resources/golden/inputddl7-12-2c56d4a781242b0521f82bb0d2cd277 new file mode 100644 index 0000000000000..ef633a4aa09e0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a b/sql/hive/src/test/resources/golden/inputddl7-13-45059a21c202b4658285738ee62a018a new file mode 100644 index 0000000000000..86d5d8a125fbe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 b/sql/hive/src/test/resources/golden/inputddl7-14-30c87bc734c2afa4fea0facdf7279145 new file mode 100644 index 0000000000000..b14d78536150c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe b/sql/hive/src/test/resources/golden/inputddl7-15-fed9badc255db68c3ed1cd1446d9fbe new file mode 100644 index 0000000000000..e476a7fa3959b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 b/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl7-2-7bc7f4f4a29dc0721ad4bb6fb9b64291 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 b/sql/hive/src/test/resources/golden/inputddl7-3-b44cf61cbfeb5a4ddf5acbff1429b466 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 b/sql/hive/src/test/resources/golden/inputddl7-4-7513658e8abe9d9b72c7219321c56fa9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 b/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl7-5-e3c5d1248a06e6b33b15fc6ec8c67f68 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a b/sql/hive/src/test/resources/golden/inputddl7-6-a590512af0ed021d33e2d71b7932a20a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 b/sql/hive/src/test/resources/golden/inputddl7-7-48640fff8428a0dc6e90a7243adaf730 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 b/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/inputddl7-8-495dc87b0bde752c890f213ff9531508 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac b/sql/hive/src/test/resources/golden/inputddl7-9-c943e781fb448aea5467251ee208fbac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 b/sql/hive/src/test/resources/golden/inputddl8-0-6de6ae6cbb97de8a2d5674b59a154cf3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 b/sql/hive/src/test/resources/golden/inputddl8-1-c70f2d2544633366b76b92bcff18e995 new file mode 100644 index 0000000000000..66e6efceed3dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/insert into and insert overwrite-0-2e2de36571343c0ebaf02c0c0c61ba8c b/sql/hive/src/test/resources/golden/insert into and insert overwrite-0-2e2de36571343c0ebaf02c0c0c61ba8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert into and insert overwrite-1-548d5a45ef20e066df145944dce3415a b/sql/hive/src/test/resources/golden/insert into and insert overwrite-1-548d5a45ef20e066df145944dce3415a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert into and insert overwrite-2-77baf70da8adf8b4b82d076fee937fe1 b/sql/hive/src/test/resources/golden/insert into and insert overwrite-2-77baf70da8adf8b4b82d076fee937fe1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert into and insert overwrite-3-b9002c1d71895be765575b62656d1928 b/sql/hive/src/test/resources/golden/insert into and insert overwrite-3-b9002c1d71895be765575b62656d1928 new file mode 100644 index 0000000000000..813f70020af3e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/insert into and insert overwrite-4-3b8e0155870264d89d275a1d5bd345d0 b/sql/hive/src/test/resources/golden/insert into and insert overwrite-4-3b8e0155870264d89d275a1d5bd345d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert into and insert overwrite-5-b9002c1d71895be765575b62656d1928 b/sql/hive/src/test/resources/golden/insert into and insert overwrite-5-b9002c1d71895be765575b62656d1928 new file mode 100644 index 0000000000000..e87fb81a6f043 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert into and insert overwrite-5-b9002c1d71895be765575b62656d1928 @@ -0,0 +1 @@ +86 val_86 diff --git a/sql/hive/src/test/resources/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac b/sql/hive/src/test/resources/golden/insert table with db name-0-a253b1ed35dbf503d1b8902dacbe23ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 b/sql/hive/src/test/resources/golden/insert table with db name-1-4f5be7056d8dc15cf2334a7cc146a636 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 b/sql/hive/src/test/resources/golden/insert table with db name-2-a81aea129f78bc05305b8e887ee88f86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 b/sql/hive/src/test/resources/golden/insert table with db name-3-ce780d068b8d24786e639e361101a0c7 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c b/sql/hive/src/test/resources/golden/insert table with db name-4-afd6e46b6a289c3c24a8eec75a94043c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_compressed-0-ea607fbed28d20e5726f4501285d698d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 b/sql/hive/src/test/resources/golden/insert_compressed-1-d20899578068ff4adfee5355cddca819 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 b/sql/hive/src/test/resources/golden/insert_compressed-2-8dec751fd4148a431af064b7317f1530 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 b/sql/hive/src/test/resources/golden/insert_compressed-3-35f4fbde823a5664fe9928a685745b35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_compressed-4-5133d2457097962811a2adf0ecd9e4ef @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 b/sql/hive/src/test/resources/golden/insert_compressed-5-a60ea2e04d6bcdc99238eeb6ac6365c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef new file mode 100644 index 0000000000000..e37d32abba426 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_compressed-6-5133d2457097962811a2adf0ecd9e4ef @@ -0,0 +1 @@ +1000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 b/sql/hive/src/test/resources/golden/insert_compressed-7-a60ea2e04d6bcdc99238eeb6ac6365c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef b/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef new file mode 100644 index 0000000000000..37021f4a27201 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_compressed-8-5133d2457097962811a2adf0ecd9e4ef @@ -0,0 +1 @@ +1500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 b/sql/hive/src/test/resources/golden/insert_compressed-9-d20899578068ff4adfee5355cddca819 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 b/sql/hive/src/test/resources/golden/insert_into1-0-ae5ea07929262bde22fbe7ebe80d4992 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b b/sql/hive/src/test/resources/golden/insert_into1-1-f1699bc0ef0a84dd9c23ccff37e13d7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 b/sql/hive/src/test/resources/golden/insert_into1-10-c260979323c1ebdf68c6fbe003d43792 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb new file mode 100644 index 0000000000000..5e96d815b6b78 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into1-11-41015d6409c5ebf670eed4999157fdb @@ -0,0 +1 @@ +-826625916 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 b/sql/hive/src/test/resources/golden/insert_into1-12-ae5ea07929262bde22fbe7ebe80d4992 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 b/sql/hive/src/test/resources/golden/insert_into1-2-ff6a1b25c911def274921df1bae476b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 b/sql/hive/src/test/resources/golden/insert_into1-3-6687f7d58bd1faf1c2ee4f52f03ce048 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb new file mode 100644 index 0000000000000..eb9dc5833c2f9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into1-4-41015d6409c5ebf670eed4999157fdb @@ -0,0 +1 @@ +10226524244 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 b/sql/hive/src/test/resources/golden/insert_into1-5-d47a5e2ff879b37c8b6ca948ed47b7d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 b/sql/hive/src/test/resources/golden/insert_into1-6-4b2e32fe57b2297d22bccb4656bdee30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb b/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb new file mode 100644 index 0000000000000..28ced898ab537 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into1-7-41015d6409c5ebf670eed4999157fdb @@ -0,0 +1 @@ +20453048488 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d b/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d new file mode 100644 index 0000000000000..ae4ee13c08e76 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into1-8-2a2bd9dc41eaa041aff7eca70a72cb0d @@ -0,0 +1 @@ +200 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b b/sql/hive/src/test/resources/golden/insert_into1-9-31eb4770dc60e4765065ac7f84811d1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 b/sql/hive/src/test/resources/golden/insert_into2-0-957c6402df0fd7d2fccbc688e49e9661 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 b/sql/hive/src/test/resources/golden/insert_into2-1-9828bb831fd11667b48678e5952a0941 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 b/sql/hive/src/test/resources/golden/insert_into2-10-df53336f364fe09e9591e769c13b5519 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa b/sql/hive/src/test/resources/golden/insert_into2-11-a19a7ab57f6ff69f1dff405bc3d4b7aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 new file mode 100644 index 0000000000000..84d7f3929d86d --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into2-12-4cf03cb3982a457f2f72220265ecc844 @@ -0,0 +1 @@ +-27100860056 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 b/sql/hive/src/test/resources/golden/insert_into2-13-957c6402df0fd7d2fccbc688e49e9661 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 b/sql/hive/src/test/resources/golden/insert_into2-2-f83e3ad75a2c7b290f8cf5f6153b9671 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 b/sql/hive/src/test/resources/golden/insert_into2-3-9d5556040de01fd02d5501d141effff2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 b/sql/hive/src/test/resources/golden/insert_into2-4-9d5556040de01fd02d5501d141effff2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 b/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 new file mode 100644 index 0000000000000..ae4ee13c08e76 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into2-5-de6e50a2ae0ab5a9c466998b57f86b08 @@ -0,0 +1 @@ +200 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 new file mode 100644 index 0000000000000..3395f3bcc7b51 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into2-6-4cf03cb3982a457f2f72220265ecc844 @@ -0,0 +1 @@ +-24159954504 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 b/sql/hive/src/test/resources/golden/insert_into2-7-6db9da2d1a64be8cd58f0313c4970fc9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 b/sql/hive/src/test/resources/golden/insert_into2-8-452111285dda40205ee587de8e972896 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 b/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 new file mode 100644 index 0000000000000..ee0a47c9f6e00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into2-9-4cf03cb3982a457f2f72220265ecc844 @@ -0,0 +1 @@ +-36239931656 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 b/sql/hive/src/test/resources/golden/insert_into3-0-7e35c30348a53b0748bc1bb85f75c195 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 b/sql/hive/src/test/resources/golden/insert_into3-1-f6dd7262d45a21781d85bc343ede8fb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 b/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 new file mode 100644 index 0000000000000..5e96d815b6b78 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into3-10-40a108b4c89bc5d6424f21f3b8a2f5e7 @@ -0,0 +1 @@ +-826625916 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 b/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 new file mode 100644 index 0000000000000..2ed5a7da11dcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into3-11-9cfd2d054f84262eb74a870b6365db87 @@ -0,0 +1 @@ +9399898328 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 b/sql/hive/src/test/resources/golden/insert_into3-12-7e35c30348a53b0748bc1bb85f75c195 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 b/sql/hive/src/test/resources/golden/insert_into3-13-f6dd7262d45a21781d85bc343ede8fb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 b/sql/hive/src/test/resources/golden/insert_into3-2-e51c25bae2408422a56826a263479468 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d b/sql/hive/src/test/resources/golden/insert_into3-3-c46699c465fefe6baab35499a32b452d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd b/sql/hive/src/test/resources/golden/insert_into3-4-e9f4f47686fe97482b0a769a15481dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 b/sql/hive/src/test/resources/golden/insert_into3-5-a83d22fa3c2fb32bd08305a1729bf7f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 b/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 new file mode 100644 index 0000000000000..00ffdd24b0cff --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into3-6-40a108b4c89bc5d6424f21f3b8a2f5e7 @@ -0,0 +1 @@ +7813690682 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 b/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 new file mode 100644 index 0000000000000..eb9dc5833c2f9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into3-7-9cfd2d054f84262eb74a870b6365db87 @@ -0,0 +1 @@ +10226524244 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 b/sql/hive/src/test/resources/golden/insert_into3-8-e3b8d90def4a6ec1e3b5ee9cdaf31c01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 b/sql/hive/src/test/resources/golden/insert_into3-9-19d1be183f7985f7066f86572abc82c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 b/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into4-0-b4831621b2a02fc4e8e655b03c289310 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 b/sql/hive/src/test/resources/golden/insert_into4-1-3d466d45197fcf1eff55d76ef0a29720 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 b/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 new file mode 100644 index 0000000000000..e2954bd63682e --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into4-10-a8016ca816f7b362cf3a0384ca91b159 @@ -0,0 +1 @@ +-1653251832 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa b/sql/hive/src/test/resources/golden/insert_into4-11-bdbfaf06bc4e323863db65fd29882eaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 b/sql/hive/src/test/resources/golden/insert_into4-12-49b12993ebc1647b9dca8b9af19deca9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b b/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b new file mode 100644 index 0000000000000..e2954bd63682e --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into4-13-a5bb41af609f38f01d8c71334cc8d71b @@ -0,0 +1 @@ +-1653251832 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 b/sql/hive/src/test/resources/golden/insert_into4-14-3d466d45197fcf1eff55d76ef0a29720 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 b/sql/hive/src/test/resources/golden/insert_into4-15-f6ad1a49459fb6cd232fccc4a6062b25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 b/sql/hive/src/test/resources/golden/insert_into4-2-f6ad1a49459fb6cd232fccc4a6062b25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b b/sql/hive/src/test/resources/golden/insert_into4-3-43629aaa698fb6e2db4586124561e69b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a b/sql/hive/src/test/resources/golden/insert_into4-4-8adba808fd505f4bf0ffcc61a618480a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 b/sql/hive/src/test/resources/golden/insert_into4-5-6bc47894aa917051abb98d0b52f43881 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 b/sql/hive/src/test/resources/golden/insert_into4-6-8c1683bee2927da76bb0dbf44a373738 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 b/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 new file mode 100644 index 0000000000000..5e96d815b6b78 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into4-7-a8016ca816f7b362cf3a0384ca91b159 @@ -0,0 +1 @@ +-826625916 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 b/sql/hive/src/test/resources/golden/insert_into4-8-6bc47894aa917051abb98d0b52f43881 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 b/sql/hive/src/test/resources/golden/insert_into4-9-8c1683bee2927da76bb0dbf44a373738 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 b/sql/hive/src/test/resources/golden/insert_into5-0-9afa473f2111cf0d9ae62041bd97f840 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c b/sql/hive/src/test/resources/golden/insert_into5-1-8fc8db6a5564324339192f23208ffc1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 b/sql/hive/src/test/resources/golden/insert_into5-10-375cae396c768c1efe5d17b9f5f45f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d b/sql/hive/src/test/resources/golden/insert_into5-11-a36eeaf2fb0ef52b63a21f0f9b80b27d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 b/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 new file mode 100644 index 0000000000000..3b6ef434b37e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into5-12-a49d025c7556a83f301b6ecf9ceb2ce7 @@ -0,0 +1 @@ +-18626052920 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 b/sql/hive/src/test/resources/golden/insert_into5-13-e06a0b7252278141d50466e08f15b391 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 b/sql/hive/src/test/resources/golden/insert_into5-14-a3a4c16b0b723f97f654b4b5e80575c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 b/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 new file mode 100644 index 0000000000000..0744f3dae0e0a --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into5-15-a49d025c7556a83f301b6ecf9ceb2ce7 @@ -0,0 +1 @@ +-37252105840 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 b/sql/hive/src/test/resources/golden/insert_into5-16-9afa473f2111cf0d9ae62041bd97f840 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c b/sql/hive/src/test/resources/golden/insert_into5-2-548a3a10c15c333c63fc1c239ee8b62c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 b/sql/hive/src/test/resources/golden/insert_into5-3-a4b25f172af356ec98035329b95ddbd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 b/sql/hive/src/test/resources/golden/insert_into5-4-3d5343a79ee8b680f3b74b22db6658e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 b/sql/hive/src/test/resources/golden/insert_into5-5-f382c5b31afe9e0251fa9457c46c12a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 b/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 new file mode 100644 index 0000000000000..c0066b75af40e --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into5-6-a058ba199b9777d48b6c6595f2388533 @@ -0,0 +1 @@ +481928560 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f b/sql/hive/src/test/resources/golden/insert_into5-7-2004b4ecf6ceddb033727f8d5ebab42f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 b/sql/hive/src/test/resources/golden/insert_into5-8-4e489b45a3dacf7fdf0b12e4f8ab7873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 b/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 new file mode 100644 index 0000000000000..0b4a44e064f85 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into5-9-a058ba199b9777d48b6c6595f2388533 @@ -0,0 +1 @@ +963857120 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into6-0-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into6-1-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 b/sql/hive/src/test/resources/golden/insert_into6-10-16500f4af2c8638a670e867e59f6d457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba b/sql/hive/src/test/resources/golden/insert_into6-11-95297f4dcc4dd6a51de4785ccd25fbba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 b/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 new file mode 100644 index 0000000000000..d541b5d207233 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into6-12-29afaab2cf10eaa65afaa6c0fcca0902 @@ -0,0 +1 @@ +-35226404960 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 b/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 new file mode 100644 index 0000000000000..c21f4017362c1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into6-13-b357a845fb5f668b09dea94c6d2e7d66 @@ -0,0 +1,2 @@ +ds=1 +ds=2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 b/sql/hive/src/test/resources/golden/insert_into6-14-1f00d9cc34225cb358a54c6d4008cb47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/sql/hive/src/test/resources/golden/insert_into6-15-14d4d62ee9a5da9fbd7c3e0c021fdc0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 b/sql/hive/src/test/resources/golden/insert_into6-2-1f00d9cc34225cb358a54c6d4008cb47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d b/sql/hive/src/test/resources/golden/insert_into6-3-14d4d62ee9a5da9fbd7c3e0c021fdc0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f b/sql/hive/src/test/resources/golden/insert_into6-4-5f12794e99c74355a23d2fda9c7c170f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 b/sql/hive/src/test/resources/golden/insert_into6-5-de641eb41a9100e755a9ae641c752b30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 b/sql/hive/src/test/resources/golden/insert_into6-6-f6e7141a435922193937aa10085b0656 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 b/sql/hive/src/test/resources/golden/insert_into6-7-6826c610b8d04ab3464712bd9cddbcd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 b/sql/hive/src/test/resources/golden/insert_into6-8-33ec9514947e0b737e957bdcbbd87573 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab b/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab new file mode 100644 index 0000000000000..d541b5d207233 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert_into6-9-3ccb7bc735e406ad1723c758f01bcaab @@ -0,0 +1 @@ +-35226404960 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 b/sql/hive/src/test/resources/golden/insertexternal1-0-eb0745518b859c8497506a627bfd9860 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a b/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/insertexternal1-1-6d1b6c44f9f52ec67f9d4a3cdf580a1a @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b b/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/insertexternal1-2-bc513eeb5cbba0b15c8f425b9cc3cd7b @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 b/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/insertexternal1-3-f64289fb03ab105e12659fc3972ca241 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 b/sql/hive/src/test/resources/golden/join0-0-7fbd114e2fc7f55cf1421698eac39397 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join0-1-83f5e512d6058a47f92b5218781f5070 b/sql/hive/src/test/resources/golden/join0-1-83f5e512d6058a47f92b5218781f5070 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 b/sql/hive/src/test/resources/golden/join0-2-52055f2dce8eac4e5249d02c42c0da87 new file mode 100644 index 0000000000000..5707ed08e7e54 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join1-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e b/sql/hive/src/test/resources/golden/join1-1-f1293ebf768eb04f2f0bfe6297c4509e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join1-2-7b07671864bbfdc627ee794932e32b1e b/sql/hive/src/test/resources/golden/join1-2-7b07671864bbfdc627ee794932e32b1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 b/sql/hive/src/test/resources/golden/join1-3-f6046c5229e3b0aa21498a3872f43b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 b/sql/hive/src/test/resources/golden/join1-4-35c7611180562dcd9dab834f41654095 new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 b/sql/hive/src/test/resources/golden/join10-0-1e7a0e1d539a39ed19833e6a6d0b1644 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d b/sql/hive/src/test/resources/golden/join10-1-73136f8e6e9ba82f75570afd15c2828d new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 b/sql/hive/src/test/resources/golden/join11-0-e2cc67c543c1209ebcd3f9048c4cb29 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 b/sql/hive/src/test/resources/golden/join11-1-b580f87daf1763cd8c5e59ad5b271232 new file mode 100644 index 0000000000000..c888cfd6a0479 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 b/sql/hive/src/test/resources/golden/join12-0-12fa53be2c0ddbcfc85212e573a46af1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f b/sql/hive/src/test/resources/golden/join12-1-496c8aabaf3261e8fefd7b357f2ac7f new file mode 100644 index 0000000000000..1780076ae513f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 b/sql/hive/src/test/resources/golden/join13-0-c4ebf24269e27919f4ba9ce3993e48f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 b/sql/hive/src/test/resources/golden/join13-1-696b36d15c4358145f77c2b15b7507d5 new file mode 100644 index 0000000000000..a82a9a22c6b87 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 b/sql/hive/src/test/resources/golden/join14-0-c85f3dcbab496811604ea0ab84d0e995 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e b/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join14-1-9b141c1e5917ca82c6bc36a9a2950a1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 b/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join14-2-2b9ccaa793eae0e73bf76335d3d6880 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join14-3-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 b/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join14-4-27f1a57fbb815d169af86ae2f8305cb6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 b/sql/hive/src/test/resources/golden/join14-5-4c3f537b4df1ef16788a53cf65574187 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join14-6-de39302191b63d7aa8f92885b089fe2 b/sql/hive/src/test/resources/golden/join14-6-de39302191b63d7aa8f92885b089fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/sql/hive/src/test/resources/golden/join14-7-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 new file mode 100644 index 0000000000000..941e6bb7c887e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 b/sql/hive/src/test/resources/golden/join14_hadoop20-0-98b7542190092fafcc8b1ad5b0024a22 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f b/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join14_hadoop20-1-db1cd54a4cb36de2087605f32e41824f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join14_hadoop20-2-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 b/sql/hive/src/test/resources/golden/join14_hadoop20-3-4c3f537b4df1ef16788a53cf65574187 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 b/sql/hive/src/test/resources/golden/join14_hadoop20-4-de39302191b63d7aa8f92885b089fe2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 b/sql/hive/src/test/resources/golden/join14_hadoop20-5-4e04dbb8b2e0ee18f6cb4bd89b0ae3d3 new file mode 100644 index 0000000000000..941e6bb7c887e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join15-0-930db2823512f7f8f34cac104f2162e0 b/sql/hive/src/test/resources/golden/join15-0-930db2823512f7f8f34cac104f2162e0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 b/sql/hive/src/test/resources/golden/join15-1-81d76d3bf59889b07b413b6f88772667 new file mode 100644 index 0000000000000..c2e3ea8b0c8e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c b/sql/hive/src/test/resources/golden/join16-0-1bec3b0892d5c4a174d1b39f6d1b610c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 b/sql/hive/src/test/resources/golden/join17-0-387dd86b1e13f788ec677a08dc162c97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b b/sql/hive/src/test/resources/golden/join17-1-37cef87fe56f66692799ccda8cda2e8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 b/sql/hive/src/test/resources/golden/join17-2-478a9f270a5d70f6f82f81e6962fb251 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join17-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..66fafbdf72b63 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join18-0-269843197b598569f4b50cbe911960a5 b/sql/hive/src/test/resources/golden/join18-0-269843197b598569f4b50cbe911960a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 b/sql/hive/src/test/resources/golden/join18-1-49f8ba8d43a6bb1d892ba66a812304f5 new file mode 100644 index 0000000000000..10628f6c64bd4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 b/sql/hive/src/test/resources/golden/join19-0-26ba1cd9acfcd799239da60c38e68a39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join19-1-7e7d1f07c34dd4be5425264196201766 b/sql/hive/src/test/resources/golden/join19-1-7e7d1f07c34dd4be5425264196201766 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca b/sql/hive/src/test/resources/golden/join2-0-3d1692c4710db1ff716d35e921f2bcca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c b/sql/hive/src/test/resources/golden/join2-1-87896c3f04a259b81b1cfde3eec7d64c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a b/sql/hive/src/test/resources/golden/join2-2-d4673c03d04084b838fcd8149f59ad9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 b/sql/hive/src/test/resources/golden/join2-3-cac2c9e0f8601dd56822c990774e0696 new file mode 100644 index 0000000000000..2945c31ed4f00 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e b/sql/hive/src/test/resources/golden/join20-0-16cda49c2b2cd0c0cdae40af4d5c900e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 b/sql/hive/src/test/resources/golden/join20-1-9685f2d31ffe922d3ea217de32ca3049 new file mode 100644 index 0000000000000..bd6e20b8de1e4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join20-2-222655137ca9242f063c7717acbe4c65 b/sql/hive/src/test/resources/golden/join20-2-222655137ca9242f063c7717acbe4c65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 b/sql/hive/src/test/resources/golden/join20-3-3331a020843caccf2fa32a1225c5c3a9 new file mode 100644 index 0000000000000..bd6e20b8de1e4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 b/sql/hive/src/test/resources/golden/join21-0-d89bb8082a39fcaf8ed23a0f5b1cb410 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f b/sql/hive/src/test/resources/golden/join21-1-3536b7d78713e86ee67f5f6c9b88958f new file mode 100644 index 0000000000000..9672e21fa0323 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join22-0-97081b5f73052a612fb391a57ad9b7b b/sql/hive/src/test/resources/golden/join22-0-97081b5f73052a612fb391a57ad9b7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d b/sql/hive/src/test/resources/golden/join23-0-1fb76d3404f09a5c2b10422157c3cf5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 b/sql/hive/src/test/resources/golden/join23-1-91b8e7fe75a7e3ba8147c56734436681 new file mode 100644 index 0000000000000..5707ed08e7e54 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join24-0-721dfa03bfea05e55506c571b6c3585b b/sql/hive/src/test/resources/golden/join24-0-721dfa03bfea05e55506c571b6c3585b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d b/sql/hive/src/test/resources/golden/join24-1-36de83b0ed6c9fdc03661b2f65b23a3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 b/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/join24-2-d79325ef6494aa87843fdfd78de7c812 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join25-0-8934d9b821aa4b34b760f73eff56cd06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join25-1-360b3676605ccb3d89cf555342db87af b/sql/hive/src/test/resources/golden/join25-1-360b3676605ccb3d89cf555342db87af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join25-2-ef930bed933341636c71622e6f89e9e9 b/sql/hive/src/test/resources/golden/join25-2-ef930bed933341636c71622e6f89e9e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 b/sql/hive/src/test/resources/golden/join25-3-e39577008d28ddd5741d8518b92eaa94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join25-4-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..183353e5c705b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join26-0-775b1af8eae9b2250052be1b72b4086 b/sql/hive/src/test/resources/golden/join26-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join26-1-10fbe6c99e93cfeafcd454261744728e b/sql/hive/src/test/resources/golden/join26-1-10fbe6c99e93cfeafcd454261744728e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d b/sql/hive/src/test/resources/golden/join26-2-8aa0193a2a80d4efe4c66bca19218a4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join26-3-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..4c88927e56579 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join27-0-360b3676605ccb3d89cf555342db87af b/sql/hive/src/test/resources/golden/join27-0-360b3676605ccb3d89cf555342db87af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 b/sql/hive/src/test/resources/golden/join27-1-d56d8bbececcf48417b7dde9292a9cc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 b/sql/hive/src/test/resources/golden/join27-2-6d139e1aa07ff8d02bdf52b409956879 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 b/sql/hive/src/test/resources/golden/join27-3-e86808fdbd54120d1e2356f8f61c02f9 new file mode 100644 index 0000000000000..2b75023ea4a9f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 b/sql/hive/src/test/resources/golden/join28-0-7f2c5c987bc8e918f7b6fc222e1abc62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join28-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join28-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join28-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 b/sql/hive/src/test/resources/golden/join28-4-bd4e19a3608681b0270668fe68b7d124 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 b/sql/hive/src/test/resources/golden/join28-5-24cc9ff9485313ade08ee83ecc4c0621 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join28-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..da3c427cab73f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join29-0-e78402f7585b17c76c32790571d749eb b/sql/hive/src/test/resources/golden/join29-0-e78402f7585b17c76c32790571d749eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join29-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join29-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join29-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join29-4-b0524de7985cdb7b57fd995721654856 b/sql/hive/src/test/resources/golden/join29-4-b0524de7985cdb7b57fd995721654856 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 b/sql/hive/src/test/resources/golden/join29-5-9e35870a3bc8224cde6b1ab322568d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join29-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..f553ce0ca41f3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/join3-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 b/sql/hive/src/test/resources/golden/join3-1-4ef57a9e40b2e7fd01e1778bed71b5b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 b/sql/hive/src/test/resources/golden/join3-2-e9f6d17b15064f953a588fb40aee2f90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join3-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..8886dc7e8f229 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 b/sql/hive/src/test/resources/golden/join30-0-54a7280ab9eed0d2e3b33df35a721b66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d b/sql/hive/src/test/resources/golden/join30-1-5859eba43fb180c2142035cd4c77ea4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join30-2-194edec0592cf441617ca7caf9756baa b/sql/hive/src/test/resources/golden/join30-2-194edec0592cf441617ca7caf9756baa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join30-3-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..71094ee7360db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join31-0-619db2f9dd69aa236aa804cced726c9a b/sql/hive/src/test/resources/golden/join31-0-619db2f9dd69aa236aa804cced726c9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join31-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join31-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join31-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 b/sql/hive/src/test/resources/golden/join31-4-34e30fa5d6f35a20561c7754197f0626 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 b/sql/hive/src/test/resources/golden/join31-5-a2082be28be44e2bfa9a58fb45c23f07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join31-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..5d28208ab255c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join32-0-775b1af8eae9b2250052be1b72b4086 b/sql/hive/src/test/resources/golden/join32-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join32-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join32-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join32-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb b/sql/hive/src/test/resources/golden/join32-4-fc6880e4e0750a3daa7fc108b72d11bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32-5-425c5c35b23c42254dabec49aa6613bf b/sql/hive/src/test/resources/golden/join32-5-425c5c35b23c42254dabec49aa6613bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join32-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 b/sql/hive/src/test/resources/golden/join32_lessSize-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 b/sql/hive/src/test/resources/golden/join32_lessSize-1-7a874b25490d3e1e186803646eb6a789 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/sql/hive/src/test/resources/golden/join32_lessSize-10-e4ca54ef5e9c845b5bd7fb6b56cdc45a new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca b/sql/hive/src/test/resources/golden/join32_lessSize-11-79c0e9faf5798c465fedee25c72e7ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 b/sql/hive/src/test/resources/golden/join32_lessSize-12-83544f47197cccad30dd4f0ede433b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-13-ed70124968560328930327ecb108c4e9 new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a b/sql/hive/src/test/resources/golden/join32_lessSize-14-ac014eb214f762f1a61125af1964788a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e b/sql/hive/src/test/resources/golden/join32_lessSize-15-78edd67daa31711374b4c25faf89f77e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-16-ed70124968560328930327ecb108c4e9 new file mode 100644 index 0000000000000..b1251794645ce --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 b/sql/hive/src/test/resources/golden/join32_lessSize-17-b6bd9e2ca2bfabec16846b4a1fa3ea20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d b/sql/hive/src/test/resources/golden/join32_lessSize-18-f9b31c552044d8e5cc193121eb71024d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-19-ed70124968560328930327ecb108c4e9 new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join32_lessSize-2-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 b/sql/hive/src/test/resources/golden/join32_lessSize-20-ecf04f68517b6104d9cc8995796b424 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 b/sql/hive/src/test/resources/golden/join32_lessSize-21-f6822e1bb0b193eec397ebb9c79fbf47 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 b/sql/hive/src/test/resources/golden/join32_lessSize-22-ed70124968560328930327ecb108c4e9 new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join32_lessSize-3-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 b/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join32_lessSize-4-7f7f3daa7ff45944c3d534f9feff3bb6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb b/sql/hive/src/test/resources/golden/join32_lessSize-5-fc6880e4e0750a3daa7fc108b72d11bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf b/sql/hive/src/test/resources/golden/join32_lessSize-6-425c5c35b23c42254dabec49aa6613bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a b/sql/hive/src/test/resources/golden/join32_lessSize-7-e4ca54ef5e9c845b5bd7fb6b56cdc45a new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b b/sql/hive/src/test/resources/golden/join32_lessSize-8-5e277b322f831494ecd6f51a0c727c7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 b/sql/hive/src/test/resources/golden/join32_lessSize-9-e22030c2383aa7f16b7cf89bebb1e85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join33-0-775b1af8eae9b2250052be1b72b4086 b/sql/hive/src/test/resources/golden/join33-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join33-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join33-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join33-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb b/sql/hive/src/test/resources/golden/join33-4-fc6880e4e0750a3daa7fc108b72d11bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join33-5-425c5c35b23c42254dabec49aa6613bf b/sql/hive/src/test/resources/golden/join33-5-425c5c35b23c42254dabec49aa6613bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join33-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..a6538b605a817 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join34-0-775b1af8eae9b2250052be1b72b4086 b/sql/hive/src/test/resources/golden/join34-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join34-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join34-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join34-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join34-4-51cc5d253befd784a9d466c0b402f47a b/sql/hive/src/test/resources/golden/join34-4-51cc5d253befd784a9d466c0b402f47a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 b/sql/hive/src/test/resources/golden/join34-5-abf5cf110d6a0eb85ae257768c805f39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join34-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..e18d1ff802c93 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 b/sql/hive/src/test/resources/golden/join35-0-1ef68e1b8dd5f69f4a5149f3ebec42b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join35-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join35-2-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join35-3-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join35-4-99cb704ccc2813400908a62571891dc7 b/sql/hive/src/test/resources/golden/join35-4-99cb704ccc2813400908a62571891dc7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join35-5-60956f3307651237f52ffbe41c827c1c b/sql/hive/src/test/resources/golden/join35-5-60956f3307651237f52ffbe41c827c1c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join35-6-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..efdd80c9f8b89 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join36-0-8934d9b821aa4b34b760f73eff56cd06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join36-1-3aba153d62042206c0f001ea74980da b/sql/hive/src/test/resources/golden/join36-1-3aba153d62042206c0f001ea74980da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 b/sql/hive/src/test/resources/golden/join36-2-5b1cac8ef7857d2f8c675ed6f0224471 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 b/sql/hive/src/test/resources/golden/join36-3-c87d5b10b6ce87ac8ee2ab0d2b3755e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 b/sql/hive/src/test/resources/golden/join36-4-fbe7d714e988877ebe6202a60e8ef776 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 b/sql/hive/src/test/resources/golden/join36-5-9353d3e0b11c781f782af4013e8569e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 b/sql/hive/src/test/resources/golden/join36-6-d76aff886201d223c7dbed2b0f0d5512 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join36-7-171ede21302050a33468d95e329a38a3 b/sql/hive/src/test/resources/golden/join36-7-171ede21302050a33468d95e329a38a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join36-8-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..a250f202c4df0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 b/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join37-0-8934d9b821aa4b34b760f73eff56cd06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join37-1-360b3676605ccb3d89cf555342db87af b/sql/hive/src/test/resources/golden/join37-1-360b3676605ccb3d89cf555342db87af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join37-2-2310f2b8726137811543e04db98dd1db b/sql/hive/src/test/resources/golden/join37-2-2310f2b8726137811543e04db98dd1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join37-3-c4b3a85965108ad7013ac3931598af09 b/sql/hive/src/test/resources/golden/join37-3-c4b3a85965108ad7013ac3931598af09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join37-4-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..183353e5c705b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join38-0-42e6ad674a5142d05258a2d29578e046 b/sql/hive/src/test/resources/golden/join38-0-42e6ad674a5142d05258a2d29578e046 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f b/sql/hive/src/test/resources/golden/join38-1-d8ba5ec8a5a0bb702958400a3b3e458f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 b/sql/hive/src/test/resources/golden/join38-2-eacdb3417c4a563982c488812d654c9 new file mode 100644 index 0000000000000..5e020e3aff653 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 b/sql/hive/src/test/resources/golden/join38-3-22ead86c36dcd9ac9403fb52ac752046 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef b/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef new file mode 100644 index 0000000000000..fce97c9211916 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join38-4-53d219706847e890de1dcd369563ebef @@ -0,0 +1 @@ +val_111 105 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join39-0-54e9f7924abb40d1c601de908de6f25b b/sql/hive/src/test/resources/golden/join39-0-54e9f7924abb40d1c601de908de6f25b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 b/sql/hive/src/test/resources/golden/join39-1-60178dec03adcbccbd4e5cae936c38f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 b/sql/hive/src/test/resources/golden/join39-2-7fe3308e0a153b36c8eb21edcd3dbe96 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join39-3-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..5b45955ceb274 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join4-0-531e91e11b3891627c2675935fda14cd b/sql/hive/src/test/resources/golden/join4-0-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 b/sql/hive/src/test/resources/golden/join4-1-8884a9b112b0c767aa0bd8e745bd7b21 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join4-2-dc967001beb776f3a859e9360823c361 b/sql/hive/src/test/resources/golden/join4-2-dc967001beb776f3a859e9360823c361 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join4-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..e45f3234d5a88 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac b/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join40-0-d2b5e23edec42a62e61750b110ecbaac @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 b/sql/hive/src/test/resources/golden/join40-1-4b663d50fee0a8afd32cdf25c3b82de8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa b/sql/hive/src/test/resources/golden/join40-10-1d1f79e8e773d630f355c1a89d84b5aa new file mode 100644 index 0000000000000..8543fe173f87f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 b/sql/hive/src/test/resources/golden/join40-11-c8c5b97a744acf897888ab2d51d33a80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda b/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda new file mode 100644 index 0000000000000..67d892c80f493 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join40-12-aaddbef9069aba3ebeb310be74671cda @@ -0,0 +1 @@ +1028 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 b/sql/hive/src/test/resources/golden/join40-2-507b1d9f6abbdb756a589d7bc4826251 new file mode 100644 index 0000000000000..8543fe173f87f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 b/sql/hive/src/test/resources/golden/join40-3-4ef939dcda3b87f66c1bb3b2ebe32ec6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 b/sql/hive/src/test/resources/golden/join40-4-61fb097bda1751910de966d6a4a8f0b7 new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e b/sql/hive/src/test/resources/golden/join40-5-16cda49c2b2cd0c0cdae40af4d5c900e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 b/sql/hive/src/test/resources/golden/join40-6-9685f2d31ffe922d3ea217de32ca3049 new file mode 100644 index 0000000000000..bd6e20b8de1e4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join40-7-222655137ca9242f063c7717acbe4c65 b/sql/hive/src/test/resources/golden/join40-7-222655137ca9242f063c7717acbe4c65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 b/sql/hive/src/test/resources/golden/join40-8-3331a020843caccf2fa32a1225c5c3a9 new file mode 100644 index 0000000000000..bd6e20b8de1e4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join40-9-6b7404af014eeb4de196577f14ef18cc b/sql/hive/src/test/resources/golden/join40-9-6b7404af014eeb4de196577f14ef18cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 b/sql/hive/src/test/resources/golden/join41-0-b74d94933c6670ccc4a1ad8161185686 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 b/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join41-1-25e434b6d05e08fdd5f4d9957438917 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a b/sql/hive/src/test/resources/golden/join41-2-e02c931c8bb07ad765d0a6cd73abc2a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 b/sql/hive/src/test/resources/golden/join41-3-1dc98363e7da167dc45711a87ef3a988 new file mode 100644 index 0000000000000..7265626e5dbde --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join41-4-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e b/sql/hive/src/test/resources/golden/join41-5-f18ea17bcfafde311a4bdc589362e00e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 b/sql/hive/src/test/resources/golden/join41-6-1dc98363e7da167dc45711a87ef3a988 new file mode 100644 index 0000000000000..7265626e5dbde --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join5-0-531e91e11b3891627c2675935fda14cd b/sql/hive/src/test/resources/golden/join5-0-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join5-1-76feff1282895f38b673f52acfd7429e b/sql/hive/src/test/resources/golden/join5-1-76feff1282895f38b673f52acfd7429e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 b/sql/hive/src/test/resources/golden/join5-2-b07bb1fdcd0eeeb62a8f7acb70cd2330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join5-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..06b2b4d7e6d47 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join6-0-531e91e11b3891627c2675935fda14cd b/sql/hive/src/test/resources/golden/join6-0-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 b/sql/hive/src/test/resources/golden/join6-1-c3702956d2a1ff2bf1ae967b9f04bad3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 b/sql/hive/src/test/resources/golden/join6-2-bc6b6640b266ebe9b73702d3baf09c20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join6-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..36ca4a0805f4b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 b/sql/hive/src/test/resources/golden/join7-0-8f82881057bec4abf5a4d770a6f35838 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 b/sql/hive/src/test/resources/golden/join7-1-1aa2f2b13c2522b0457c0dd2ceb615d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 b/sql/hive/src/test/resources/golden/join7-2-56a30a1aa948bcf5ee54481897fc2208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join7-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..eb0a6246c1988 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join8-0-531e91e11b3891627c2675935fda14cd b/sql/hive/src/test/resources/golden/join8-0-531e91e11b3891627c2675935fda14cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 b/sql/hive/src/test/resources/golden/join8-1-13f11eaafaa42f8bdf0ed0a446f5bfa7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join8-2-f1196bca86a749375da35f134206a8ca b/sql/hive/src/test/resources/golden/join8-2-f1196bca86a749375da35f134206a8ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join8-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..f2e6e07d8fa1f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/join9-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a b/sql/hive/src/test/resources/golden/join9-1-fb3387ec28fc9d527e773c8b70bdf67a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 b/sql/hive/src/test/resources/golden/join9-2-971c44e81ce17eb0849850b72ebd20f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/join9-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 b/sql/hive/src/test/resources/golden/join_1to1-0-7ea7cf54372f262b952108113a97a294 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a b/sql/hive/src/test/resources/golden/join_1to1-1-5bab379018a4fbef12cc93658f26580a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e b/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_1to1-10-a1caf8c546f519e2dfb5e17c8addf62e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-11-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-12-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-13-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-14-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-15-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_1to1-16-9914f44ecb6ae7587b62e5349ff60d04 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-17-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-18-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-19-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 b/sql/hive/src/test/resources/golden/join_1to1-2-1c35d445eca04025196ea700d02f9987 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-20-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-21-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 b/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_1to1-22-5a065a27a36bb0ff980fa7ffef3a2600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-23-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-24-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-25-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-26-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-27-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e b/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_1to1-28-a1caf8c546f519e2dfb5e17c8addf62e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-29-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 b/sql/hive/src/test/resources/golden/join_1to1-3-ee6db4188755bf471a12316ec7301500 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-30-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-31-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-32-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-33-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 b/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_1to1-34-9914f44ecb6ae7587b62e5349ff60d04 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-35-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-36-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-37-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-38-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-39-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 b/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_1to1-4-5a065a27a36bb0ff980fa7ffef3a2600 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e b/sql/hive/src/test/resources/golden/join_1to1-5-ce1542ccec99ccfdc9b5c3f713ab1c0e new file mode 100644 index 0000000000000..52a4d2c18e701 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b b/sql/hive/src/test/resources/golden/join_1to1-6-6081eb976b4aef2132418510756a385b new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c b/sql/hive/src/test/resources/golden/join_1to1-7-281b888188eac90c4bf670417f25cc0c new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 b/sql/hive/src/test/resources/golden/join_1to1-8-2bc944d7dcc8eba8f25794d99ea35b84 new file mode 100644 index 0000000000000..fb5560e2d006c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 b/sql/hive/src/test/resources/golden/join_1to1-9-5e48ba086f1376939535081b60f82727 new file mode 100644 index 0000000000000..0ae4eb2d33595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 b/sql/hive/src/test/resources/golden/join_array-0-60904bf2bd7bbfa8739d8e047e83e038 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_array-1-2c086fcf118fd2538bfa00724209091e b/sql/hive/src/test/resources/golden/join_array-1-2c086fcf118fd2538bfa00724209091e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de b/sql/hive/src/test/resources/golden/join_array-2-a4363f7c7e4b7d717ed90e77c37581de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 b/sql/hive/src/test/resources/golden/join_array-3-ddd65703cdad8959cd0cd831304b0ab9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 b/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 new file mode 100644 index 0000000000000..ce9688a065b34 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_array-4-b235265cd6bd58fd743c27b02e547d62 @@ -0,0 +1 @@ +10320092026892491 3312 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a b/sql/hive/src/test/resources/golden/join_array-5-a9b9419b94631f8fe1c2297ebf103a9a new file mode 100644 index 0000000000000..f937af554adef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c b/sql/hive/src/test/resources/golden/join_array-6-6eded94bd39189ea6d67fe383f9b865c new file mode 100644 index 0000000000000..f937af554adef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 b/sql/hive/src/test/resources/golden/join_casesensitive-0-3c6c18f40c89c8fe9891869f2f6acb48 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 b/sql/hive/src/test/resources/golden/join_casesensitive-1-404d691e85c7b74bad73576ee80de290 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d b/sql/hive/src/test/resources/golden/join_casesensitive-2-d508d777b4e4156bff9774925a6ca4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc b/sql/hive/src/test/resources/golden/join_casesensitive-3-3c88c1f52d27e9fb093aaf10f97c7cfc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd b/sql/hive/src/test/resources/golden/join_casesensitive-4-c880b2256f97413b8fe68d19d99747fd new file mode 100644 index 0000000000000..810f01f682c37 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad b/sql/hive/src/test/resources/golden/join_empty-0-3fef0a906d6618ab28406d1edf0dc9ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 b/sql/hive/src/test/resources/golden/join_empty-1-9bcf09f835d785760c0d4da0680837f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e b/sql/hive/src/test/resources/golden/join_empty-2-ff98d5f09dd67ae5f1bd8d09a3c4db8e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_empty-3-3b2231f25886c152f222856256ac3473 b/sql/hive/src/test/resources/golden/join_empty-3-3b2231f25886c152f222856256ac3473 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a b/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_empty-4-df2401785dfa257de49c3ad80b0f480a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_empty-5-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a b/sql/hive/src/test/resources/golden/join_empty-6-e807e73f2636bf03a572c9e0b8430b5a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 b/sql/hive/src/test/resources/golden/join_empty-7-7d6e4e5bc4cae56192cc737fb6dc9c72 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 b/sql/hive/src/test/resources/golden/join_filters-0-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 b/sql/hive/src/test/resources/golden/join_filters-1-100faa7fd01bfb4390c782bb262a9403 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 b/sql/hive/src/test/resources/golden/join_filters-10-f0c5c51de4151a17a77c780be0c13e01 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe b/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-100-a4c7cd87175222bea19fd33018890efe @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b b/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-101-f086409eb336282af5a00f4c7192ef2b @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 b/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-102-634888c4fa78ce10181c9514c6393554 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b b/sql/hive/src/test/resources/golden/join_filters-103-c020d3461658ae8e118281f40264ae5b new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 b/sql/hive/src/test/resources/golden/join_filters-104-c9b79f30e1f25672ec89014f966b41b0 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb b/sql/hive/src/test/resources/golden/join_filters-105-b3d9dcbb7e1caef97646eb89edf82eb new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 b/sql/hive/src/test/resources/golden/join_filters-106-870999978978b22f21997899f1e652b8 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 b/sql/hive/src/test/resources/golden/join_filters-107-94824a62d882227f270a66ec7ef28cd4 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e b/sql/hive/src/test/resources/golden/join_filters-108-d793c1514545499f58fb1b355cbd3f0e new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 b/sql/hive/src/test/resources/golden/join_filters-109-2709001b4aa57ed01ba975e83b556475 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 b/sql/hive/src/test/resources/golden/join_filters-11-6fb35e1a65211e68de5df4a1cb9c7117 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba b/sql/hive/src/test/resources/golden/join_filters-110-86868ef0f28c5b42f04fb9ca64aaa3ba new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 b/sql/hive/src/test/resources/golden/join_filters-111-fd3188d025e4c84d45cbb265d08ca569 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 b/sql/hive/src/test/resources/golden/join_filters-112-a0cd07949ff7dbc2287dc060bedb1942 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba b/sql/hive/src/test/resources/golden/join_filters-113-48c4978768872751832149d72cdf0ba new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 b/sql/hive/src/test/resources/golden/join_filters-114-58b8db46ea5080791c7416838d3e8f95 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 b/sql/hive/src/test/resources/golden/join_filters-115-98e45a3167d19d09219076a2d93afa49 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 b/sql/hive/src/test/resources/golden/join_filters-116-bec2950fff2c1b3c4dc5be846653ea6 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 b/sql/hive/src/test/resources/golden/join_filters-117-ee8471d93c3778f4b48c5c59f473dd35 new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e b/sql/hive/src/test/resources/golden/join_filters-118-9b395d7db8722a467d46588d0f27fc9e new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a b/sql/hive/src/test/resources/golden/join_filters-119-7688784396db55ff148292a9755c918a new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e b/sql/hive/src/test/resources/golden/join_filters-12-4c29d6be8717481332cd1ee7ca17690e new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 b/sql/hive/src/test/resources/golden/join_filters-120-6578651ffbf95c0b02825e2125e32709 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada b/sql/hive/src/test/resources/golden/join_filters-121-6dc4977da7f8dc7c636a03d7ad892ada new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 b/sql/hive/src/test/resources/golden/join_filters-122-64bb9f9a0eef32d2392b80b93c9b2c98 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 b/sql/hive/src/test/resources/golden/join_filters-13-c6a291879bdb37f0c84f6074f257d52 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf b/sql/hive/src/test/resources/golden/join_filters-14-ef8255dcad808f9128d79e6ee9f368cf new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 b/sql/hive/src/test/resources/golden/join_filters-15-a83678913c62249c8fdf2dac1f6e3046 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 b/sql/hive/src/test/resources/golden/join_filters-16-a6511579b5e4c2d845664f0c4a6ce622 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 b/sql/hive/src/test/resources/golden/join_filters-17-c22aab929016fa6f4b6d1e14cc3a4940 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 b/sql/hive/src/test/resources/golden/join_filters-18-8ef51af7455eaeeaefff1c98e16dcc65 new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f b/sql/hive/src/test/resources/golden/join_filters-19-e164192e6538b428c8b53e008a978d3f new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 b/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-2-67fff9e6931a7320444e857e01b3d496 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf b/sql/hive/src/test/resources/golden/join_filters-20-7a5da20822bf51ed69ccf640cbb816cf new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e b/sql/hive/src/test/resources/golden/join_filters-21-13d6d5335625fc3386a8011dc557002e new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e b/sql/hive/src/test/resources/golden/join_filters-22-f12ffd6cc9213d9c64cbea1b6db0632e new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f b/sql/hive/src/test/resources/golden/join_filters-23-a800b885358695758afdb719cdefa94f new file mode 100644 index 0000000000000..ea646edf6d143 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 b/sql/hive/src/test/resources/golden/join_filters-24-df3db5a91f3f4e88e18d2b1cc5b47113 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 b/sql/hive/src/test/resources/golden/join_filters-25-435ecfc7f9cb5f32019fe498c21ccad3 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca b/sql/hive/src/test/resources/golden/join_filters-26-da36fab3d1686aedd415a7a0f0253eca new file mode 100644 index 0000000000000..ea646edf6d143 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 b/sql/hive/src/test/resources/golden/join_filters-27-5f4a5437696f2a60bec9ac1443244242 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 b/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-28-2acf41a9f6efac0d800df557db716359 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 b/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-29-cf5fef4ddc313209f5ab1e5ea3763e35 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 b/sql/hive/src/test/resources/golden/join_filters-3-989b3dbd15ba601ae80fe454e03213d7 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf b/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-30-97f94f35ebc30f776a19bed59c7fb4bf @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f b/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-31-4923f8ba429f0c71ad9b6b9499e73a7f @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab b/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-32-5978cd7936c296493a16a31b926043ab @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 b/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-33-607d64d50ef9aad424bd22b358efe027 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 b/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-34-35c2c8d74bc0ebc4710c81333cb254a9 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe b/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-35-a4c7cd87175222bea19fd33018890efe @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b b/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-36-f086409eb336282af5a00f4c7192ef2b @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 b/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-37-634888c4fa78ce10181c9514c6393554 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b b/sql/hive/src/test/resources/golden/join_filters-38-c020d3461658ae8e118281f40264ae5b new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 b/sql/hive/src/test/resources/golden/join_filters-39-c9b79f30e1f25672ec89014f966b41b0 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 b/sql/hive/src/test/resources/golden/join_filters-4-33bfcd576019d7e32683556f66e3757 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb b/sql/hive/src/test/resources/golden/join_filters-40-b3d9dcbb7e1caef97646eb89edf82eb new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 b/sql/hive/src/test/resources/golden/join_filters-41-870999978978b22f21997899f1e652b8 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 b/sql/hive/src/test/resources/golden/join_filters-42-94824a62d882227f270a66ec7ef28cd4 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e b/sql/hive/src/test/resources/golden/join_filters-43-d793c1514545499f58fb1b355cbd3f0e new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e b/sql/hive/src/test/resources/golden/join_filters-44-6d8955591f62d9cfc6af17df63d3d88e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 b/sql/hive/src/test/resources/golden/join_filters-45-23ab7ac8229a53d391195be7ca092429 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/join_filters-46-aa161b0d9fe9d1aad10654fce0e3670b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/join_filters-47-3c52df82c7d78501610f3f898103f753 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/join_filters-48-1d85bb008e02ef4025171a4bc0866a6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/join_filters-49-e79c906b894fed049ddfab4496a4e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e b/sql/hive/src/test/resources/golden/join_filters-5-f0c0d07019afb1bbe162e3183e18023e new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 b/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-50-3e6612a89e9124592e790594775054b1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac b/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-51-60a5f56f33fc8854a2b687005f0d96ac @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-52-64cabe5164130a94f387288f37b62d71 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 b/sql/hive/src/test/resources/golden/join_filters-53-2709001b4aa57ed01ba975e83b556475 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba b/sql/hive/src/test/resources/golden/join_filters-54-86868ef0f28c5b42f04fb9ca64aaa3ba new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 b/sql/hive/src/test/resources/golden/join_filters-55-fd3188d025e4c84d45cbb265d08ca569 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 b/sql/hive/src/test/resources/golden/join_filters-56-a0cd07949ff7dbc2287dc060bedb1942 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba b/sql/hive/src/test/resources/golden/join_filters-57-48c4978768872751832149d72cdf0ba new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 b/sql/hive/src/test/resources/golden/join_filters-58-58b8db46ea5080791c7416838d3e8f95 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 b/sql/hive/src/test/resources/golden/join_filters-59-98e45a3167d19d09219076a2d93afa49 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 b/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-6-c0c40d001cac0bc91095dddda1513ad9 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 b/sql/hive/src/test/resources/golden/join_filters-60-bec2950fff2c1b3c4dc5be846653ea6 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 b/sql/hive/src/test/resources/golden/join_filters-61-ee8471d93c3778f4b48c5c59f473dd35 new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e b/sql/hive/src/test/resources/golden/join_filters-62-9b395d7db8722a467d46588d0f27fc9e new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a b/sql/hive/src/test/resources/golden/join_filters-63-7688784396db55ff148292a9755c918a new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 b/sql/hive/src/test/resources/golden/join_filters-64-6578651ffbf95c0b02825e2125e32709 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada b/sql/hive/src/test/resources/golden/join_filters-65-6dc4977da7f8dc7c636a03d7ad892ada new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 b/sql/hive/src/test/resources/golden/join_filters-66-64bb9f9a0eef32d2392b80b93c9b2c98 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 b/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-67-67fff9e6931a7320444e857e01b3d496 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 b/sql/hive/src/test/resources/golden/join_filters-68-989b3dbd15ba601ae80fe454e03213d7 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 b/sql/hive/src/test/resources/golden/join_filters-69-33bfcd576019d7e32683556f66e3757 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d b/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-7-89963646509154a2fb1ddbbf1f55349d @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e b/sql/hive/src/test/resources/golden/join_filters-70-f0c0d07019afb1bbe162e3183e18023e new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 b/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-71-c0c40d001cac0bc91095dddda1513ad9 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d b/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-72-89963646509154a2fb1ddbbf1f55349d @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 b/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-73-69e0235472d7cee7d83037cd083544a5 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa b/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-74-b6372cc006844e8488a3b7836c67daaa @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 b/sql/hive/src/test/resources/golden/join_filters-75-f0c5c51de4151a17a77c780be0c13e01 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 b/sql/hive/src/test/resources/golden/join_filters-76-6fb35e1a65211e68de5df4a1cb9c7117 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e b/sql/hive/src/test/resources/golden/join_filters-77-4c29d6be8717481332cd1ee7ca17690e new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 b/sql/hive/src/test/resources/golden/join_filters-78-c6a291879bdb37f0c84f6074f257d52 new file mode 100644 index 0000000000000..51a29d71030ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf b/sql/hive/src/test/resources/golden/join_filters-79-ef8255dcad808f9128d79e6ee9f368cf new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 b/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-8-69e0235472d7cee7d83037cd083544a5 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 b/sql/hive/src/test/resources/golden/join_filters-80-a83678913c62249c8fdf2dac1f6e3046 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 b/sql/hive/src/test/resources/golden/join_filters-81-a6511579b5e4c2d845664f0c4a6ce622 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 b/sql/hive/src/test/resources/golden/join_filters-82-c22aab929016fa6f4b6d1e14cc3a4940 new file mode 100644 index 0000000000000..759113b3f1c96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 b/sql/hive/src/test/resources/golden/join_filters-83-8ef51af7455eaeeaefff1c98e16dcc65 new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f b/sql/hive/src/test/resources/golden/join_filters-84-e164192e6538b428c8b53e008a978d3f new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf b/sql/hive/src/test/resources/golden/join_filters-85-7a5da20822bf51ed69ccf640cbb816cf new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e b/sql/hive/src/test/resources/golden/join_filters-86-13d6d5335625fc3386a8011dc557002e new file mode 100644 index 0000000000000..24a0c79d42923 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e b/sql/hive/src/test/resources/golden/join_filters-87-f12ffd6cc9213d9c64cbea1b6db0632e new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f b/sql/hive/src/test/resources/golden/join_filters-88-a800b885358695758afdb719cdefa94f new file mode 100644 index 0000000000000..ea646edf6d143 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 b/sql/hive/src/test/resources/golden/join_filters-89-df3db5a91f3f4e88e18d2b1cc5b47113 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa b/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-9-b6372cc006844e8488a3b7836c67daaa @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 b/sql/hive/src/test/resources/golden/join_filters-90-435ecfc7f9cb5f32019fe498c21ccad3 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca b/sql/hive/src/test/resources/golden/join_filters-91-da36fab3d1686aedd415a7a0f0253eca new file mode 100644 index 0000000000000..ea646edf6d143 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 b/sql/hive/src/test/resources/golden/join_filters-92-5f4a5437696f2a60bec9ac1443244242 new file mode 100644 index 0000000000000..d735f349a9542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 b/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-93-2acf41a9f6efac0d800df557db716359 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 b/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-94-cf5fef4ddc313209f5ab1e5ea3763e35 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf b/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-95-97f94f35ebc30f776a19bed59c7fb4bf @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f b/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-96-4923f8ba429f0c71ad9b6b9499e73a7f @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab b/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-97-5978cd7936c296493a16a31b926043ab @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 b/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-98-607d64d50ef9aad424bd22b358efe027 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 b/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_filters-99-35c2c8d74bc0ebc4710c81333cb254a9 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce b/sql/hive/src/test/resources/golden/join_filters_overlap-0-990e447b6447ced0d9684eb7db9e63ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 b/sql/hive/src/test/resources/golden/join_filters_overlap-1-a7336cd2d643f92715b42f9c6eb2c108 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 b/sql/hive/src/test/resources/golden/join_hive_626-0-3491c764e44ee92af3a88d301a1bd498 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 b/sql/hive/src/test/resources/golden/join_hive_626-1-2b67cd3c7cec844a7ceb19e8dbb0e164 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e b/sql/hive/src/test/resources/golden/join_hive_626-2-42997e626819de9cdad544d7fe132c6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 b/sql/hive/src/test/resources/golden/join_hive_626-3-2609f2809e0c6ae1ca93806e37960990 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 b/sql/hive/src/test/resources/golden/join_hive_626-4-387e3bc517b49d4e1c9752c07b72b790 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 b/sql/hive/src/test/resources/golden/join_hive_626-5-a0eb25c15b6ca1a04da14781b1213e11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 b/sql/hive/src/test/resources/golden/join_hive_626-6-392f6f482886b2e7be5903d44bfba3f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 b/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 new file mode 100644 index 0000000000000..e3143e49c2bad --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_hive_626-7-d9c817eebc533bedc3ef9172d325a2c2 @@ -0,0 +1 @@ +foo1 bar10 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 b/sql/hive/src/test/resources/golden/join_map_ppr-0-775b1af8eae9b2250052be1b72b4086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 b/sql/hive/src/test/resources/golden/join_map_ppr-1-7b9086c721214279a2f0c64d6d35df77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join_map_ppr-10-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..4c88927e56579 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e b/sql/hive/src/test/resources/golden/join_map_ppr-2-e3e5db334ed33b17026b43f776daee1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa b/sql/hive/src/test/resources/golden/join_map_ppr-3-c9859bf9c9942c59f3b19d82bd1a3afa new file mode 100644 index 0000000000000..4c88927e56579 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 b/sql/hive/src/test/resources/golden/join_map_ppr-4-1230f694ae20d2a21e41ac609471b8c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 b/sql/hive/src/test/resources/golden/join_map_ppr-5-b7e6358104eceede7389c7d2a212b058 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a b/sql/hive/src/test/resources/golden/join_map_ppr-6-6aab4a0b43b769d7f6b6f5ad6dc0558a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db b/sql/hive/src/test/resources/golden/join_map_ppr-7-45704df71b70e3f5aa31966ac96b9db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed b/sql/hive/src/test/resources/golden/join_map_ppr-8-e790d28a6c0818b1bafbe252879bd8ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a b/sql/hive/src/test/resources/golden/join_map_ppr-9-921ba3c3c23751ac1d724319a1b9b22a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 b/sql/hive/src/test/resources/golden/join_nulls-0-5644ab44e5ba9f2941216b8d5dc33a99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 b/sql/hive/src/test/resources/golden/join_nulls-1-97641998eb9ddf2dff56de6758486aa0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b b/sql/hive/src/test/resources/golden/join_nulls-10-39071d8e6b246cfd405714dbf0b5337b new file mode 100644 index 0000000000000..4bc2d3969f17f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 b/sql/hive/src/test/resources/golden/join_nulls-11-545dff878ea0d79cdfee517572c9e0c8 new file mode 100644 index 0000000000000..8414cfbede40a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 b/sql/hive/src/test/resources/golden/join_nulls-12-5e1ca8627aa685435142d8e339e77062 new file mode 100644 index 0000000000000..c11ae6cce7abb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d b/sql/hive/src/test/resources/golden/join_nulls-13-5f670a20cbec42a34685ee0398ad7e2d new file mode 100644 index 0000000000000..e28e1b1e5e8b1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 b/sql/hive/src/test/resources/golden/join_nulls-14-e97ba69145da387a4a66635b8499077 new file mode 100644 index 0000000000000..85192b86801e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 b/sql/hive/src/test/resources/golden/join_nulls-15-e9f9be8be6e936fb832483dfa0be5126 new file mode 100644 index 0000000000000..c19237d5d4309 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 b/sql/hive/src/test/resources/golden/join_nulls-16-b000ccd6c2c05af1875e21428aa3d9b9 new file mode 100644 index 0000000000000..e28e1b1e5e8b1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f b/sql/hive/src/test/resources/golden/join_nulls-17-2b0bfe5754456475ceb6af4694165f new file mode 100644 index 0000000000000..56ca29ffe263c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 b/sql/hive/src/test/resources/golden/join_nulls-18-321cf9d31dac835c3def6ca3b3b860a2 new file mode 100644 index 0000000000000..92e228d381628 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c b/sql/hive/src/test/resources/golden/join_nulls-19-739bf8e440e698540d18c29226c3564c new file mode 100644 index 0000000000000..f57bbd5152852 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 b/sql/hive/src/test/resources/golden/join_nulls-2-f4b71841cfff5294bc1e5ac163eadbe5 new file mode 100644 index 0000000000000..0d14bf4e38509 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 b/sql/hive/src/test/resources/golden/join_nulls-20-fff960f1cb832bc93e3d1de519e573d5 new file mode 100644 index 0000000000000..56ca29ffe263c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e b/sql/hive/src/test/resources/golden/join_nulls-21-62a25fb819ae5c1ea757b6e759082a2e new file mode 100644 index 0000000000000..0db9ffd61a3ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 b/sql/hive/src/test/resources/golden/join_nulls-22-5b2df5518994ae86c041484561857da0 new file mode 100644 index 0000000000000..ed1b35fbe80ae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 b/sql/hive/src/test/resources/golden/join_nulls-23-982c3e55235cafa3d89b5dee4366fdf8 new file mode 100644 index 0000000000000..0db9ffd61a3ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 b/sql/hive/src/test/resources/golden/join_nulls-24-86ad66543a735d396f7336cb5bdfa495 new file mode 100644 index 0000000000000..0d14bf4e38509 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 b/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nulls-25-45bd90c71bc44c2accf0009f1ceb5f56 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 b/sql/hive/src/test/resources/golden/join_nulls-26-3fb736c1d060bbf9ec7ce58be4531dc9 new file mode 100644 index 0000000000000..bdd09da460bbe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 b/sql/hive/src/test/resources/golden/join_nulls-27-6ee7affed896b1c539628ab081842b83 new file mode 100644 index 0000000000000..dfe4f4318fc66 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb b/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nulls-28-455aace3472c5840a885b6fab6a046cb @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb b/sql/hive/src/test/resources/golden/join_nulls-29-2c278a88713aef5cd30ff3720ef3eeeb new file mode 100644 index 0000000000000..bdd09da460bbe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 b/sql/hive/src/test/resources/golden/join_nulls-3-88d2da3bcb2c8b04df1b6f18ff2226c4 new file mode 100644 index 0000000000000..0d14bf4e38509 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 b/sql/hive/src/test/resources/golden/join_nulls-30-2c0c41da38714d1b16feffb00fa08bb1 new file mode 100644 index 0000000000000..dfe4f4318fc66 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 b/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nulls-31-a33c48d38817ee3a7aca511dc7793486 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 b/sql/hive/src/test/resources/golden/join_nulls-32-e6b104ae96622ff75d510efc6efc9352 new file mode 100644 index 0000000000000..c11ae6cce7abb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 b/sql/hive/src/test/resources/golden/join_nulls-33-1284a11bf6aeef8ff87b471d41985f26 new file mode 100644 index 0000000000000..8414cfbede40a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 b/sql/hive/src/test/resources/golden/join_nulls-34-aeb90811861431cadc5512637793afc1 new file mode 100644 index 0000000000000..4bc2d3969f17f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 b/sql/hive/src/test/resources/golden/join_nulls-35-2d1d18d4e9775ec69333947fbd62bc82 new file mode 100644 index 0000000000000..e28e1b1e5e8b1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 b/sql/hive/src/test/resources/golden/join_nulls-36-7c029c91141b36af79ba0dc1de73a257 new file mode 100644 index 0000000000000..85192b86801e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc b/sql/hive/src/test/resources/golden/join_nulls-37-fa84731f5a6beec20d64a7981815b9bc new file mode 100644 index 0000000000000..c19237d5d4309 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e b/sql/hive/src/test/resources/golden/join_nulls-38-6d8955591f62d9cfc6af17df63d3d88e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 b/sql/hive/src/test/resources/golden/join_nulls-39-23ab7ac8229a53d391195be7ca092429 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 b/sql/hive/src/test/resources/golden/join_nulls-4-a1f20b4863428627bae1316755cc2d94 new file mode 100644 index 0000000000000..0d14bf4e38509 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b b/sql/hive/src/test/resources/golden/join_nulls-40-aa161b0d9fe9d1aad10654fce0e3670b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 b/sql/hive/src/test/resources/golden/join_nulls-41-3c52df82c7d78501610f3f898103f753 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c b/sql/hive/src/test/resources/golden/join_nulls-42-1d85bb008e02ef4025171a4bc0866a6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 b/sql/hive/src/test/resources/golden/join_nulls-43-e79c906b894fed049ddfab4496a4e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 b/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nulls-44-2db30531137611e06fdba478ca7a8412 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 b/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nulls-45-e58b2754e8d9c56a473557a549d0d2b9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 b/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nulls-46-64cabe5164130a94f387288f37b62d71 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 b/sql/hive/src/test/resources/golden/join_nulls-47-ebf794e8b51be738e2d664f249869de1 new file mode 100644 index 0000000000000..7d9efd15fa287 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 b/sql/hive/src/test/resources/golden/join_nulls-48-979c3ee0ee49bfd246d5372b8873fd3 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 b/sql/hive/src/test/resources/golden/join_nulls-49-2d20d8f4221281a6b1cd579e761f81b7 new file mode 100644 index 0000000000000..4d2ff6a237416 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c b/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nulls-5-5ec6e3df7181e0738244623cc01bf22c @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 b/sql/hive/src/test/resources/golden/join_nulls-50-8b26343e74c161cf74ece5f0bd21470 new file mode 100644 index 0000000000000..7d9efd15fa287 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f b/sql/hive/src/test/resources/golden/join_nulls-51-75339d2eb2afabf5dd088074b2563d8f new file mode 100644 index 0000000000000..695bb1544ec89 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 b/sql/hive/src/test/resources/golden/join_nulls-52-caad1db99085760daaf8f96c0ce5564 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 b/sql/hive/src/test/resources/golden/join_nulls-53-ec965e66e7fad403cd0ea388c3e70363 new file mode 100644 index 0000000000000..b89f5e6d42a0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 b/sql/hive/src/test/resources/golden/join_nulls-54-6f15c794b5587430ebb685cc61c502 new file mode 100644 index 0000000000000..b81820f886894 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 b/sql/hive/src/test/resources/golden/join_nulls-55-a1c73326f8c8d9beccda3ba365352564 new file mode 100644 index 0000000000000..71cd5ce0041b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 b/sql/hive/src/test/resources/golden/join_nulls-56-f7b9629093c818b6c384c79d1458d178 new file mode 100644 index 0000000000000..86b3dab6bffc8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 b/sql/hive/src/test/resources/golden/join_nulls-57-cf353446d7f358a508f17d0984b90158 new file mode 100644 index 0000000000000..169df31887b96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a b/sql/hive/src/test/resources/golden/join_nulls-58-5f9a59160a76f9b649102a97987ed33a new file mode 100644 index 0000000000000..86b3dab6bffc8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 b/sql/hive/src/test/resources/golden/join_nulls-59-8753a39412ac59c7a05951aeeea73b24 new file mode 100644 index 0000000000000..bdd5dcf63fd00 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 b/sql/hive/src/test/resources/golden/join_nulls-6-7eea211c80e7f1146098e80ffb890d67 new file mode 100644 index 0000000000000..bdd09da460bbe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 b/sql/hive/src/test/resources/golden/join_nulls-7-1692e12155c37a8d94d63d274a9eb8e0 new file mode 100644 index 0000000000000..dfe4f4318fc66 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 b/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_nulls-8-609f6bd812a44b20da0a39c827e4d870 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b b/sql/hive/src/test/resources/golden/join_nulls-9-ef4b27a877efc377cd5750af6725194b new file mode 100644 index 0000000000000..c11ae6cce7abb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 b/sql/hive/src/test/resources/golden/join_rc-0-fcbbe26c6881f81800fe86abbfe6aa87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e b/sql/hive/src/test/resources/golden/join_rc-1-6a7685f30de00ebb4867a4002d641a5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 b/sql/hive/src/test/resources/golden/join_rc-2-88aabbe1fcd7735ae8cc97418b4a59a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee b/sql/hive/src/test/resources/golden/join_rc-3-4a737f3e518f5322ba41a189e79c1dee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 b/sql/hive/src/test/resources/golden/join_rc-4-f60f7fdd08e85fae90af59475192b725 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 b/sql/hive/src/test/resources/golden/join_rc-5-1aef75afe38d512addb44dbf9a650263 new file mode 100644 index 0000000000000..59821aeea3008 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/join_reorder2-0-caf1c5fd299fdbdb655234d01d44caf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/join_reorder2-1-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d b/sql/hive/src/test/resources/golden/join_reorder2-10-45349471e0e919bd2185f584e87b891d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 b/sql/hive/src/test/resources/golden/join_reorder2-11-f8460b061fa00f6afbfe8eeccf4d3564 new file mode 100644 index 0000000000000..06a83e95d4f79 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/join_reorder2-2-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 b/sql/hive/src/test/resources/golden/join_reorder2-3-38a5e7f36d579835f9c2b1c52efd9643 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder2-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder2-5-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder2-6-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec b/sql/hive/src/test/resources/golden/join_reorder2-7-512b75ccb9459a6334da1d9699f4a5ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 b/sql/hive/src/test/resources/golden/join_reorder2-8-68b763a6f1f673daa35fee55aeae84a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 b/sql/hive/src/test/resources/golden/join_reorder2-9-26ffed826eceda953b7124ee39ace828 new file mode 100644 index 0000000000000..06a83e95d4f79 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/join_reorder3-0-caf1c5fd299fdbdb655234d01d44caf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/join_reorder3-1-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 b/sql/hive/src/test/resources/golden/join_reorder3-10-ff036400019164ed743ecd9cfc222ce1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 b/sql/hive/src/test/resources/golden/join_reorder3-11-d6392b851f7dd5e3705e8ff51e1c6583 new file mode 100644 index 0000000000000..06a83e95d4f79 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/join_reorder3-2-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 b/sql/hive/src/test/resources/golden/join_reorder3-3-38a5e7f36d579835f9c2b1c52efd9643 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder3-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder3-5-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder3-6-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec b/sql/hive/src/test/resources/golden/join_reorder3-7-512b75ccb9459a6334da1d9699f4a5ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 b/sql/hive/src/test/resources/golden/join_reorder3-8-743fefaadfe31ce43cd2dead85b7d741 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 b/sql/hive/src/test/resources/golden/join_reorder3-9-9d45e66a3990b7c53fd780f07cd52e13 new file mode 100644 index 0000000000000..06a83e95d4f79 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf b/sql/hive/src/test/resources/golden/join_reorder4-0-799a471861f35cb065a6b1105c7d9ccf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a b/sql/hive/src/test/resources/golden/join_reorder4-1-ee3326e9aa49562ac854bbd0b3f3d90a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f b/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f new file mode 100644 index 0000000000000..872146532307a --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_reorder4-10-1d6b948747ac31296710a491a5652e3f @@ -0,0 +1 @@ +2 12 2 22 2 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 b/sql/hive/src/test/resources/golden/join_reorder4-11-7ec0f5b60991a54d86ec9f8a27809c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 b/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 new file mode 100644 index 0000000000000..872146532307a --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_reorder4-12-203aed2c4298eba4a3c51816a21a1c1 @@ -0,0 +1 @@ +2 12 2 22 2 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c b/sql/hive/src/test/resources/golden/join_reorder4-2-8b4643dd260f09d32af5d79de7359c0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/join_reorder4-3-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/join_reorder4-4-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/join_reorder4-5-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_reorder4-6-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 b/sql/hive/src/test/resources/golden/join_reorder4-7-58a3ea3c82886b1eb7cee7403b1f96a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 b/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 new file mode 100644 index 0000000000000..872146532307a --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_reorder4-8-4bbb8e937245e2fa7cafbb423814754 @@ -0,0 +1 @@ +2 12 2 22 2 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba b/sql/hive/src/test/resources/golden/join_reorder4-9-b401eef67c1c04f875c084c9e0fda9ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-0-6001f83f7780835737e3262a34c2b832 b/sql/hive/src/test/resources/golden/join_star-0-6001f83f7780835737e3262a34c2b832 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d b/sql/hive/src/test/resources/golden/join_star-1-9f29849fe78f15b7594378a10fe3ed9d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e b/sql/hive/src/test/resources/golden/join_star-10-57ce75f989b3b3bfd2f2eceb228e892e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae b/sql/hive/src/test/resources/golden/join_star-11-eba1397e66f25cba4fd264209cc92bae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 b/sql/hive/src/test/resources/golden/join_star-12-89b53ae954ec88171ef87e0459f6eb82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb b/sql/hive/src/test/resources/golden/join_star-13-342b7249c9ce1484869169b1b33191cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 b/sql/hive/src/test/resources/golden/join_star-14-75513308d30b781fd2e06d81963c4363 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 b/sql/hive/src/test/resources/golden/join_star-15-24a0b1d0257bad4f85b3a122acf6bef9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_star-16-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_star-17-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 b/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_star-18-1c22e451845667bd6b4eac3c49c36965 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa b/sql/hive/src/test/resources/golden/join_star-19-a1209595ce68e24a111c2a0d27863bfa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 b/sql/hive/src/test/resources/golden/join_star-2-75d7e03808482c11361ce72f2dd38de0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 b/sql/hive/src/test/resources/golden/join_star-20-76473cb68a69b9408207fd43ddae9339 new file mode 100644 index 0000000000000..e60bca6269264 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 b/sql/hive/src/test/resources/golden/join_star-21-6b8dbb1dbcf0096f0af3209bb6efd2c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b b/sql/hive/src/test/resources/golden/join_star-22-429cfd140488242d569aee6594aae76b new file mode 100644 index 0000000000000..e08d04997c56b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_star-23-bada259628918dbfb4837bcb58258530 b/sql/hive/src/test/resources/golden/join_star-23-bada259628918dbfb4837bcb58258530 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 b/sql/hive/src/test/resources/golden/join_star-24-e11ab68d72d874c7c6c658c1018f5a49 new file mode 100644 index 0000000000000..b89911131bd13 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_star-25-be5af28b6bccf529a31ef68619de699e b/sql/hive/src/test/resources/golden/join_star-25-be5af28b6bccf529a31ef68619de699e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 b/sql/hive/src/test/resources/golden/join_star-26-a412a0620a359a62e2ab4e45fa8e0330 new file mode 100644 index 0000000000000..b89911131bd13 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 b/sql/hive/src/test/resources/golden/join_star-27-9ae4ebb8ceaa724bdbc88126f86692f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 b/sql/hive/src/test/resources/golden/join_star-28-98a3f4d8a2209e771c57462d4b723ff9 new file mode 100644 index 0000000000000..19611e75e33c3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da b/sql/hive/src/test/resources/golden/join_star-3-4f4ca11033af3894cc2e720ebb69e5da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-4-aaba115af7543b214820a48a694f2463 b/sql/hive/src/test/resources/golden/join_star-4-aaba115af7543b214820a48a694f2463 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da b/sql/hive/src/test/resources/golden/join_star-5-c72a8aff3a51f9e59bb62fd99807d8da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a b/sql/hive/src/test/resources/golden/join_star-6-73d96ed451a4964e5c364dbdb56d487a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a b/sql/hive/src/test/resources/golden/join_star-7-b007c3d4b6ed10df9e875e23644b673a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 b/sql/hive/src/test/resources/golden/join_star-8-a957982d8981ff0a35397ca449297024 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 b/sql/hive/src/test/resources/golden/join_star-9-904e30d8615eb411fb24c2cc08df94f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 b/sql/hive/src/test/resources/golden/join_thrift-0-811b62ecbacdb26c67fa260ff3747a41 new file mode 100644 index 0000000000000..07433297e6de1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db b/sql/hive/src/test/resources/golden/join_thrift-1-4f47dd107d2be1212411bda214c0d1db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b b/sql/hive/src/test/resources/golden/join_thrift-2-d4bfb7bab262dc2898431680711dec1b new file mode 100644 index 0000000000000..4aeaf4ff1f1b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 b/sql/hive/src/test/resources/golden/join_vc-0-f6269c9b545c9f908ef9aebf1a2ac097 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/join_view-0-d286410aa1d5f5c8d91b863a6d6e29c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 b/sql/hive/src/test/resources/golden/join_view-1-4e6a4fd729bac739f5f9b9e2c42b2467 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 b/sql/hive/src/test/resources/golden/join_view-2-eda40dba9678df62dde73fc5dafb2b44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b b/sql/hive/src/test/resources/golden/join_view-3-eccb00d8dada8ab56a48c373e381e02b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 b/sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/join_view-4-763ab5853bff619e6525c01e46b2a923 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 b/sql/hive/src/test/resources/golden/join_view-5-85baeea910adc4589bc3ec2ce0b1e856 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef b/sql/hive/src/test/resources/golden/join_view-6-c37b7962ab7371f94a9c54d61f7638ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d b/sql/hive/src/test/resources/golden/join_view-7-a14cfe3eff322066e61023ec06c7735d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 b/sql/hive/src/test/resources/golden/join_view-8-d286410aa1d5f5c8d91b863a6d6e29c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 b/sql/hive/src/test/resources/golden/join_view-9-4e6a4fd729bac739f5f9b9e2c42b2467 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee b/sql/hive/src/test/resources/golden/lateral view1-0-85bfbfd635aee807ede359073fb26dee new file mode 100644 index 0000000000000..1b8942763520b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb b/sql/hive/src/test/resources/golden/lateral view2-0-6d92d10040c715df89db0f61ae4250bb new file mode 100644 index 0000000000000..86296b92d98ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 b/sql/hive/src/test/resources/golden/lateral view3-0-55b6de1dbad92682804e98524c0d6c12 new file mode 100644 index 0000000000000..8216956a5c636 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 b/sql/hive/src/test/resources/golden/lateral view4-0-3ed6bfa23add415990b443fc232b4ba0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 b/sql/hive/src/test/resources/golden/lateral view4-1-6ced0ab5d543a2c5c6c7f7e27ed04814 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e b/sql/hive/src/test/resources/golden/lateral view4-2-ed9961425eee97a5f35c5b6e69dc368e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b b/sql/hive/src/test/resources/golden/lateral view5-0-f797cf9d04a98cfe477d14703aeb294b new file mode 100644 index 0000000000000..4895558ca6142 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a b/sql/hive/src/test/resources/golden/lateral view6-0-f92fe6429a6630d4991f8ad76ff1d79a new file mode 100644 index 0000000000000..269df08795dc9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b b/sql/hive/src/test/resources/golden/lateral_view-0-bc2e73d1da334f7bf550244ce6f6e77b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 b/sql/hive/src/test/resources/golden/lateral_view-1-82287db1270c77c3af7a60d65c4bdc80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e b/sql/hive/src/test/resources/golden/lateral_view-10-1f2df08433fe12a9ae086e3d9309039e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 b/sql/hive/src/test/resources/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view-11-d884acac077c5a02ef048069dd8c16d7 @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/sql/hive/src/test/resources/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 b/sql/hive/src/test/resources/golden/lateral_view-12-f539cc98118a276d38c61fcc0ad0b09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba b/sql/hive/src/test/resources/golden/lateral_view-13-f1ca785bf643a4a3cd1f2f823da158ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 b/sql/hive/src/test/resources/golden/lateral_view-14-f29001fbaafbdc4aaa1e0a0c18481aa7 new file mode 100644 index 0000000000000..fd859feff393b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e b/sql/hive/src/test/resources/golden/lateral_view-15-e1fad51e2f77de35d5781d788d70057e new file mode 100644 index 0000000000000..512cb498a2f6a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 b/sql/hive/src/test/resources/golden/lateral_view-16-5eafb0317b6974eb588075d152719a79 new file mode 100644 index 0000000000000..1f752ac7f4a39 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab b/sql/hive/src/test/resources/golden/lateral_view-17-4c6ff06840694c274bf8a91bade677ab new file mode 100644 index 0000000000000..fd859feff393b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e b/sql/hive/src/test/resources/golden/lateral_view-18-a98a136672c920c0463bd49b4ec7b65e new file mode 100644 index 0000000000000..512cb498a2f6a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 b/sql/hive/src/test/resources/golden/lateral_view-19-9b417d5554cbeb8f536a1613879295f7 new file mode 100644 index 0000000000000..f372b042130e7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 b/sql/hive/src/test/resources/golden/lateral_view-2-bfd7354229ac8a934c8f978b43b28565 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 b/sql/hive/src/test/resources/golden/lateral_view-20-d75015d8e44499526fec784ec00b905 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 b/sql/hive/src/test/resources/golden/lateral_view-21-9eec1a93e07de6acbc36b1156424ca11 new file mode 100644 index 0000000000000..30e4fa100f778 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e b/sql/hive/src/test/resources/golden/lateral_view-3-16a7be14adbb71fb9e6c7e0f00cfe86e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 b/sql/hive/src/test/resources/golden/lateral_view-4-f5855ce2bc7877c698e7f94504d12d62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae b/sql/hive/src/test/resources/golden/lateral_view-5-857f93a5c8d5ba00d214384117147bae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 b/sql/hive/src/test/resources/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 new file mode 100644 index 0000000000000..d6426b33c4d5d --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view-6-e78723941c3b42349f9eb804b4b82d15 @@ -0,0 +1 @@ +0 val_0 1 diff --git a/sql/hive/src/test/resources/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 b/sql/hive/src/test/resources/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view-7-7c88418f83112c55d08a0727d6248825 @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/sql/hive/src/test/resources/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 b/sql/hive/src/test/resources/golden/lateral_view-8-d2bff21465d3db7bbff8b9913c012452 new file mode 100644 index 0000000000000..423bb3e88a24e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 b/sql/hive/src/test/resources/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 new file mode 100644 index 0000000000000..01e79c32a8c99 --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view-9-60290e9a3b75a39c21dd44817d90d4a7 @@ -0,0 +1,3 @@ +1 +2 +3 diff --git a/sql/hive/src/test/resources/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 b/sql/hive/src/test/resources/golden/lateral_view_cp-0-6ae48c990343343aad4145203c364f79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 b/sql/hive/src/test/resources/golden/lateral_view_cp-1-72a14433dd2f25dd8c38107235fe1756 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 b/sql/hive/src/test/resources/golden/lateral_view_cp-2-647d019098d676b2fdb8a5127b633287 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 b/sql/hive/src/test/resources/golden/lateral_view_cp-3-3622d591963e6a7f021d07c7c35a1cd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_cp-4-507876b610812506343e9d251645170b b/sql/hive/src/test/resources/golden/lateral_view_cp-4-507876b610812506343e9d251645170b new file mode 100644 index 0000000000000..1f3d8a7a1fc08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view_cp-4-507876b610812506343e9d251645170b @@ -0,0 +1 @@ +1028 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 b/sql/hive/src/test/resources/golden/lateral_view_noalias-0-72509f06e1f7c5d5ccc292f775f8eea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 b/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 new file mode 100644 index 0000000000000..0da0d93886e01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view_noalias-1-6d5806dd1d2511911a5de1e205523f42 @@ -0,0 +1,2 @@ +key1 100 +key2 200 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a b/sql/hive/src/test/resources/golden/lateral_view_noalias-2-155b3cc2f5054725a9c2acca3c38c00a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 b/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 new file mode 100644 index 0000000000000..0da0d93886e01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view_noalias-3-3b7045ace234af8e5e86d8ac23ccee56 @@ -0,0 +1,2 @@ +key1 100 +key2 200 diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe b/sql/hive/src/test/resources/golden/lateral_view_noalias-4-e1eca4e08216897d090259d4fd1e3fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 b/sql/hive/src/test/resources/golden/lateral_view_noalias-5-16d227442dd775615c6ecfceedc6c612 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 b/sql/hive/src/test/resources/golden/lateral_view_outer-0-b66c363516d1f717765df9b91da3b5b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa b/sql/hive/src/test/resources/golden/lateral_view_outer-1-8d4332785ff69bb86607700c133d0baa new file mode 100644 index 0000000000000..a2a7fdd233a2a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 b/sql/hive/src/test/resources/golden/lateral_view_outer-2-b4474ec12d042fca7a21a2cd551c1068 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d b/sql/hive/src/test/resources/golden/lateral_view_outer-3-57b2ffd7b60708551238c491a2a8685d new file mode 100644 index 0000000000000..37d918a386d7d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e b/sql/hive/src/test/resources/golden/lateral_view_outer-4-96fe3dc4f8116e535c322437a59b5e4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 b/sql/hive/src/test/resources/golden/lateral_view_outer-5-2ec3aeb923621c64da042402709e64e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 b/sql/hive/src/test/resources/golden/lateral_view_outer-6-511e4df505342e04c20e50fda8962120 new file mode 100644 index 0000000000000..3ad56f646ed85 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view_ppd-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 b/sql/hive/src/test/resources/golden/lateral_view_ppd-1-cd326124717660a333f0915395f96768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f b/sql/hive/src/test/resources/golden/lateral_view_ppd-2-25c2f5138fc88cf7fc34339a501de05f new file mode 100644 index 0000000000000..f0a72370f697b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 b/sql/hive/src/test/resources/golden/lateral_view_ppd-3-a7d1b9240892d32ae3adf6ff2289a760 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 b/sql/hive/src/test/resources/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 new file mode 100644 index 0000000000000..1c7ec3df89594 --- /dev/null +++ b/sql/hive/src/test/resources/golden/lateral_view_ppd-4-c746547306ed67a5e8a0fce57023c9c9 @@ -0,0 +1,3 @@ +val_0 1 +val_0 1 +val_0 1 diff --git a/sql/hive/src/test/resources/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f b/sql/hive/src/test/resources/golden/lateral_view_ppd-5-2d087c55c7ad9f3dc51d8406833feb1f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa b/sql/hive/src/test/resources/golden/lateral_view_ppd-6-63f8025bbf66831e84b98f9429e6c7fa new file mode 100644 index 0000000000000..3c52da67708d8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b b/sql/hive/src/test/resources/golden/lateral_view_ppd-7-b8eb877331710c06ebc7dbaab5a7155b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 b/sql/hive/src/test/resources/golden/lateral_view_ppd-8-fe627f796b01fe05ef7ccc30afe7c510 new file mode 100644 index 0000000000000..78b21ab365aac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 b/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 new file mode 100644 index 0000000000000..bf0d87ab1b2b0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/length.udf-0-e85ebb8ce5d939964fd87bd13b326c02 @@ -0,0 +1 @@ +4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb b/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/limit_pushdown-0-79b294d0081c3dfd36c5b8b5e78dc7fb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/limit_pushdown-1-d0a93f40892e3894460553b443c77428 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 b/sql/hive/src/test/resources/golden/limit_pushdown-10-19842c7708a0787e59cc5e6b8c3a4250 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f b/sql/hive/src/test/resources/golden/limit_pushdown-11-b435672262fc3c02d1ed1a93ff8d856f new file mode 100644 index 0000000000000..9e3b31ad52c13 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 b/sql/hive/src/test/resources/golden/limit_pushdown-12-a3516c212d6c79986536edbd9c961098 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 b/sql/hive/src/test/resources/golden/limit_pushdown-13-93906f7e87e5e3dc4b9590ec13b6af78 new file mode 100644 index 0000000000000..ebf6c0424c26d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 b/sql/hive/src/test/resources/golden/limit_pushdown-14-cfae77212d164efb18208f71332fd368 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 b/sql/hive/src/test/resources/golden/limit_pushdown-15-62a90d3ba54cc68055b3502c3864a3c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 b/sql/hive/src/test/resources/golden/limit_pushdown-16-b618008e10acc4ee96b6ea2dbdf0f32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 b/sql/hive/src/test/resources/golden/limit_pushdown-17-11c349c0db0f869be88351256650fe60 new file mode 100644 index 0000000000000..153dcec21bc5b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 b/sql/hive/src/test/resources/golden/limit_pushdown-18-66ecdd019c936dec1ab5c9389c9071b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/limit_pushdown-19-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 b/sql/hive/src/test/resources/golden/limit_pushdown-2-4d1e292b00635298240ff909be64dce4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb b/sql/hive/src/test/resources/golden/limit_pushdown-20-d4de935f7a059cce44889c6ba69cfddb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 b/sql/hive/src/test/resources/golden/limit_pushdown-21-f04dee0f94443ca01320657897cbb914 new file mode 100644 index 0000000000000..ae8f0265b71ca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e b/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/limit_pushdown-22-4d378725e22e7e48c861983ba935bf5e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a b/sql/hive/src/test/resources/golden/limit_pushdown-23-53b4dd4899cb2ba681a801a3b5ca155a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 b/sql/hive/src/test/resources/golden/limit_pushdown-24-cb58113b9dfd3e93d4623cba6a090976 new file mode 100644 index 0000000000000..e604892422d59 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a b/sql/hive/src/test/resources/golden/limit_pushdown-25-1b6cdcbc002f5f1bde0f369a0dd1632a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c b/sql/hive/src/test/resources/golden/limit_pushdown-26-e691b284f1b830d7b83a36750105737c new file mode 100644 index 0000000000000..92dc6ce9dbf9e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 b/sql/hive/src/test/resources/golden/limit_pushdown-3-cc674af3ce71c06670e366932df43462 new file mode 100644 index 0000000000000..95f5492558a9b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f b/sql/hive/src/test/resources/golden/limit_pushdown-4-81bbb7300da27bc63f7a139677faac3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 b/sql/hive/src/test/resources/golden/limit_pushdown-5-c0664bc5bff121823ac2f37df13d6bd7 new file mode 100644 index 0000000000000..0bac402cfa497 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 b/sql/hive/src/test/resources/golden/limit_pushdown-6-b722630e977a5ca509234fc417d7b30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 b/sql/hive/src/test/resources/golden/limit_pushdown-7-e8c8087a3910ea53a725b01b9d05b921 new file mode 100644 index 0000000000000..b57d0cc951566 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 b/sql/hive/src/test/resources/golden/limit_pushdown-8-6194a19c1d5065731ec26ea1d5a390e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a b/sql/hive/src/test/resources/golden/limit_pushdown-9-1446f634128a1c3e7cd224ea82452d0a new file mode 100644 index 0000000000000..0e95c446f3b0d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 b/sql/hive/src/test/resources/golden/lineage1-0-c021950f995f1d6a7b83ffe044daa750 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 b/sql/hive/src/test/resources/golden/lineage1-1-aeb863486b8fe899ee741fc8c8418fc9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c b/sql/hive/src/test/resources/golden/lineage1-2-f92e96c6357273ea6cbb2195a418df9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f b/sql/hive/src/test/resources/golden/literal_double-0-10ef1098e35d900983be3814de8f974f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 b/sql/hive/src/test/resources/golden/literal_double-1-3863c17e03c9c1cd68452106a8721d13 new file mode 100644 index 0000000000000..69505e73feb1d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 b/sql/hive/src/test/resources/golden/literal_ints-0-5ffd1b49cdda4149aef2c61c53a56890 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 b/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 new file mode 100644 index 0000000000000..6dc85dd550540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/literal_ints-1-b41c42ce2f6ba483b68bb08752b95ec4 @@ -0,0 +1 @@ +100 100 100 100 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 b/sql/hive/src/test/resources/golden/literal_string-0-9b48d41e5c9e41ddc070e2fd31ace15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d b/sql/hive/src/test/resources/golden/literal_string-1-2cf4b7268b47246afdf6c792acca379d new file mode 100644 index 0000000000000..6323f4efa99c7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part1-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d b/sql/hive/src/test/resources/golden/load_dyn_part1-1-cd849c4fe1229428da98947e3e43b46d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 b/sql/hive/src/test/resources/golden/load_dyn_part1-10-fca9513ea05bfb8b7e0e6f337d184d66 new file mode 100644 index 0000000000000..d7a8f25b41301 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 b/sql/hive/src/test/resources/golden/load_dyn_part1-11-9c82167763a771c175c656786d545798 new file mode 100644 index 0000000000000..653516475da22 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 b/sql/hive/src/test/resources/golden/load_dyn_part1-2-9c0d4354b6a9be351fa32a66ff58a177 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 b/sql/hive/src/test/resources/golden/load_dyn_part1-3-e17dba1884f6a1d2b5002925afd7c2d3 new file mode 100644 index 0000000000000..e0dcec0869734 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part1-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part1-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f b/sql/hive/src/test/resources/golden/load_dyn_part1-6-255ad4511130fb8c9ee9d65b7c95743f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 b/sql/hive/src/test/resources/golden/load_dyn_part1-7-a33b2c9d962e4921c98e62387f3989f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 b/sql/hive/src/test/resources/golden/load_dyn_part1-8-ea921e0af59a4940a11c94143b1c4b32 new file mode 100644 index 0000000000000..0a88e449f625a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 b/sql/hive/src/test/resources/golden/load_dyn_part1-9-30bc31441828a053d1a675b225a5d617 new file mode 100644 index 0000000000000..1634adfc4d70d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part10-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 b/sql/hive/src/test/resources/golden/load_dyn_part10-1-a5b30075b6c79d1a02e240f46ea1d318 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e b/sql/hive/src/test/resources/golden/load_dyn_part10-2-151ba0c3b8317902f1235ac07d58135e new file mode 100644 index 0000000000000..9d3a522ff81dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part10-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part10-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part10-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 b/sql/hive/src/test/resources/golden/load_dyn_part10-6-48d24be440fbbd48f82efeeb05f663c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 b/sql/hive/src/test/resources/golden/load_dyn_part10-7-17d0630e1125ac326c5a7a83e6b8bcc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 b/sql/hive/src/test/resources/golden/load_dyn_part10-8-245027204484e281e1cfaf74386d2967 new file mode 100644 index 0000000000000..574727266b183 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 b/sql/hive/src/test/resources/golden/load_dyn_part10-9-eb29a5b2392bf821b2ff51b70863d531 new file mode 100644 index 0000000000000..3994499f92576 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part11-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 b/sql/hive/src/test/resources/golden/load_dyn_part11-1-bcf9e49f6a630b18108da59b243455f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 b/sql/hive/src/test/resources/golden/load_dyn_part11-2-4301f87a8dbf9576788637386e26f9a2 new file mode 100644 index 0000000000000..6aa67737fa92d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part11-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part11-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part11-5-ea607fbed28d20e5726f4501285d698d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part11-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 b/sql/hive/src/test/resources/golden/load_dyn_part11-7-b377ea0092e921e5c07e8f34d7c9f920 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb b/sql/hive/src/test/resources/golden/load_dyn_part11-8-9a4433518ac9ff49cb4b71812705adbb new file mode 100644 index 0000000000000..29ade2aff0542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a b/sql/hive/src/test/resources/golden/load_dyn_part11-9-3889a0cba1cf3c8f8c2e67957e69406a new file mode 100644 index 0000000000000..b7b1b914db857 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part12-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 b/sql/hive/src/test/resources/golden/load_dyn_part12-1-37e5641a1239a05bbd1fb9707f69cd59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b b/sql/hive/src/test/resources/golden/load_dyn_part12-2-4a10b19bdc478379bb8c8c9e5fe52c9b new file mode 100644 index 0000000000000..0c283c5378a6f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part12-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part12-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part12-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 b/sql/hive/src/test/resources/golden/load_dyn_part12-6-e5c79bdfc92b7b754b003d718d9717a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 b/sql/hive/src/test/resources/golden/load_dyn_part12-7-fd1422a86c9b12ce915a7fa2f2b7fc97 new file mode 100644 index 0000000000000..8eb67466a7a89 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 b/sql/hive/src/test/resources/golden/load_dyn_part12-8-fd656b581b8f8fbb8ac22f444dbc345 new file mode 100644 index 0000000000000..9f50245c18177 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part13-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 b/sql/hive/src/test/resources/golden/load_dyn_part13-1-55bf30e1b5f1aeac9ef284e5e4f19c28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 b/sql/hive/src/test/resources/golden/load_dyn_part13-2-d52536b8ac62f6e8152e394fe135a3e0 new file mode 100644 index 0000000000000..08699f286e384 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part13-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part13-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part13-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d b/sql/hive/src/test/resources/golden/load_dyn_part13-6-67ee926bc6b325a6bfc952bb81752a3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 b/sql/hive/src/test/resources/golden/load_dyn_part13-7-99993811a25b02e7904a9403f51775d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 b/sql/hive/src/test/resources/golden/load_dyn_part13-8-930d8e7a94f0cdf922322dae4a02e16 new file mode 100644 index 0000000000000..dfe9bcc93bcd3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d b/sql/hive/src/test/resources/golden/load_dyn_part13-9-c8de411bc094b37b59a2eb0baf6de55d new file mode 100644 index 0000000000000..88fe75804e584 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a b/sql/hive/src/test/resources/golden/load_dyn_part14-0-ff0d0473e56406f7cb66e83b9af25a6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e b/sql/hive/src/test/resources/golden/load_dyn_part14-1-253e2a15bfaef9aa781dc29fa324b51e new file mode 100644 index 0000000000000..170e3b095c5a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part14-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part14-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd b/sql/hive/src/test/resources/golden/load_dyn_part14-4-584fc7f440280f67bf5bfdb23370cafd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 b/sql/hive/src/test/resources/golden/load_dyn_part14-5-ce75e50bda381af53c8549b0d8662d94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 b/sql/hive/src/test/resources/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 new file mode 100644 index 0000000000000..538ed22d29976 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part14-6-3fd73cd9f8f9b991b7e72405c00cf5d6 @@ -0,0 +1,2 @@ +value= +value=__HIVE_DEFAULT_PARTITION__ \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e b/sql/hive/src/test/resources/golden/load_dyn_part14-7-7c931249465f330d51ef0610f214429e new file mode 100644 index 0000000000000..9e1bc82fe15f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 b/sql/hive/src/test/resources/golden/load_dyn_part14_win-0-a58efbee279cc96fb5738e6ab389927 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e b/sql/hive/src/test/resources/golden/load_dyn_part14_win-1-253e2a15bfaef9aa781dc29fa324b51e new file mode 100644 index 0000000000000..df0090edb6b9e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part14_win-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part14_win-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd b/sql/hive/src/test/resources/golden/load_dyn_part14_win-4-584fc7f440280f67bf5bfdb23370cafd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 b/sql/hive/src/test/resources/golden/load_dyn_part14_win-5-ce75e50bda381af53c8549b0d8662d94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 b/sql/hive/src/test/resources/golden/load_dyn_part14_win-6-3fd73cd9f8f9b991b7e72405c00cf5d6 new file mode 100644 index 0000000000000..538ed22d29976 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e b/sql/hive/src/test/resources/golden/load_dyn_part14_win-7-7c931249465f330d51ef0610f214429e new file mode 100644 index 0000000000000..9e1bc82fe15f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part15-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part15-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c b/sql/hive/src/test/resources/golden/load_dyn_part15-2-1379abc2de057dc6d240a526f0dd8a3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 b/sql/hive/src/test/resources/golden/load_dyn_part15-3-9940fad8d8590e60bd726e9503ae4fa9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a b/sql/hive/src/test/resources/golden/load_dyn_part15-4-fa01199bab3e6aa47a82c9aec88aa76a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 b/sql/hive/src/test/resources/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 new file mode 100644 index 0000000000000..99a66d603300d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part15-5-9940fad8d8590e60bd726e9503ae4fa9 @@ -0,0 +1,3 @@ +part_key=%7B2 +part_key=1 +part_key=3%5D diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f b/sql/hive/src/test/resources/golden/load_dyn_part2-0-294e3beb0241f9f8eac7f54e1bfd775f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 b/sql/hive/src/test/resources/golden/load_dyn_part2-1-845923af04bb331c9f5995a3a3e84424 new file mode 100644 index 0000000000000..def850839a47b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part2-2-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part2-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part2-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b b/sql/hive/src/test/resources/golden/load_dyn_part2-5-fdb342199af53c5c18529c3a0472a38b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 b/sql/hive/src/test/resources/golden/load_dyn_part2-6-60864ea22e1173981ed651ddc2d944c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 b/sql/hive/src/test/resources/golden/load_dyn_part2-7-86ffa99b03fa88235b61bf1af7062c33 new file mode 100644 index 0000000000000..1f345dad614ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf b/sql/hive/src/test/resources/golden/load_dyn_part2-8-a1ff8a12d94378e7e1165bd78cf724cf new file mode 100644 index 0000000000000..5f936fa91d2f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 b/sql/hive/src/test/resources/golden/load_dyn_part2-9-3f29de9877006f9448272ef2422d6132 new file mode 100644 index 0000000000000..45c45d0082ee3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part3-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f b/sql/hive/src/test/resources/golden/load_dyn_part3-1-ce25d1f46dd5e5d9147e39566581514f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 b/sql/hive/src/test/resources/golden/load_dyn_part3-2-dbbba335c008a61a13c1472b34470397 new file mode 100644 index 0000000000000..d35fbec80c19e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part3-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part3-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part3-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 b/sql/hive/src/test/resources/golden/load_dyn_part3-6-17dee8c004f1e7db4542fbf4241edce3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 b/sql/hive/src/test/resources/golden/load_dyn_part3-7-867958e24e25ad098c5001bbc7102762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 b/sql/hive/src/test/resources/golden/load_dyn_part3-8-c3b6f0cc860b84851dd60c53b36a6437 new file mode 100644 index 0000000000000..2857cdf0aba86 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part4-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 b/sql/hive/src/test/resources/golden/load_dyn_part4-1-50822afef6986cfea20523ea6092e455 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 b/sql/hive/src/test/resources/golden/load_dyn_part4-10-7f0c1e54518d4d1d345577a100e670e8 new file mode 100644 index 0000000000000..3bbd322e374ff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 b/sql/hive/src/test/resources/golden/load_dyn_part4-11-1313644c9dad948bfcebd7386c309ab7 new file mode 100644 index 0000000000000..f1801743dd4e1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd b/sql/hive/src/test/resources/golden/load_dyn_part4-2-9893d8c02ab8bd59d3e9fb0665f508bd new file mode 100644 index 0000000000000..8017948fc5f2f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part4-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part4-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part4-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 b/sql/hive/src/test/resources/golden/load_dyn_part4-6-2869679fad49c57ba08169ea32271379 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 b/sql/hive/src/test/resources/golden/load_dyn_part4-7-f31c7a56b50a5ba425e85480e13c39e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 b/sql/hive/src/test/resources/golden/load_dyn_part4-8-cdff1da8e9e1e9242c49d895751af0a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 b/sql/hive/src/test/resources/golden/load_dyn_part4-9-3c344e5840c1df354a2a71722c27f0a0 new file mode 100644 index 0000000000000..4cf846d3d3559 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 b/sql/hive/src/test/resources/golden/load_dyn_part5-0-d9a2e2fa436aaf37e91ccf52e04226b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b b/sql/hive/src/test/resources/golden/load_dyn_part5-1-9a4d1f6a14227bb66bd01557a464da8b new file mode 100644 index 0000000000000..3f5f6b6f83715 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 b/sql/hive/src/test/resources/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 new file mode 100644 index 0000000000000..2dcdfd1217ced --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part5-10-be7953ca7bd26623d3897f5060e13737 @@ -0,0 +1,3 @@ +0 val_0 +0 val_0 +0 val_0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d b/sql/hive/src/test/resources/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d new file mode 100644 index 0000000000000..dcd1d8643e3cb --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part5-11-6da8fa1e639db104128ba7e2f88f764d @@ -0,0 +1 @@ +2 val_2 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part5-2-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part5-3-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part5-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 b/sql/hive/src/test/resources/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part5-5-67f2c7448db01b6804c846f9f4f76928 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 b/sql/hive/src/test/resources/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part5-6-f6dcb13dead8bb4c003eb19099908190 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df b/sql/hive/src/test/resources/golden/load_dyn_part5-7-3ca3cc56a27939d62db1b52d86309df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 b/sql/hive/src/test/resources/golden/load_dyn_part5-8-669ccdcc0e0f3162ee44d51ff449fdd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 b/sql/hive/src/test/resources/golden/load_dyn_part5-9-854026d1c2add692f2f90ad74668bbf6 new file mode 100644 index 0000000000000..414ce6d5a4941 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part6-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 b/sql/hive/src/test/resources/golden/load_dyn_part6-1-9657a48a4266c176f84c7aaf115fbc30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d b/sql/hive/src/test/resources/golden/load_dyn_part6-2-779aa345cf2875089312ec26b046415d new file mode 100644 index 0000000000000..4b9e5b31f4401 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part6-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part6-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part6-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 b/sql/hive/src/test/resources/golden/load_dyn_part6-6-559ff31031a6a0ff6959655c6acd07a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 b/sql/hive/src/test/resources/golden/load_dyn_part6-7-17cbb9aeb8dfaa3f4ba31f4b5d871dd1 new file mode 100644 index 0000000000000..7df5f90186db3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f b/sql/hive/src/test/resources/golden/load_dyn_part6-8-1009bd2cbd88ddba97186fb76e96a4f new file mode 100644 index 0000000000000..c869646753baa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part7-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 b/sql/hive/src/test/resources/golden/load_dyn_part7-1-e31d34a1b14d706d2b78d083ea858c81 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a b/sql/hive/src/test/resources/golden/load_dyn_part7-2-5d4c3c48f53d55e26ca142ee70d1706a new file mode 100644 index 0000000000000..d62156d392d5e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 b/sql/hive/src/test/resources/golden/load_dyn_part7-3-ad94ac1192dbace6b5cf5915387e94e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 b/sql/hive/src/test/resources/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 new file mode 100644 index 0000000000000..ed0d81e7a9d44 --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part7-4-e98039d0d4ef775fb5594bebffacf4f6 @@ -0,0 +1 @@ +ds=2010-03-03/hr=12 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 b/sql/hive/src/test/resources/golden/load_dyn_part7-5-b3615ce72a55dba805303145030c8a93 new file mode 100644 index 0000000000000..542a689ea8311 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part8-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c b/sql/hive/src/test/resources/golden/load_dyn_part8-1-ff5ce932ae5ba496badee7f2465f272c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a b/sql/hive/src/test/resources/golden/load_dyn_part8-2-9e1df41acabef83f62464f52c2396c8a new file mode 100644 index 0000000000000..91fa51871a4b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part8-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part8-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part8-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 b/sql/hive/src/test/resources/golden/load_dyn_part8-6-86db06cb739ceef70c8885469d847495 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 b/sql/hive/src/test/resources/golden/load_dyn_part8-7-be6ace08b68bc4346456103640308cf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b b/sql/hive/src/test/resources/golden/load_dyn_part8-8-cc6aa0fc59ede89158d4f19752660b8b new file mode 100644 index 0000000000000..a3c07bb68d96d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db b/sql/hive/src/test/resources/golden/load_dyn_part8-9-a7456fceb35f62a282db750384f480db new file mode 100644 index 0000000000000..4c0ed5d6fc5f1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/load_dyn_part9-0-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f b/sql/hive/src/test/resources/golden/load_dyn_part9-1-2c61920dcd46ece096fd12875871709f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a b/sql/hive/src/test/resources/golden/load_dyn_part9-2-748ac33315295f8e55f2480f7714c27a new file mode 100644 index 0000000000000..267cb634d6bd3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part9-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part9-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_dyn_part9-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 b/sql/hive/src/test/resources/golden/load_dyn_part9-6-3c2f7b27f021441c1d6b174a6e2c3045 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b b/sql/hive/src/test/resources/golden/load_dyn_part9-7-ffe5e03b1884bf7285a7e5463bf31e1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 b/sql/hive/src/test/resources/golden/load_dyn_part9-8-504a475a583d33f7c71db57e6774919 new file mode 100644 index 0000000000000..0a88e449f625a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb b/sql/hive/src/test/resources/golden/load_dyn_part9-9-72b1ad2231269b704130903b35ac04bb new file mode 100644 index 0000000000000..d7a8f25b41301 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 b/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 b/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-1-c84cd1b5b491bded8ac3b0521de599c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-2-7368973ec1870dd3b237c37eb3857b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 b/sql/hive/src/test/resources/golden/load_exist_part_authsuccess-3-9ee887603dcba5200918ae5200afa5d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 b/sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-0-8b3d200fd29aeafd07f16e7a732585a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f b/sql/hive/src/test/resources/golden/load_file_with_space_in_the_name-1-4399d9b3f970d3c5a34d1f9bf7b7447f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 b/sql/hive/src/test/resources/golden/load_fs2-0-517732da2c84ae17095b0e1d96f74d97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 b/sql/hive/src/test/resources/golden/load_fs2-1-5018c84e09be70bf663594a89f3ad731 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 b/sql/hive/src/test/resources/golden/load_fs2-2-94d2317b453b3b49bb9f2b58040b4748 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc b/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-0-f99b4f29506d65c841fb1db18263fbcc @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 b/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-1-b64a19f7101a4fb3b5d08b2f6e296400 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 b/sql/hive/src/test/resources/golden/load_hdfs_file_with_space_in_the_name-2-2087e00fe000e00f64e819dca59be450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 b/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-0-fa705a031ff5d97558f29c2b5b9de282 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 b/sql/hive/src/test/resources/golden/load_nonpart_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 b/sql/hive/src/test/resources/golden/load_part_authsuccess-0-84028c4ca541d126baffc20d6d876810 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e b/sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/load_part_authsuccess-1-7368973ec1870dd3b237c37eb3857b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 b/sql/hive/src/test/resources/golden/load_part_authsuccess-2-9ee887603dcba5200918ae5200afa5d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 b/sql/hive/src/test/resources/golden/loadpart1-0-fa705a031ff5d97558f29c2b5b9de282 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c b/sql/hive/src/test/resources/golden/loadpart1-1-4bf1504274319c44d370b58092fe016c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 b/sql/hive/src/test/resources/golden/loadpart1-2-b0ebbe71c220979b8fd4a36ffa501bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 b/sql/hive/src/test/resources/golden/loadpart1-3-21f4ee91fa1c65e8579e4cbe4777d7a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 b/sql/hive/src/test/resources/golden/loadpart1-4-e2e5e7a0378c6f0c28391c447ec9cee9 new file mode 100644 index 0000000000000..cf367dacbdf0e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc b/sql/hive/src/test/resources/golden/loadpart1-5-892cb7ecc26e84f1c033b95a3ee3edc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/loadpart1-6-ca5e3149f2b190d7df923a3e5c1cb07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 b/sql/hive/src/test/resources/golden/loadpart1-7-c6493490f898e72dc7ed1bc2d4721aa4 new file mode 100644 index 0000000000000..cf367dacbdf0e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/loadpart1-8-ca5e3149f2b190d7df923a3e5c1cb07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 b/sql/hive/src/test/resources/golden/loadpart1-9-c012b29f0d7720fbc515aa5fe2759ac7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 b/sql/hive/src/test/resources/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/loadpart_err-0-8d7fa992960b4c8ec2cb874be479de37 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 b/sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/loadpart_err-1-8e68053c7f96f268ae1e25082e9b7517 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e b/sql/hive/src/test/resources/golden/loadpart_err-2-21fe8ff9059167209647e7ea086f483e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/lock1-0-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/lock1-1-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/lock1-2-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/lock2-0-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/lock2-1-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/lock2-2-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock2-3-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/lock2-3-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/lock2-4-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca b/sql/hive/src/test/resources/golden/lock2-5-e8d1d10c308a73eef78dde414a5e40ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock3-0-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/lock3-0-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/lock3-1-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 b/sql/hive/src/test/resources/golden/lock3-2-b1ca816784e88f105b2fce1175340c33 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/lock3-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/lock3-4-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac b/sql/hive/src/test/resources/golden/lock3-5-8096935c5c1755f9b88583e8c72921ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe b/sql/hive/src/test/resources/golden/lock3-6-598ba296ba1c6931f4161a9f50b00cbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 b/sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/lock4-0-9583a6f9fe5ca6c74ff44d7d064fed92 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/lock4-1-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/lock4-1-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/lock4-2-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 b/sql/hive/src/test/resources/golden/lock4-3-b1ca816784e88f105b2fce1175340c33 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/lock4-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/lock4-5-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac b/sql/hive/src/test/resources/golden/lock4-6-8096935c5c1755f9b88583e8c72921ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe b/sql/hive/src/test/resources/golden/lock4-7-598ba296ba1c6931f4161a9f50b00cbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/louter_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd b/sql/hive/src/test/resources/golden/louter_join_ppr-1-498e526f13a05a053a338d766f7351cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 b/sql/hive/src/test/resources/golden/louter_join_ppr-2-3d41e966f69a64babb783d1aad0f1b73 new file mode 100644 index 0000000000000..19492fd335bcb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 b/sql/hive/src/test/resources/golden/louter_join_ppr-3-ac75aeb98d142f514ed7b9b44e78c7c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef b/sql/hive/src/test/resources/golden/louter_join_ppr-4-a4f0ff6d2a367242836379fa9e3fe3ef new file mode 100644 index 0000000000000..3a57720041fb3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 b/sql/hive/src/test/resources/golden/louter_join_ppr-5-1209db6544b421ea32a360d863becd94 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 b/sql/hive/src/test/resources/golden/louter_join_ppr-6-dedfbaea184f5e3a29226e6e6bc6735 new file mode 100644 index 0000000000000..19492fd335bcb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 b/sql/hive/src/test/resources/golden/louter_join_ppr-7-9548b9e389f361ac8eccb3de7255da42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 b/sql/hive/src/test/resources/golden/louter_join_ppr-8-6fca189c46645f124d5fcb82564b703 new file mode 100644 index 0000000000000..3a57720041fb3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d b/sql/hive/src/test/resources/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d new file mode 100644 index 0000000000000..657eea30d475f --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin1-0-a267d586eb00766a0ac1b16f5b45cf9d @@ -0,0 +1 @@ +1114788 diff --git a/sql/hive/src/test/resources/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 b/sql/hive/src/test/resources/golden/mapjoin1-1-abd9364d276ec89352232da5e2237768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 b/sql/hive/src/test/resources/golden/mapjoin1-10-c08fefa00b89f50dd365208151593505 new file mode 100644 index 0000000000000..cfae441c4a10a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 b/sql/hive/src/test/resources/golden/mapjoin1-11-fb5e414c98754b7e79c744606aa6ccb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 b/sql/hive/src/test/resources/golden/mapjoin1-12-6fa8df1d49ba571bb9d2615ad22958d9 new file mode 100644 index 0000000000000..6d8155efd76cd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 b/sql/hive/src/test/resources/golden/mapjoin1-2-fe84593f006c85e68fbb797394cdccd0 new file mode 100644 index 0000000000000..44f1acd59de68 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d b/sql/hive/src/test/resources/golden/mapjoin1-3-8439a0592619790b64d16d2506f2233d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 b/sql/hive/src/test/resources/golden/mapjoin1-4-c08fefa00b89f50dd365208151593505 new file mode 100644 index 0000000000000..cfae441c4a10a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba b/sql/hive/src/test/resources/golden/mapjoin1-5-72068bd4cdac40e4d18fd729f39855ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 b/sql/hive/src/test/resources/golden/mapjoin1-6-6fa8df1d49ba571bb9d2615ad22958d9 new file mode 100644 index 0000000000000..6d8155efd76cd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 b/sql/hive/src/test/resources/golden/mapjoin1-7-757dfb540b8a49b3663f8caba7476ec5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 b/sql/hive/src/test/resources/golden/mapjoin1-8-fe84593f006c85e68fbb797394cdccd0 new file mode 100644 index 0000000000000..44f1acd59de68 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 b/sql/hive/src/test/resources/golden/mapjoin1-9-5eabdf151ff9fedb64559d2fbd1ae266 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-1-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 b/sql/hive/src/test/resources/golden/mapjoin_distinct-10-53a3e706e797dc6b9e7f5ee8b100fa56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f b/sql/hive/src/test/resources/golden/mapjoin_distinct-11-4489654b888efc588b13ee1cda1b6a9f new file mode 100644 index 0000000000000..eab7fd7a51ea4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-12-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-13-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 b/sql/hive/src/test/resources/golden/mapjoin_distinct-14-53a3e706e797dc6b9e7f5ee8b100fa56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f b/sql/hive/src/test/resources/golden/mapjoin_distinct-15-4489654b888efc588b13ee1cda1b6a9f new file mode 100644 index 0000000000000..eab7fd7a51ea4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 b/sql/hive/src/test/resources/golden/mapjoin_distinct-2-53a3e706e797dc6b9e7f5ee8b100fa56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f b/sql/hive/src/test/resources/golden/mapjoin_distinct-3-4489654b888efc588b13ee1cda1b6a9f new file mode 100644 index 0000000000000..eab7fd7a51ea4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-4-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-5-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 b/sql/hive/src/test/resources/golden/mapjoin_distinct-6-53a3e706e797dc6b9e7f5ee8b100fa56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f b/sql/hive/src/test/resources/golden/mapjoin_distinct-7-4489654b888efc588b13ee1cda1b6a9f new file mode 100644 index 0000000000000..eab7fd7a51ea4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-8-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_distinct-9-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_filter_on_outerjoin-0-407016bf2679fb9e9d076a2d115e859d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-1-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-2-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-3-cb38700198e89779e4dc1b25026d92a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-4-f9a2e0792bfe37c48895b8044a3a3702 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 new file mode 100644 index 0000000000000..8e7fe1e4cae08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_mapjoin-5-c47698bac140454637a999e583941ce7 @@ -0,0 +1,2 @@ +5308 +5308 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_subquery-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_subquery-1-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_subquery-2-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b b/sql/hive/src/test/resources/golden/mapjoin_subquery-3-4422532005e909173a4799d8d5091f1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 b/sql/hive/src/test/resources/golden/mapjoin_subquery-4-7df121f9774cb23edc557b98ad1e1924 new file mode 100644 index 0000000000000..482848c1ef8aa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c b/sql/hive/src/test/resources/golden/mapjoin_subquery-5-d9e59bfa950495629b7ba4bc6700405c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b b/sql/hive/src/test/resources/golden/mapjoin_subquery-6-dcdc6b87a9b87b0ab10ad0e44a197f1b new file mode 100644 index 0000000000000..482848c1ef8aa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-0-59fdb2842fbf4f530366f0237ff235e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-1-c0a9763a930555f846a2576d003fb517 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_subquery2-10-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_subquery2-11-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-12-7ebf0bee394756c0e0c78bdd1034f183 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-13-c876a518451059f17fc15e29f6f57951 new file mode 100644 index 0000000000000..5c62121a6600c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-14-59fdb2842fbf4f530366f0237ff235e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-15-c0a9763a930555f846a2576d003fb517 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-16-2f8ecc54049960ee4707f5e6f491fdf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-2-2f8ecc54049960ee4707f5e6f491fdf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-3-681c6f08aac965e4156dbd5800064a68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-4-ab6020d67f5a99c0a87e630790507345 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-5-c0e460b0c5bceeeb5820a1240fa9f7d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-6-5353ee601eb42d5842690d3941683be1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-7-fb516ed5906b1f485d3e7e7eeaedd862 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e b/sql/hive/src/test/resources/golden/mapjoin_subquery2-8-d524906728fef9f559709fe0922ab24e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_subquery2-9-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d b/sql/hive/src/test/resources/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-0-407016bf2679fb9e9d076a2d115e859d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc b/sql/hive/src/test/resources/golden/mapjoin_test_outer-1-b1ac944eac23eb8af1f1f659659c7bcc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a b/sql/hive/src/test/resources/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mapjoin_test_outer-10-ce1ef910fff98f174931cc641f7cef3a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-11-cfaaae6c3ef2a5eb8cd7ec5065ca7795 new file mode 100644 index 0000000000000..271f3e168fd23 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-12-80993ab7f757001e1f058bf8609f8420 new file mode 100644 index 0000000000000..271f3e168fd23 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-13-a5d200c74f7000ad3e36fac90c980d34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-14-7fe52008c4a98853d086d17fc3c21906 new file mode 100644 index 0000000000000..f00666f6113d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-2-43561759b314d02b2dba5eb49a65c515 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f b/sql/hive/src/test/resources/golden/mapjoin_test_outer-3-6ee8075ac3ad15ad1ac103d815544e7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-4-9e1f369b391b4a050250e0a954ffbb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-5-c95eb9bb8b40a43691c4ef432d8f38b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-6-7fe52008c4a98853d086d17fc3c21906 new file mode 100644 index 0000000000000..f00666f6113d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-7-ab7726921abfa43bb20ddfbc05f73c24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a b/sql/hive/src/test/resources/golden/mapjoin_test_outer-8-dfb08d397d3fe163d75c3b758097b68a new file mode 100644 index 0000000000000..f00666f6113d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 b/sql/hive/src/test/resources/golden/mapjoin_test_outer-9-6c45ce60b3dfce0e8bd19eedd57ee017 new file mode 100644 index 0000000000000..271f3e168fd23 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c b/sql/hive/src/test/resources/golden/mapreduce1-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe b/sql/hive/src/test/resources/golden/mapreduce1-1-e43492b1834d823a66b0f6499b7d2fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e b/sql/hive/src/test/resources/golden/mapreduce1-2-c32bd8b1734c410b3686469a7a3500e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/mapreduce1-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..3af52be3cd1fc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c b/sql/hive/src/test/resources/golden/mapreduce2-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 b/sql/hive/src/test/resources/golden/mapreduce2-1-22edb61e7c8a162887c58ebbdc4e4a45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b b/sql/hive/src/test/resources/golden/mapreduce2-2-3442d32acb17e007a8f844b38fda9a1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc b/sql/hive/src/test/resources/golden/mapreduce2-3-adea843673e541da8a735a5a34e7c7dc new file mode 100644 index 0000000000000..4ab18ace38f46 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c b/sql/hive/src/test/resources/golden/mapreduce3-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b b/sql/hive/src/test/resources/golden/mapreduce3-1-efaeccafc3db890b344dc6037266b71b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a b/sql/hive/src/test/resources/golden/mapreduce3-2-51905797c8299159dc1cf1ef69cd7a9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/mapreduce3-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..1504c19f78eb2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c b/sql/hive/src/test/resources/golden/mapreduce4-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f b/sql/hive/src/test/resources/golden/mapreduce4-1-7fed3aa9a8e544556f5b58f301d8bd2f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab b/sql/hive/src/test/resources/golden/mapreduce4-2-6906be683bdd3666075760de019ac5ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/mapreduce4-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..bf6c49af3ba4e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c b/sql/hive/src/test/resources/golden/mapreduce5-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 b/sql/hive/src/test/resources/golden/mapreduce5-1-52bff54eba53868ef3fd026cc6301a13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca b/sql/hive/src/test/resources/golden/mapreduce5-2-2cfcb0d4e390cd5fdd6d8a9bb26555ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/mapreduce5-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..1ecd5ad908e74 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c b/sql/hive/src/test/resources/golden/mapreduce6-0-904b34e86c266384dc261655162dde3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb b/sql/hive/src/test/resources/golden/mapreduce6-1-d5ed2c0aafd39b76fa414194add93ffb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 b/sql/hive/src/test/resources/golden/mapreduce6-2-c55bb2d4c519d62331671a5d7685f2b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/mapreduce6-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..1ecd5ad908e74 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 b/sql/hive/src/test/resources/golden/mapreduce7-0-78c8b5bf2d2a258066786ba03696ab82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a b/sql/hive/src/test/resources/golden/mapreduce7-1-8b7f1f886b749e43460052c98acd082a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 b/sql/hive/src/test/resources/golden/mapreduce7-2-b57470174a24aa3861df022375754e90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/mapreduce7-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..7fcf1a078a3da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 b/sql/hive/src/test/resources/golden/mapreduce8-0-78c8b5bf2d2a258066786ba03696ab82 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b b/sql/hive/src/test/resources/golden/mapreduce8-1-301e5bdcbbcdaea60348b3c24336629b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 b/sql/hive/src/test/resources/golden/mapreduce8-2-ec9012d7823a6b1d0e97315c275e2ad8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/mapreduce8-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..7fcf1a078a3da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge1-0-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge1-1-2c73c923962b91afdf0004a705432550 b/sql/hive/src/test/resources/golden/merge1-1-2c73c923962b91afdf0004a705432550 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 b/sql/hive/src/test/resources/golden/merge1-10-49d94c94e0a08155c9e5cb6e4efc6501 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 b/sql/hive/src/test/resources/golden/merge1-11-dcc5d9564bb8df6bac382c82c33ccd87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 b/sql/hive/src/test/resources/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge1-12-2e8e4adbfb21f25e7557dd86363c7138 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 b/sql/hive/src/test/resources/golden/merge1-13-cb13a05d07f9f1ad6c43edfc8b0e9359 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 b/sql/hive/src/test/resources/golden/merge1-14-dcc5d9564bb8df6bac382c82c33ccd87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 b/sql/hive/src/test/resources/golden/merge1-2-34854e1283de8e9ada3edd6bf897bc67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-3-1f560722f18ef618a0343313a0cac462 b/sql/hive/src/test/resources/golden/merge1-3-1f560722f18ef618a0343313a0cac462 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-4-3277fe538b66923cd879b45371838d2b b/sql/hive/src/test/resources/golden/merge1-4-3277fe538b66923cd879b45371838d2b new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 b/sql/hive/src/test/resources/golden/merge1-5-2f16345a20407b01e5cc5ae26ea902b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 b/sql/hive/src/test/resources/golden/merge1-6-1c02b26792354eae7fb53fa3cb752ac1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 b/sql/hive/src/test/resources/golden/merge1-7-5570ef5461d4a5cd01ec91da3a474dd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a b/sql/hive/src/test/resources/golden/merge1-8-37bd183ad3b7ad1e8550a138f7beb88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge1-9-64678b0928c4fd054e1578458001c86 b/sql/hive/src/test/resources/golden/merge1-9-64678b0928c4fd054e1578458001c86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge2-0-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge2-1-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 b/sql/hive/src/test/resources/golden/merge2-10-9f60e54bc4add2b1aff8473e2a756c79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 b/sql/hive/src/test/resources/golden/merge2-11-1c02b26792354eae7fb53fa3cb752ac1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 b/sql/hive/src/test/resources/golden/merge2-12-d2a36a13f8531cde3c66d4003048416 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a b/sql/hive/src/test/resources/golden/merge2-13-37bd183ad3b7ad1e8550a138f7beb88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-14-64678b0928c4fd054e1578458001c86 b/sql/hive/src/test/resources/golden/merge2-14-64678b0928c4fd054e1578458001c86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 b/sql/hive/src/test/resources/golden/merge2-15-aaaf38f80d7313738a51a49bd5aa14c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 b/sql/hive/src/test/resources/golden/merge2-16-d75f4405b860e4187882a72418ed6c83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 b/sql/hive/src/test/resources/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge2-17-2e8e4adbfb21f25e7557dd86363c7138 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f b/sql/hive/src/test/resources/golden/merge2-18-945682265ce2c1fe7fa69aeb57b4290f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 b/sql/hive/src/test/resources/golden/merge2-19-d75f4405b860e4187882a72418ed6c83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge2-2-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge2-3-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge2-4-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge2-5-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 b/sql/hive/src/test/resources/golden/merge2-6-319e3c6b684d2abd4dfd7314a05d4307 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 b/sql/hive/src/test/resources/golden/merge2-7-fa2f100bc67426120cb97cfc75a5fb36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 b/sql/hive/src/test/resources/golden/merge2-8-7435e0bf03e747705b0858d5dcccfcc1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd b/sql/hive/src/test/resources/golden/merge2-9-b81efaa65e1263e48278ef9062cca1dd new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-0-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-1-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32 b/sql/hive/src/test/resources/golden/merge4-10-692a197bd688b48f762e72978f54aa32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 b/sql/hive/src/test/resources/golden/merge4-11-ca0336ac3f600cb8b4230d9904686868 new file mode 100644 index 0000000000000..67c6db8591549 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11 b/sql/hive/src/test/resources/golden/merge4-12-62541540a18d68a3cb8497a741061d11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 b/sql/hive/src/test/resources/golden/merge4-13-ed1103f06609365b40e78d13c654cc71 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 b/sql/hive/src/test/resources/golden/merge4-14-ba5dbcd0527b8ddab284bc322255bfc7 new file mode 100644 index 0000000000000..30becc42d7b5a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a b/sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a new file mode 100644 index 0000000000000..4c867a5deff08 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-15-68f50dc2ad6ff803a372bdd88dd8e19a @@ -0,0 +1 @@ +1 1 2010-08-15 file, diff --git a/sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-2-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-3-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge4-4-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d b/sql/hive/src/test/resources/golden/merge4-5-3d24d877366c42030f6d9a596665720d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af b/sql/hive/src/test/resources/golden/merge4-6-b3a76420183795720ab3a384046e5af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d b/sql/hive/src/test/resources/golden/merge4-7-631a45828eae3f5f562d992efe4cd56d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 b/sql/hive/src/test/resources/golden/merge4-8-ca0336ac3f600cb8b4230d9904686868 new file mode 100644 index 0000000000000..9feba1dea5fd8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 b/sql/hive/src/test/resources/golden/merge4-9-ad3dc168c8b6f048717e39ab16b0a319 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-10-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-11-a49c9ee01ce8858a5f00c05523329200 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-12-dbf4eae8430a97a6e70b1c6222218cd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da b/sql/hive/src/test/resources/golden/merge_dynamic_partition-13-16adcdb0e324ad233769e124b5b349da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-14-79da9a1ce5c2d058b924387ac9fcde92 new file mode 100644 index 0000000000000..a4c81ff9f99cd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-15-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..d66ce5d097ce6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-16-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-17-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-18-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-19-a49c9ee01ce8858a5f00c05523329200 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-2-190cefc93e46906e404039de0fd5f513 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-20-d295db835d4fdeea34298702295ff7c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-21-9b9493a68ef7663e95ad86d02c45ec88 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-22-79da9a1ce5c2d058b924387ac9fcde92 new file mode 100644 index 0000000000000..a4c81ff9f99cd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-23-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..06444f372bd60 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-24-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-25-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-26-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-27-a49c9ee01ce8858a5f00c05523329200 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-28-ef7b35be7210f099d46448994d9dc605 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d b/sql/hive/src/test/resources/golden/merge_dynamic_partition-29-876c8fa15a32ac5bfcc6cb60993f6a4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f b/sql/hive/src/test/resources/golden/merge_dynamic_partition-3-25401dd2c1c258e06f1b96fefd19e27f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-30-79da9a1ce5c2d058b924387ac9fcde92 new file mode 100644 index 0000000000000..a4c81ff9f99cd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-31-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..352ab5a6b96a4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf b/sql/hive/src/test/resources/golden/merge_dynamic_partition-4-41df7b4938cff8b7ebffc3f5c701dccf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-5-8026cdd2390eec2c72a0591ae5668185 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-6-3b57c0e3fccea5322373f3725c95ec00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b b/sql/hive/src/test/resources/golden/merge_dynamic_partition-7-777de794b7f27ea63f29a9784663393b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-8-2f1578dbc029b62daa9d47d8fa473960 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition-9-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-10-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-11-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-12-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-13-2a9cffbef1ebd3df8e4de4eb22777cf9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-14-3a4c24fd561f459025264baa3fb6d87 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-15-dbf4eae8430a97a6e70b1c6222218cd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-16-16adcdb0e324ad233769e124b5b349da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-17-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..f3812861b3d6d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-2-190cefc93e46906e404039de0fd5f513 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-3-25401dd2c1c258e06f1b96fefd19e27f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-4-41df7b4938cff8b7ebffc3f5c701dccf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-5-8026cdd2390eec2c72a0591ae5668185 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-6-3b57c0e3fccea5322373f3725c95ec00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-7-777de794b7f27ea63f29a9784663393b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-8-6f7f59de1fbd607e844a2dc9394a2df8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba b/sql/hive/src/test/resources/golden/merge_dynamic_partition2-9-982f10fc4e4f6bab9d4ca5f3cecca2ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-10-d176a1b243ac7190fbc319d73a164e2d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-11-f64c176e6e3b2d9ffa1b9e14491dc4c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-12-a37d4d5c4eec82d56f91754e5fdb0f9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-13-c512eee654e7313ff9c6efb35c5b0a88 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-14-ea3e89ffe987e20dffd8388a2dbcc260 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-15-e525a096de36a3d157db1b4947e1fbb0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-16-5621a4ac88bf8cffea061cb7cb9f8d73 new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-17-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-18-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-19-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-2-190cefc93e46906e404039de0fd5f513 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-20-2a9cffbef1ebd3df8e4de4eb22777cf9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-21-3a4c24fd561f459025264baa3fb6d87 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-22-bf8877c86d4b4cd7da2939cdf517acc5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-23-3ffba3098571099bc2b13614ae3defc5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-24-c9f91a11d9be1e981e6ec284572bbb2a new file mode 100644 index 0000000000000..5e44ab6b5cef9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-25-d60297fed03b455c29daa4afb4d1e858 new file mode 100644 index 0000000000000..8cd9e4d2c201a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-3-25401dd2c1c258e06f1b96fefd19e27f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-4-41df7b4938cff8b7ebffc3f5c701dccf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-5-8026cdd2390eec2c72a0591ae5668185 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-6-3b57c0e3fccea5322373f3725c95ec00 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-7-777de794b7f27ea63f29a9784663393b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-8-6916eceaa04091d1453a7d0d5257213c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 b/sql/hive/src/test/resources/golden/merge_dynamic_partition3-9-8d0305d089aa5198601cc39073fff528 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 b/sql/hive/src/test/resources/golden/mergejoins-0-90c36ed2dea064c1951856a1a2cd3d38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b b/sql/hive/src/test/resources/golden/mergejoins-1-63de7fdfd7513d63a4eadafc8534f69b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c b/sql/hive/src/test/resources/golden/mergejoins-2-6b9f3810606db1e9036561f1173ac75c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins-3-c408f69470d652da283442a62b384e46 b/sql/hive/src/test/resources/golden/mergejoins-3-c408f69470d652da283442a62b384e46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 b/sql/hive/src/test/resources/golden/mergejoins-4-80b6c6ce31a4d4e26f6d4be49beae996 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e b/sql/hive/src/test/resources/golden/mergejoins-5-adae80fe415023783fca5499e3edf6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df b/sql/hive/src/test/resources/golden/mergejoins-6-6169410e9f077097d1a766724dfc51df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 b/sql/hive/src/test/resources/golden/mergejoins_mixed-0-7537f6537cfced8ba5bd203e3d1c8a87 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 b/sql/hive/src/test/resources/golden/mergejoins_mixed-1-bd83c916b6b06b6e813d7ecc6ada1d03 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 b/sql/hive/src/test/resources/golden/mergejoins_mixed-10-abb02d2cadc535ff51660d83e6609dc8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 b/sql/hive/src/test/resources/golden/mergejoins_mixed-11-725ed77dd110398f461926f190b4b5c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b b/sql/hive/src/test/resources/golden/mergejoins_mixed-12-b5d5c30fab0edd7cb5dc535000d20e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff b/sql/hive/src/test/resources/golden/mergejoins_mixed-2-de24edb80e51049d241fa0ce2e3165ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 b/sql/hive/src/test/resources/golden/mergejoins_mixed-3-5d4bbcec2f91b849642725fa843bf4e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 b/sql/hive/src/test/resources/golden/mergejoins_mixed-4-24a8048db8d561e28bcb4941498b9687 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 b/sql/hive/src/test/resources/golden/mergejoins_mixed-5-ab187bec261e23cca73a144d4ffcb4d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 b/sql/hive/src/test/resources/golden/mergejoins_mixed-6-e108b1560a601946194cecaf4da12491 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 b/sql/hive/src/test/resources/golden/mergejoins_mixed-7-d2068e6569b5d253932ce9d59be5221 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 b/sql/hive/src/test/resources/golden/mergejoins_mixed-8-bdba45fc8a7bd7af0a8f983160d550b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 b/sql/hive/src/test/resources/golden/mergejoins_mixed-9-b7a719a0596b89bf6c35b3dc5dfbe1e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d b/sql/hive/src/test/resources/golden/metadataonly1-0-f0c7b56d5ad529ae6c98875501a9548d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 b/sql/hive/src/test/resources/golden/metadataonly1-1-4d93504b19d34fd360de6af7fb1f3715 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d b/sql/hive/src/test/resources/golden/metadataonly1-10-1c7bb3877b2e1e7bb9166537715d442d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 b/sql/hive/src/test/resources/golden/metadataonly1-11-30df79b460916f3f67ccf1d7b7a076f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d b/sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/metadataonly1-2-9eadfd16be30c653625fce7b74048d9d @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 b/sql/hive/src/test/resources/golden/metadataonly1-3-7980a98d580a002b7ad7eef780039f67 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 b/sql/hive/src/test/resources/golden/metadataonly1-4-4d93504b19d34fd360de6af7fb1f3715 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d b/sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/metadataonly1-5-9eadfd16be30c653625fce7b74048d9d @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf b/sql/hive/src/test/resources/golden/metadataonly1-6-537256f669bc9101d4834df67aae8cdf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc b/sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/metadataonly1-7-92512b7ba2cb393d1335dcc2bcf5c2bc @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 b/sql/hive/src/test/resources/golden/metadataonly1-8-b140da3b2e7f4bdf101a8bfb0db88999 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 b/sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/metadataonly1-9-2c3b29a3df9b07e7add15e4e7c561644 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mi-0-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mi-1-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/mi-1-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mi-1-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e b/sql/hive/src/test/resources/golden/mi-2-abf8847fb25b96e0f9477808d8378e5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mi-3-b66a495f7bdf106a7886b72267b8659d b/sql/hive/src/test/resources/golden/mi-3-b66a495f7bdf106a7886b72267b8659d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c b/sql/hive/src/test/resources/golden/mi-4-ba6a06aef35cbd1e7c5a3c253f757b2c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc b/sql/hive/src/test/resources/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc new file mode 100644 index 0000000000000..3b011a048ba42 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mi-5-4aad9be12cbe4e7be5540924e6b5e7dc @@ -0,0 +1 @@ +ds=2008-04-08/hr=11 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 b/sql/hive/src/test/resources/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 new file mode 100644 index 0000000000000..0cdd3e8594c59 --- /dev/null +++ b/sql/hive/src/test/resources/golden/mi-6-b331d7aa963adac3b2e1de803f90e199 @@ -0,0 +1 @@ +ds=2008-04-08/hr=12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 b/sql/hive/src/test/resources/golden/mi-7-fca4e06de103c3cbb675fa43e7077800 new file mode 100644 index 0000000000000..f4026a591a958 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 b/sql/hive/src/test/resources/golden/mi-8-e946bdb81b0a831908c1c8196fdff215 new file mode 100644 index 0000000000000..f132aba9f3ee1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 b/sql/hive/src/test/resources/golden/multiMapJoin1-0-d2ea84f719d9ae2fb19e9e2a72c3d834 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a b/sql/hive/src/test/resources/golden/multiMapJoin1-1-10b9657a205f63e6127e29e8b477b30a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-10-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa b/sql/hive/src/test/resources/golden/multiMapJoin1-11-d814d6eb99b04dc19ae83dadf372c7aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 b/sql/hive/src/test/resources/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 new file mode 100644 index 0000000000000..55b22b1aa9f24 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-12-204073e1287b0582d50e652d466f1e66 @@ -0,0 +1 @@ +580 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-13-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-14-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 b/sql/hive/src/test/resources/golden/multiMapJoin1-15-4ab52efffec4d72a5b01bd253eeddcf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 b/sql/hive/src/test/resources/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 new file mode 100644 index 0000000000000..55b22b1aa9f24 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-16-c14b300770b329ecb71e0275c88532d3 @@ -0,0 +1 @@ +580 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 b/sql/hive/src/test/resources/golden/multiMapJoin1-17-17e0bad549b265f8237e6c539b848dd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee b/sql/hive/src/test/resources/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee new file mode 100644 index 0000000000000..fa8ef14301a13 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-18-49bc7f430b2591978067ca8f7d181cee @@ -0,0 +1,6 @@ +270 +10 +10 +270 +10 +10 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 b/sql/hive/src/test/resources/golden/multiMapJoin1-19-51acc9734833601aa37e8da9f3a06b91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 b/sql/hive/src/test/resources/golden/multiMapJoin1-2-feca2d1242cf09e54dc177881a708842 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 b/sql/hive/src/test/resources/golden/multiMapJoin1-20-6fb5b848305f559c6377cb7d34cc3216 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac b/sql/hive/src/test/resources/golden/multiMapJoin1-21-79973475ca07cb9932f752f6547779ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 b/sql/hive/src/test/resources/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-22-25e434b6d05e08fdd5f4d9957438917 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a b/sql/hive/src/test/resources/golden/multiMapJoin1-23-be8d26e5f6d5d41d5ce20b0ad443f0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 b/sql/hive/src/test/resources/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 new file mode 100644 index 0000000000000..a3a38a80b7910 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-24-feed626e3216bcbda66b17f48305b5a1 @@ -0,0 +1 @@ +247580 247580 247580 247580 247580 247580 548662743780 548662743780 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-25-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-26-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 b/sql/hive/src/test/resources/golden/multiMapJoin1-27-36bd62bd41b1a0dd13f12d8d813f1943 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 b/sql/hive/src/test/resources/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 new file mode 100644 index 0000000000000..a3a38a80b7910 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-28-feed626e3216bcbda66b17f48305b5a1 @@ -0,0 +1 @@ +247580 247580 247580 247580 247580 247580 548662743780 548662743780 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f b/sql/hive/src/test/resources/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-29-ea23403b9eb55e8b06d1c198e439569f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 b/sql/hive/src/test/resources/golden/multiMapJoin1-3-9280a4b7f1087dda801c81f3a5827785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 b/sql/hive/src/test/resources/golden/multiMapJoin1-30-d8c886592f60bedef5d8cb967adcead3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 b/sql/hive/src/test/resources/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 new file mode 100644 index 0000000000000..a3a38a80b7910 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-31-feed626e3216bcbda66b17f48305b5a1 @@ -0,0 +1 @@ +247580 247580 247580 247580 247580 247580 548662743780 548662743780 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 b/sql/hive/src/test/resources/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-32-e93301ee4ba157b466d7460775f3d350 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b b/sql/hive/src/test/resources/golden/multiMapJoin1-33-7be587cefa8323cbe42cbf469b998f7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 b/sql/hive/src/test/resources/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 new file mode 100644 index 0000000000000..a3a38a80b7910 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin1-34-feed626e3216bcbda66b17f48305b5a1 @@ -0,0 +1 @@ +247580 247580 247580 247580 247580 247580 548662743780 548662743780 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 b/sql/hive/src/test/resources/golden/multiMapJoin1-4-53b81d2e161acea2ee58b7ac849ffe48 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe b/sql/hive/src/test/resources/golden/multiMapJoin1-5-1e0b02b515a588ea99f6027f0aca36fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 b/sql/hive/src/test/resources/golden/multiMapJoin1-6-74628c956e66e192e0cfeb7bd09f8b73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba b/sql/hive/src/test/resources/golden/multiMapJoin1-7-23813f3b8b47a1f5c4a8ee57cc8a66ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c b/sql/hive/src/test/resources/golden/multiMapJoin1-8-273de5cc585b04ea2210d90c1755568c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 b/sql/hive/src/test/resources/golden/multiMapJoin1-9-3df5ed60d70bc1a826a0e32c8019dc49 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-1-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 b/sql/hive/src/test/resources/golden/multiMapJoin2-10-1905c7759350b107679aef86226739f8 new file mode 100644 index 0000000000000..3c505d9663010 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-11-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e b/sql/hive/src/test/resources/golden/multiMapJoin2-12-1d4f22cb25ffe3d84e73fe2acc81a92e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe b/sql/hive/src/test/resources/golden/multiMapJoin2-13-6b984427a771fe650fa875be98722cbe new file mode 100644 index 0000000000000..a877195ef3ab1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-14-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b b/sql/hive/src/test/resources/golden/multiMapJoin2-15-a4ac00f2a50cd08e7dd5543adb81972b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe b/sql/hive/src/test/resources/golden/multiMapJoin2-16-6b984427a771fe650fa875be98722cbe new file mode 100644 index 0000000000000..a877195ef3ab1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f b/sql/hive/src/test/resources/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-17-b9d963d24994c47c3776dda6f7d3881f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 b/sql/hive/src/test/resources/golden/multiMapJoin2-18-cb1f964731ee7ac045db89266a919586 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 b/sql/hive/src/test/resources/golden/multiMapJoin2-19-77324702b091d514ca16d029f65d3d56 new file mode 100644 index 0000000000000..a877195ef3ab1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 b/sql/hive/src/test/resources/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-2-7f7f3daa7ff45944c3d534f9feff3bb6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 b/sql/hive/src/test/resources/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-20-777edd9d575f3480ca6cebe4be57b1f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 b/sql/hive/src/test/resources/golden/multiMapJoin2-21-3db633aa7d2c47682bb15024d6abdd33 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 b/sql/hive/src/test/resources/golden/multiMapJoin2-22-77324702b091d514ca16d029f65d3d56 new file mode 100644 index 0000000000000..a877195ef3ab1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 b/sql/hive/src/test/resources/golden/multiMapJoin2-23-18139096bb78fa5080054686f27e5e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 b/sql/hive/src/test/resources/golden/multiMapJoin2-24-e148026f8994e22ca756c68753a0cc26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 b/sql/hive/src/test/resources/golden/multiMapJoin2-25-b04195464e014cb47fd20a76b5f9ac0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f b/sql/hive/src/test/resources/golden/multiMapJoin2-26-2136f3783a9764de762e49c1ca28637f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e b/sql/hive/src/test/resources/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e new file mode 100644 index 0000000000000..25bf17fc5aaab --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-27-d28d0f671f5d913a56d75812d24cca8e @@ -0,0 +1 @@ +18 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d b/sql/hive/src/test/resources/golden/multiMapJoin2-3-622f276b2eb5d55649a7a8689aacea5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 b/sql/hive/src/test/resources/golden/multiMapJoin2-4-5ede8243cc4ba2fbd24a77578502a656 new file mode 100644 index 0000000000000..8984af9eb83dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 b/sql/hive/src/test/resources/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-5-21f96f651fb4f28ae5e2999fc1f82192 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 b/sql/hive/src/test/resources/golden/multiMapJoin2-6-d7733ae25ad6fcb1bac1135271732502 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 b/sql/hive/src/test/resources/golden/multiMapJoin2-7-5ede8243cc4ba2fbd24a77578502a656 new file mode 100644 index 0000000000000..8984af9eb83dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 b/sql/hive/src/test/resources/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multiMapJoin2-8-7f7f3daa7ff45944c3d534f9feff3bb6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 b/sql/hive/src/test/resources/golden/multiMapJoin2-9-e14b355524f37fe25ebbb59b52e12c74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 b/sql/hive/src/test/resources/golden/multi_insert_gby-0-18cb60d5d7080d1eda9b830f50cfa782 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d b/sql/hive/src/test/resources/golden/multi_insert_gby-1-9e61989d717403353689cbbb2816210d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 b/sql/hive/src/test/resources/golden/multi_insert_gby-2-fdf89a7c60b9edcb2250efdfd1033a17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 b/sql/hive/src/test/resources/golden/multi_insert_gby-3-ed2c89fc9e4cfc07730c312b9e07d721 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby-4-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..0e3cc2ad6a4e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/multi_insert_gby-5-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c b/sql/hive/src/test/resources/golden/multi_insert_gby-6-b601e40a9c3bb8144a0447ec829ae49c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f b/sql/hive/src/test/resources/golden/multi_insert_gby-7-6c0583ab6194b67a00b19a8ec9deec5f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby-8-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..0e3cc2ad6a4e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/multi_insert_gby-9-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 b/sql/hive/src/test/resources/golden/multi_insert_gby2-0-b3ee4be40513342084411c5333416d69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 b/sql/hive/src/test/resources/golden/multi_insert_gby2-1-2fd65cd047d2295eadfc4935cf67ddf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 b/sql/hive/src/test/resources/golden/multi_insert_gby2-2-fc8c8df72e2136557a4cad9a78e921d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e b/sql/hive/src/test/resources/golden/multi_insert_gby2-3-538a02b95c066b307652c8d503470c8e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/multi_insert_gby2-4-521e0c1054cfa35116c02245874a4e69 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multi_insert_gby2-5-a9a491ed99b4629ee15bce994d5c6c63 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 b/sql/hive/src/test/resources/golden/multi_insert_gby3-0-60cf2dfb2a416f328a2fd58710eb6f01 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a b/sql/hive/src/test/resources/golden/multi_insert_gby3-1-cb42e494ade413e7bd1e426e5d6f60a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/multi_insert_gby3-10-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..e8804e62d0aaf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc b/sql/hive/src/test/resources/golden/multi_insert_gby3-11-717e2a1f7f5b7e8a0ef61cad13af4acc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f b/sql/hive/src/test/resources/golden/multi_insert_gby3-12-33d7e716735d24b7493209810d0b865f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 b/sql/hive/src/test/resources/golden/multi_insert_gby3-2-e2ee7089ea7db95d7bd86cae6f3f4bb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca b/sql/hive/src/test/resources/golden/multi_insert_gby3-3-5ae64ea7cfb7bef5a99d788016213fca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 b/sql/hive/src/test/resources/golden/multi_insert_gby3-4-6d713dc60fa4c07fb9de4a93db36fed0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 b/sql/hive/src/test/resources/golden/multi_insert_gby3-5-a66621daa1b2479beee5681a22d9d712 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby3-6-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..fc0b77c7a3628 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 b/sql/hive/src/test/resources/golden/multi_insert_gby3-7-a9a491ed99b4629ee15bce994d5c6c63 new file mode 100644 index 0000000000000..e8804e62d0aaf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b b/sql/hive/src/test/resources/golden/multi_insert_gby3-8-2ed91b92a6ca38b4ccb4acf52ee4e47b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 b/sql/hive/src/test/resources/golden/multi_insert_gby3-9-521e0c1054cfa35116c02245874a4e69 new file mode 100644 index 0000000000000..fc0b77c7a3628 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-0-531b7044d2fdaba4fff0094c4efdaf54 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-1-3ed6bfa23add415990b443fc232b4ba0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-10-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..310ad7306b634 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-11-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..afb670d7efab7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-12-622f0dd66c8633307fe56ccf9015f430 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-13-2c71748cfcdb9cc773d9ee61ae508b91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-14-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..310ad7306b634 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-15-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..f1441c3445d94 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 new file mode 100644 index 0000000000000..5421ac20e8621 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-16-33963ba7aefb7ab9a25afd540ecbbe98 @@ -0,0 +1,4 @@ +165 1 +27 1 +86 1 +98 1 diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-17-3219cf4fab8bf343bd273bd3a681fa46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-18-bfe7dcb00f8b27e00d406de603635c8a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-19-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..7dc40b688b128 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-2-6ced0ab5d543a2c5c6c7f7e27ed04814 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-20-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..77d4df18b3e16 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-21-33963ba7aefb7ab9a25afd540ecbbe98 new file mode 100644 index 0000000000000..2889b9a801049 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-22-283e0f384d0a015c252b34f79a895286 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-23-2f3b8b5fd961ee336d893cd45dc2696d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-24-4ed7df348bd8bf3fd275e15b6689c5df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-25-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..310ad7306b634 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-26-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..afb670d7efab7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-27-33963ba7aefb7ab9a25afd540ecbbe98 new file mode 100644 index 0000000000000..983e095ac80d6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-28-24e041343e158735db6262136de0e8b8 new file mode 100644 index 0000000000000..1bc1e319ea180 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-3-a60af91a18d481fe1244e21108133489 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-4-67a95497041a0e81b4d5756731d8b27d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-5-b847c5dfb2f0395cfdd21f93de611b91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-6-f26c10c6495fc3e86319cd5815caab4f new file mode 100644 index 0000000000000..acbb7cd281fb8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-7-6c6b1588ab62ad11aef3a1147a2c7874 new file mode 100644 index 0000000000000..4edc92defe127 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-8-a3eabeb4435c69ec4e242b8857c84e31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 b/sql/hive/src/test/resources/golden/multi_insert_lateral_view-9-59102a6d824074da6cc85ca760385975 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/multi_join_union-0-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b b/sql/hive/src/test/resources/golden/multi_join_union-1-af5c56bf13aaeff13d2bd7d3b9315d3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 b/sql/hive/src/test/resources/golden/multi_join_union-2-97ea2263579f8d63dfdb1a6992fe0284 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 b/sql/hive/src/test/resources/golden/multi_join_union-3-a164ae7bcb7c95beb8045b7064c9ab14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 b/sql/hive/src/test/resources/golden/multi_join_union-4-8c17422bd7041c596677251a46fa4085 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 b/sql/hive/src/test/resources/golden/multi_join_union-5-9e89b8619411f46f675a9fab73298627 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae b/sql/hive/src/test/resources/golden/multigroupby_singlemr-0-ae394b9d8cf6106b68fb2f40e8f1fae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 b/sql/hive/src/test/resources/golden/multigroupby_singlemr-1-e67c221c0c19df7b3dc48b99426cf667 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 b/sql/hive/src/test/resources/golden/multigroupby_singlemr-2-31e7358a9f1e474e67155396fe803967 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 b/sql/hive/src/test/resources/golden/multigroupby_singlemr-3-d676a455ef6e9ec68756b0c1487dc3b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 b/sql/hive/src/test/resources/golden/multigroupby_singlemr-4-13bade1e01aed5d7e1af33a28a3b3574 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 b/sql/hive/src/test/resources/golden/multigroupby_singlemr-5-c5f3f0a3ff1730f6daf04dcd78b74651 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 b/sql/hive/src/test/resources/golden/multigroupby_singlemr-6-80cb19f7a801a82add47a501756c8af7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 b/sql/hive/src/test/resources/golden/multigroupby_singlemr-7-f8a4f52cd86ee679923248e09dd312e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 b/sql/hive/src/test/resources/golden/multigroupby_singlemr-8-985e6ff98c02a64d47b25817e9fef310 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e b/sql/hive/src/test/resources/golden/multigroupby_singlemr-9-faa99964430e2aee836221de1abe1c5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 b/sql/hive/src/test/resources/golden/nested_complex-0-6a7c4841dab05ebae84309c9571bec6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 b/sql/hive/src/test/resources/golden/nested_complex-1-abac744dee1a4f4152781b1565fe1364 new file mode 100644 index 0000000000000..c10c17978a0d8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 b/sql/hive/src/test/resources/golden/nested_complex-2-f07e9814229ed440bd0ac7e4bd924ce9 new file mode 100644 index 0000000000000..8ce70d8c71782 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 b/sql/hive/src/test/resources/golden/nested_complex-3-fb014bc1e9bfd919a3703cf5afb77448 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d b/sql/hive/src/test/resources/golden/nested_complex-4-8fd07a5dec8db8e1618ab5b5f9c05c7d new file mode 100644 index 0000000000000..228853cffe527 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-0-e9a72fa546e90b8cbbd2e9470450cb56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-1-1ea9df2d7af3c79ebef07d6087c8106f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-10-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-11-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-12-e9a72fa546e90b8cbbd2e9470450cb56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-13-1ea9df2d7af3c79ebef07d6087c8106f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-14-67b834deba21676e02c155b25195a019 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-15-52f90e6bf3f2c17b82ed34318d2c612f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-16-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-17-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-2-67b834deba21676e02c155b25195a019 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-3-52f90e6bf3f2c17b82ed34318d2c612f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/nestedvirtual-4-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/nestedvirtual-5-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 b/sql/hive/src/test/resources/golden/nestedvirtual-6-e9a72fa546e90b8cbbd2e9470450cb56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f b/sql/hive/src/test/resources/golden/nestedvirtual-7-1ea9df2d7af3c79ebef07d6087c8106f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 b/sql/hive/src/test/resources/golden/nestedvirtual-8-67b834deba21676e02c155b25195a019 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f b/sql/hive/src/test/resources/golden/nestedvirtual-9-52f90e6bf3f2c17b82ed34318d2c612f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b b/sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/newline-0-43392a20a8d249a279d50d96578e6a1b @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505 b/sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/newline-1-a19a19272149c732977c37e043910505 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d b/sql/hive/src/test/resources/golden/newline-2-4eb54a664e549614d56ca088c8867d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 b/sql/hive/src/test/resources/golden/noalias_subq1-0-da2a1169620860b3dc558d21c6c5ea50 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 b/sql/hive/src/test/resources/golden/noalias_subq1-1-f91e87acd51f7477d96620b5f9deece6 new file mode 100644 index 0000000000000..cdcd12dca3ca2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b b/sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-0-535a12e87c72793bfba96520a0ea251b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 b/sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-1-8535e6c322e40f46b9a6e02fe6033ee0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 b/sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-2-c2b97a35777322fe4a08d8c2216cb223 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 b/sql/hive/src/test/resources/golden/nomore_ambiguous_table_col-3-29a7783c88e234ba32eaf1401ca8cc22 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-0-61c5ece0d210cf6158094f0f6fa24532 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-1-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-2-11ee085c0190a4007f410ddf96803ec0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-3-a873d2d6991308b21ecdc46ac777c716 new file mode 100644 index 0000000000000..a0cfcf2621f2e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-4-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-5-250d196b4449c835ddc518db2d2ab726 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 b/sql/hive/src/test/resources/golden/nonblock_op_deduplicate-6-a873d2d6991308b21ecdc46ac777c716 new file mode 100644 index 0000000000000..a0cfcf2621f2e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 b/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-0-6ed1b2ff177492c003161ee91e982c10 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 b/sql/hive/src/test/resources/golden/nonreserved_keywords_input37-1-e3ab2563222fb8678c7c269e09e1e8d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-0-d3afbe52e3a1eb246a79c1320f82c480 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-1-5c7fc72601c9add7bc86df7e4c24af63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-10-3c8f9dff0a12ca2b929d04b4873a4681 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 new file mode 100644 index 0000000000000..24e19ec6caa8f --- /dev/null +++ b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-11-6f59e35684a552a855e4dc3aee667092 @@ -0,0 +1 @@ +-826625916 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-12-d3afbe52e3a1eb246a79c1320f82c480 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-2-ef259cb012c8547e19dd4a75ac4f8ef5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-3-2983d09b973ea94bc701970a17fc3687 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 new file mode 100644 index 0000000000000..05b0c43f926bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-4-a2f6c6c77e94833197095dc48643f9c9 @@ -0,0 +1 @@ +10226524244 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-5-ef259cb012c8547e19dd4a75ac4f8ef5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-6-2983d09b973ea94bc701970a17fc3687 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 new file mode 100644 index 0000000000000..bc15b9ca54985 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-7-fe194a16b48b763e6efdf6fcc6116296 @@ -0,0 +1 @@ +20453048488 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad new file mode 100644 index 0000000000000..08839f6bb296e --- /dev/null +++ b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-8-ea9a965c3d3c8fb9271d8f7c5eee19ad @@ -0,0 +1 @@ +200 diff --git a/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 b/sql/hive/src/test/resources/golden/nonreserved_keywords_insert_into1-9-461110270cec5b6d012b2c685cf2cce9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d b/sql/hive/src/test/resources/golden/notable_alias1-0-695de796f21dce21056f8ba27cbadb0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd b/sql/hive/src/test/resources/golden/notable_alias1-1-c4d51044e6802df266aa0dc137b825cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e b/sql/hive/src/test/resources/golden/notable_alias1-2-bea59f268594a034d06b826c7a8e516e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/notable_alias1-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..fce122031e88f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d b/sql/hive/src/test/resources/golden/notable_alias2-0-695de796f21dce21056f8ba27cbadb0d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 b/sql/hive/src/test/resources/golden/notable_alias2-1-20b7bf01acc8d88670d347e6759aa407 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 b/sql/hive/src/test/resources/golden/notable_alias2-2-19c4fba994e06b28e0d912a6aa13ab78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/notable_alias2-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..fce122031e88f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 b/sql/hive/src/test/resources/golden/null_cast-0-48a80d4fdc8009234af31ebcb6e03364 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c b/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c new file mode 100644 index 0000000000000..d111428eaabb0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/null_cast-1-7257e6f8170e545962d27741353f672c @@ -0,0 +1 @@ +[null,0] [null,[]] [null,{}] [null,{"col1":0}] diff --git a/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 b/sql/hive/src/test/resources/golden/nullgroup-10-3994c1896dace613fa9f837b2f1676c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-11-54a5fd76cdeff565c8c7724695aca302 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-13-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 b/sql/hive/src/test/resources/golden/nullgroup-14-3994c1896dace613fa9f837b2f1676c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-15-54a5fd76cdeff565c8c7724695aca302 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 b/sql/hive/src/test/resources/golden/nullgroup-2-3994c1896dace613fa9f837b2f1676c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-3-54a5fd76cdeff565c8c7724695aca302 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-4-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-5-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 b/sql/hive/src/test/resources/golden/nullgroup-6-3994c1896dace613fa9f837b2f1676c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 b/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-7-54a5fd76cdeff565c8c7724695aca302 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup-9-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup2-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup2-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 b/sql/hive/src/test/resources/golden/nullgroup2-10-751fa56a198ad840868c76ad7ce2a6a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a b/sql/hive/src/test/resources/golden/nullgroup2-11-644b68261df70bbb46a3045a8abde17a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup2-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup2-13-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 b/sql/hive/src/test/resources/golden/nullgroup2-14-751fa56a198ad840868c76ad7ce2a6a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a b/sql/hive/src/test/resources/golden/nullgroup2-15-644b68261df70bbb46a3045a8abde17a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 b/sql/hive/src/test/resources/golden/nullgroup2-2-751fa56a198ad840868c76ad7ce2a6a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a b/sql/hive/src/test/resources/golden/nullgroup2-3-644b68261df70bbb46a3045a8abde17a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup2-4-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup2-5-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 b/sql/hive/src/test/resources/golden/nullgroup2-6-751fa56a198ad840868c76ad7ce2a6a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a b/sql/hive/src/test/resources/golden/nullgroup2-7-644b68261df70bbb46a3045a8abde17a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup2-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup2-9-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 b/sql/hive/src/test/resources/golden/nullgroup3-0-d7ed4ae23442da095677d751a2b86c99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 b/sql/hive/src/test/resources/golden/nullgroup3-1-a4d6d87d12f0e353d7a3ae5c087dab44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 b/sql/hive/src/test/resources/golden/nullgroup3-10-a6455ec10891deb352bca6ccab2a8bf8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 b/sql/hive/src/test/resources/golden/nullgroup3-11-d7ed4ae23442da095677d751a2b86c99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 b/sql/hive/src/test/resources/golden/nullgroup3-12-a4d6d87d12f0e353d7a3ae5c087dab44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb b/sql/hive/src/test/resources/golden/nullgroup3-13-f529994bef750d8edd02ede8a4323afb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e b/sql/hive/src/test/resources/golden/nullgroup3-14-d4e815f44f6369c991ea4390c481f31e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 b/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup3-15-cd90e16da918bac569e9f04faaedd280 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 b/sql/hive/src/test/resources/golden/nullgroup3-16-af767d7cfb4601ace72a3ed718071931 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 b/sql/hive/src/test/resources/golden/nullgroup3-17-45699aee54227552bb3ec84d92dfe450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 b/sql/hive/src/test/resources/golden/nullgroup3-18-a24e178327b39f57ca7dfa2e69296bb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 b/sql/hive/src/test/resources/golden/nullgroup3-19-b8f60d35c1c09c57efd4c339799e5404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb b/sql/hive/src/test/resources/golden/nullgroup3-2-f529994bef750d8edd02ede8a4323afb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 b/sql/hive/src/test/resources/golden/nullgroup3-20-29fb8e3c12b8a705bc67d55a10566141 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 b/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup3-21-e09c6bf2f6112981793fbd4386d43ff6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e b/sql/hive/src/test/resources/golden/nullgroup3-3-d4e815f44f6369c991ea4390c481f31e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 b/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup3-4-cd90e16da918bac569e9f04faaedd280 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 b/sql/hive/src/test/resources/golden/nullgroup3-5-45699aee54227552bb3ec84d92dfe450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 b/sql/hive/src/test/resources/golden/nullgroup3-6-a24e178327b39f57ca7dfa2e69296bb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 b/sql/hive/src/test/resources/golden/nullgroup3-7-b8f60d35c1c09c57efd4c339799e5404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 b/sql/hive/src/test/resources/golden/nullgroup3-8-29fb8e3c12b8a705bc67d55a10566141 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 b/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup3-9-e09c6bf2f6112981793fbd4386d43ff6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-1-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 b/sql/hive/src/test/resources/golden/nullgroup4-10-3a3f180144fead81b9e8b232b4132762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 b/sql/hive/src/test/resources/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 new file mode 100644 index 0000000000000..d05b1f927f4be --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-11-a37f94e6d2b4c99053d29a576ebaa56 @@ -0,0 +1 @@ +0 0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-12-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-13-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 b/sql/hive/src/test/resources/golden/nullgroup4-14-3a3f180144fead81b9e8b232b4132762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 b/sql/hive/src/test/resources/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 new file mode 100644 index 0000000000000..d05b1f927f4be --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-15-a37f94e6d2b4c99053d29a576ebaa56 @@ -0,0 +1 @@ +0 0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 b/sql/hive/src/test/resources/golden/nullgroup4-2-3a3f180144fead81b9e8b232b4132762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 b/sql/hive/src/test/resources/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 new file mode 100644 index 0000000000000..d05b1f927f4be --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-3-a37f94e6d2b4c99053d29a576ebaa56 @@ -0,0 +1 @@ +0 0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-4-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-5-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 b/sql/hive/src/test/resources/golden/nullgroup4-6-3a3f180144fead81b9e8b232b4132762 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 b/sql/hive/src/test/resources/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 new file mode 100644 index 0000000000000..d05b1f927f4be --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-7-a37f94e6d2b4c99053d29a576ebaa56 @@ -0,0 +1 @@ +0 0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-8-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4-9-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-0-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-1-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-2-43dec71d76c386394196a0e9d69457e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 new file mode 100644 index 0000000000000..06b63ea6c2f01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-3-c39b8f64123f56a05cdb0022a69f29b3 @@ -0,0 +1 @@ +0 0 0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-4-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-5-85c4f90b754cd88147d6b74e17d22063 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-6-43dec71d76c386394196a0e9d69457e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 new file mode 100644 index 0000000000000..06b63ea6c2f01 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullgroup4_multi_distinct-7-c39b8f64123f56a05cdb0022a69f29b3 @@ -0,0 +1 @@ +0 0 0 diff --git a/sql/hive/src/test/resources/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 b/sql/hive/src/test/resources/golden/nullgroup5-0-d7ed4ae23442da095677d751a2b86c99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b b/sql/hive/src/test/resources/golden/nullgroup5-1-c75bafea030f127bce9a4a5dbb14f40b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 b/sql/hive/src/test/resources/golden/nullgroup5-2-45699aee54227552bb3ec84d92dfe450 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c b/sql/hive/src/test/resources/golden/nullgroup5-3-4492a9ce0d8502584b872860d53c449c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e b/sql/hive/src/test/resources/golden/nullgroup5-4-14e2854b364b68e7526a52e6e8f7c94e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c b/sql/hive/src/test/resources/golden/nullgroup5-5-60d7180632a63c79eeba47e30b854f4c new file mode 100644 index 0000000000000..c5c8d29fdd13e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 b/sql/hive/src/test/resources/golden/nullinput-0-2be9511f8cb3c9edaf50353187f24c11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 b/sql/hive/src/test/resources/golden/nullinput-1-bf48040d6cc6213cba90dbd76a796a66 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf b/sql/hive/src/test/resources/golden/nullinput-2-61efe55ed8fef84e4cb8f9d7c317fabf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullinput2-0-40b73200f853b473855e85eba391f008 b/sql/hive/src/test/resources/golden/nullinput2-0-40b73200f853b473855e85eba391f008 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 b/sql/hive/src/test/resources/golden/nullinput2-1-507c5a854d930361d8db66a3b90a5388 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 b/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/nullinput2-2-21058230c4992a682c4adef9881fa9a2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/nullscript-0-4477b1566208a6805bb20630755a9375 b/sql/hive/src/test/resources/golden/nullscript-0-4477b1566208a6805bb20630755a9375 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 b/sql/hive/src/test/resources/golden/nullscript-1-3729d183a27e89a87ca9b9c9946e13a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 b/sql/hive/src/test/resources/golden/nullscript-2-17238164053203d56d30704e2c098e80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 b/sql/hive/src/test/resources/golden/nullscript-3-56d47cf9468111b12fcef408bc767271 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 b/sql/hive/src/test/resources/golden/nullscript-4-472199a0c6b8f760a90863deb69e9710 new file mode 100644 index 0000000000000..d23e05acf7ba5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc b/sql/hive/src/test/resources/golden/num_op_type_conv-0-c3f17f31d887479896bf454a2f6b15dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 b/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 new file mode 100644 index 0000000000000..da0ddb96957e1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/num_op_type_conv-1-aad06d01bc3c868b70dbebf88c7e64b8 @@ -0,0 +1 @@ +NULL NULL NULL 1 0 0.0 diff --git a/sql/hive/src/test/resources/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a b/sql/hive/src/test/resources/golden/optional_outer-0-c157cc7014eda416b6248cf56165b62a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 b/sql/hive/src/test/resources/golden/optional_outer-1-25aa8f4a1194304b0afc57a6b2c80205 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f b/sql/hive/src/test/resources/golden/optional_outer-2-7871ab752b7065f13fb808c8d7cdb35f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a b/sql/hive/src/test/resources/golden/optional_outer-3-4815a6c9014b80a79f61ab983dcdd23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 b/sql/hive/src/test/resources/golden/optional_outer-4-9519008c6ef6e8ed651c7f4405589c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e b/sql/hive/src/test/resources/golden/optional_outer-5-4a868da4f3f6d96124bff2cf50a2673e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-0-9e7ba3cead1b5040ee3c23e8fc235d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 b/sql/hive/src/test/resources/golden/orc_create-1-b5209e6f850fc958dc9ebced21519684 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d b/sql/hive/src/test/resources/golden/orc_create-10-3027edcda6a6030c6bff93681529c34d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-11-14ea7dcc6898979aaa61650e3fc46187 new file mode 100644 index 0000000000000..7541739b48608 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 b/sql/hive/src/test/resources/golden/orc_create-12-dd590639ea8d5b27ccf3312be2562fc2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-13-14ea7dcc6898979aaa61650e3fc46187 new file mode 100644 index 0000000000000..78c0010bebbc6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-14-9e7ba3cead1b5040ee3c23e8fc235d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 b/sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_create-15-e54b12bec275bc53ffe4b7252a99bab8 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d b/sql/hive/src/test/resources/golden/orc_create-16-3027edcda6a6030c6bff93681529c34d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb b/sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_create-17-f42f66edf8a9f6ffdbe83608cc353beb @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-18-14ea7dcc6898979aaa61650e3fc46187 new file mode 100644 index 0000000000000..6881640dd3cd4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 b/sql/hive/src/test/resources/golden/orc_create-19-a85f67347e5a19fc879ae83aa24c0ce4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 b/sql/hive/src/test/resources/golden/orc_create-2-f375f322df98df65e6b1bd0d4ce8e208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 b/sql/hive/src/test/resources/golden/orc_create-20-176d469a0edba57404416535c7d48023 new file mode 100644 index 0000000000000..dd4fa77f326ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 b/sql/hive/src/test/resources/golden/orc_create-21-e3accaf8ebc11bac7091db28d01fd2a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 b/sql/hive/src/test/resources/golden/orc_create-22-31944ad765bbf262b4ebafdb06df59a6 new file mode 100644 index 0000000000000..88c8812029d82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 b/sql/hive/src/test/resources/golden/orc_create-23-be779533ea8967231e644209114c8350 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 b/sql/hive/src/test/resources/golden/orc_create-24-fe59ff341395bc347dfb9cfdee397da2 new file mode 100644 index 0000000000000..67946888f1baf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead b/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead new file mode 100644 index 0000000000000..83db48f84ec87 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_create-25-c55e620d82921c40ebcdb94454ac1ead @@ -0,0 +1,3 @@ +line1 +line2 +line3 diff --git a/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c b/sql/hive/src/test/resources/golden/orc_create-26-aa4e0c8a3de340c1133d9fdddda8a18c new file mode 100644 index 0000000000000..db4876dd3c809 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 b/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 new file mode 100644 index 0000000000000..3b2e1cf7e0098 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_create-27-2eefc0bb4283fc255409a5c41203c89 @@ -0,0 +1,3 @@ +["a","b","c"] +["d","e","f"] +["g","h","i"] diff --git a/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 b/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 new file mode 100644 index 0000000000000..0f890dfd226d6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_create-28-cf8aa1014707dfe576820041e47436e2 @@ -0,0 +1,3 @@ +{"a":"one","b":"two"} +{"a":"three","b":"four"} +{"a":"five","b":"six"} diff --git a/sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 b/sql/hive/src/test/resources/golden/orc_create-29-cee6b57822d79ce80e14e58663bf7b86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae b/sql/hive/src/test/resources/golden/orc_create-3-8480c37b4f3a6768f459361cf8470dae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d b/sql/hive/src/test/resources/golden/orc_create-30-b5e831bf4eb811ac4f06e9ac11a5283d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 b/sql/hive/src/test/resources/golden/orc_create-31-c934a1ca351b76c5c4ef87cb5655f1d3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_create-32-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 b/sql/hive/src/test/resources/golden/orc_create-33-2151d07a07ab6b28782b7e8e4f848e36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_create-34-3b03210f94ec40db9ab02620645014d1 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 b/sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_create-35-a7bdd6d4d98ead017c08349ee53e9ac2 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a b/sql/hive/src/test/resources/golden/orc_create-36-ca15a32658195ecaffe11d11f487fb0a new file mode 100644 index 0000000000000..0c6f532488607 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-37-9e7ba3cead1b5040ee3c23e8fc235d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 b/sql/hive/src/test/resources/golden/orc_create-38-b5209e6f850fc958dc9ebced21519684 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 b/sql/hive/src/test/resources/golden/orc_create-39-f375f322df98df65e6b1bd0d4ce8e208 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/sql/hive/src/test/resources/golden/orc_create-4-7a04a1c1fe76e48f3dc8ee07146cd1e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae b/sql/hive/src/test/resources/golden/orc_create-40-8480c37b4f3a6768f459361cf8470dae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 b/sql/hive/src/test/resources/golden/orc_create-41-7a04a1c1fe76e48f3dc8ee07146cd1e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 b/sql/hive/src/test/resources/golden/orc_create-5-4f8e2d1f18399679a97c49631c4ef921 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 b/sql/hive/src/test/resources/golden/orc_create-6-2ae0fd655aa777b41015e4125f680324 new file mode 100644 index 0000000000000..ecd0704ce3386 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 b/sql/hive/src/test/resources/golden/orc_create-7-27aa4a8093e80a7437266f349ea927c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 b/sql/hive/src/test/resources/golden/orc_create-8-14ea7dcc6898979aaa61650e3fc46187 new file mode 100644 index 0000000000000..6881640dd3cd4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 b/sql/hive/src/test/resources/golden/orc_create-9-9e7ba3cead1b5040ee3c23e8fc235d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-0-a399c0ef0f1578f9d2456727008dee16 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-1-a2f8227aafaee48079235d466c5049a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-2-939fd158d52e04da7ff5c13e74f51a8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-3-1a7f96f49e2992b93d97edcbdb0075d new file mode 100644 index 0000000000000..f799fca27c353 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-4-f3df937417e80909d4a4b220779173b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-5-f5387ae613d41813186d7f1399545b02 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-6-272f5d299289829dc22cc31f70115dd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-7-acf39b28d4b76382acf5c56c21932ff9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-8-fe11a21ffbfa65e71ab62263077192a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 new file mode 100644 index 0000000000000..f953881c894c1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_dictionary_threshold-9-f7b722063a6948d22aaaab0707cddde1 @@ -0,0 +1 @@ +1082202951192 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-0-bfdd54175515a0557f8bd427ec23c453 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_diff_part_cols-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-2-ab12575b9b663420d9b3702f5a32a086 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-3-50ef26d05e69c02cd6fc2423fde00cd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e b/sql/hive/src/test/resources/golden/orc_diff_part_cols-4-1c1eab8fc34159875afe38eb2413434e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 b/sql/hive/src/test/resources/golden/orc_diff_part_cols-5-c0dce07a38f3c540c2b17db0a495a657 new file mode 100644 index 0000000000000..f172be3e72712 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c b/sql/hive/src/test/resources/golden/orc_empty_files-0-eedb40290338ab680a930542cc7ddf0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_empty_files-1-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_empty_files-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_empty_files-3-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 b/sql/hive/src/test/resources/golden/orc_empty_files-4-9c1451024d868c99833bbe1173c703d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 b/sql/hive/src/test/resources/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_empty_files-5-8c5d1eb83ebab33e284d70b11c4bc722 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 b/sql/hive/src/test/resources/golden/orc_empty_strings-0-a338239d2fc4e22e23ab82fa5f62139 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf b/sql/hive/src/test/resources/golden/orc_empty_strings-1-a67f0c90e3f99257a7364cc96e5405cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 b/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 new file mode 100644 index 0000000000000..e3c0074c9d4f6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_empty_strings-2-a34bd419a63852c1c75f195a495ff333 @@ -0,0 +1,9 @@ + + + + + + + + + diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 b/sql/hive/src/test/resources/golden/orc_empty_strings-3-3339ace17de3201296847caf29c42e99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e b/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e new file mode 100644 index 0000000000000..4cd1242d9fe61 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_empty_strings-4-eee6b7f3a881c8f56510bbd53aeebe1e @@ -0,0 +1,10 @@ + + + + + + + +165 +255 +27 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-0-a338239d2fc4e22e23ab82fa5f62139 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-1-2c16215823a5e6904059a48a3077da4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-2-2ffa91c54b8de552e8010bf00ecd2d43 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-3-bbe8d353c397b795e2732bd59648d291 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-4-a0231a52d63534679bfab3f6a0b8effc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f new file mode 100644 index 0000000000000..e7700b8aee5c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/orc_ends_with_nulls-5-4d467d3e186c59dee2c93c940a7b0e7f @@ -0,0 +1,5 @@ +1 +NULL +NULL +NULL +NULL diff --git a/sql/hive/src/test/resources/golden/order-0-5c2fda577771db3b316e0d2bd02d048a b/sql/hive/src/test/resources/golden/order-0-5c2fda577771db3b316e0d2bd02d048a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 b/sql/hive/src/test/resources/golden/order-1-57d93bd7619dfc460dfa763c12142bb9 new file mode 100644 index 0000000000000..217a1915f8826 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf b/sql/hive/src/test/resources/golden/order-2-fe90320d98850ea5e9b6100f6d259fbf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 b/sql/hive/src/test/resources/golden/order-3-8f2ca0c3a07d78ebfff23a7bc77e85e5 new file mode 100644 index 0000000000000..ab34c87e42364 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/order2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf b/sql/hive/src/test/resources/golden/order2-1-484c94d6b57ccc9fffda150bee0bccdf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f b/sql/hive/src/test/resources/golden/order2-2-4162aa366dc0836eed15cc819226907f new file mode 100644 index 0000000000000..217a1915f8826 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/outer_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c b/sql/hive/src/test/resources/golden/outer_join_ppr-1-1a374dec627d7109276f008f31be517c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 b/sql/hive/src/test/resources/golden/outer_join_ppr-2-b60ab17f7372863504804717c4276595 new file mode 100644 index 0000000000000..549da558b18da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba b/sql/hive/src/test/resources/golden/outer_join_ppr-3-5924d0d5ab868c05f488ed741a1955ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 b/sql/hive/src/test/resources/golden/outer_join_ppr-4-be64a45a645ea5363109323a378d335 new file mode 100644 index 0000000000000..549da558b18da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 b/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel-0-23a4feaede17467a8cc26e4d86ec30f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 b/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel-1-851e262128626126ae1ad87869db7c54 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 b/sql/hive/src/test/resources/golden/parallel-10-d196279de0b7c4b42521aa23634ca506 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a b/sql/hive/src/test/resources/golden/parallel-11-6230286bc168af7b010968b543690a2a new file mode 100644 index 0000000000000..c912af4a5c676 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 b/sql/hive/src/test/resources/golden/parallel-12-73a915d42e62c0e895a82602a502ee43 new file mode 100644 index 0000000000000..c912af4a5c676 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel-2-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a b/sql/hive/src/test/resources/golden/parallel-3-195007722f0c2921f9c6c1eb461b6d2a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 b/sql/hive/src/test/resources/golden/parallel-4-fcf032bbe1f1569d934da7090db60a83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel-5-e27b511ffc5828b09069121031d17a2f b/sql/hive/src/test/resources/golden/parallel-5-e27b511ffc5828b09069121031d17a2f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 b/sql/hive/src/test/resources/golden/parallel-6-d196279de0b7c4b42521aa23634ca506 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a b/sql/hive/src/test/resources/golden/parallel-7-6230286bc168af7b010968b543690a2a new file mode 100644 index 0000000000000..c912af4a5c676 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 b/sql/hive/src/test/resources/golden/parallel-8-73a915d42e62c0e895a82602a502ee43 new file mode 100644 index 0000000000000..c912af4a5c676 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/parallel-9-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 b/sql/hive/src/test/resources/golden/parenthesis_star_by-0-57a4ea931689f9475b687292f34abfa4 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 b/sql/hive/src/test/resources/golden/parenthesis_star_by-1-41d474f5e6d7c61c36f74b4bec4e9e44 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc b/sql/hive/src/test/resources/golden/parenthesis_star_by-2-7a45831bf96814d9a7fc3d78fb7bd8dc new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd b/sql/hive/src/test/resources/golden/parenthesis_star_by-3-498e2973594ccf45448ba19552bfb1cd new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec b/sql/hive/src/test/resources/golden/parenthesis_star_by-4-dc9d9d8bbb6259c89a97065902a345ec new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 b/sql/hive/src/test/resources/golden/parenthesis_star_by-5-6888c7f7894910538d82eefa23443189 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 b/sql/hive/src/test/resources/golden/parenthesis_star_by-6-6b7a4fa7be24cf775fae1b8c540c3b02 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea b/sql/hive/src/test/resources/golden/parenthesis_star_by-7-f36c71c612ab8da4f4191980c1b61fea new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 b/sql/hive/src/test/resources/golden/partInit-0-aaa3124841a8140a41d3556a4ccaa6a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b b/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partInit-1-c0ec92801bec7ece0a156d407b601f7b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 b/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partInit-2-5db6fb2267a648ac57af8f56f91cf9a2 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 b/sql/hive/src/test/resources/golden/partInit-3-878a82ca7519e3eafd3f2661b2ac1b88 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 b/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partInit-4-5db6fb2267a648ac57af8f56f91cf9a2 @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-0-8ea0fbf5d1f4c19b56bda7f05764a0c0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-1-797247d3ab7d2f3cd6fb33ad57ac7298 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-2-7bf3b5a09ed6cd06f27a0616de64ceb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-3-7e6487ca8473fa3264fdd9fa3e4a2db2 new file mode 100644 index 0000000000000..ffa0de8a0c322 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props-4-c04c695a6ebed215889ce75edcb33eb4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-0-c367ba7f534037ab96efc7f2273508c7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-1-d697ec36ecf73b0ee789972e6980e460 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-2-7bf3b5a09ed6cd06f27a0616de64ceb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_empty-3-7e6487ca8473fa3264fdd9fa3e4a2db2 new file mode 100644 index 0000000000000..0909bab11a6ec --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-0-9131f0b22a7303a9f3bd9ec0d1c85b06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-1-797247d3ab7d2f3cd6fb33ad57ac7298 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-2-7bf3b5a09ed6cd06f27a0616de64ceb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-3-7e6487ca8473fa3264fdd9fa3e4a2db2 new file mode 100644 index 0000000000000..55fb6df62f0a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/part_inherit_tbl_props_with_star-4-c04c695a6ebed215889ce75edcb33eb4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a b/sql/hive/src/test/resources/golden/partcols1-0-e234d14f9b1beb190c3c13d22ff02a0a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 b/sql/hive/src/test/resources/golden/partcols1-1-5ea0e79c5da4b87d90ab30a56f3598d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 b/sql/hive/src/test/resources/golden/partcols1-2-1fac765c5b4e1a8e4358f3babafd14e5 new file mode 100644 index 0000000000000..8fb73f609efce --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 b/sql/hive/src/test/resources/golden/partition_decode_name-0-6ae3e0805cbfa967f50ad222bc33b772 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 b/sql/hive/src/test/resources/golden/partition_decode_name-1-4de8e998198c8df484e9102f60ba05c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_decode_name-10-f1c90c8f4afd0c2ee66019e017997eb5 new file mode 100644 index 0000000000000..81df179618406 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_decode_name-11-94d637f6e5cee2771b9844438008a618 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_decode_name-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_decode_name-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 b/sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_decode_name-4-e90740a9a52c37a964ee204379f01412 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_decode_name-5-a3e8e57df86f00101d2f4dd0ce8c2bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_decode_name-6-f1c90c8f4afd0c2ee66019e017997eb5 new file mode 100644 index 0000000000000..e16053e3110ae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_decode_name-7-94d637f6e5cee2771b9844438008a618 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 b/sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_decode_name-8-bff58433eb2b500bb1d2f6ea495a5f20 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_decode_name-9-a3e8e57df86f00101d2f4dd0ce8c2bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c b/sql/hive/src/test/resources/golden/partition_schema1-0-3fc0ef3eda4a7269f205ce0203b56b0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 b/sql/hive/src/test/resources/golden/partition_schema1-1-3d21fcf667e5b0ef9e2ec0a1d502f915 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 b/sql/hive/src/test/resources/golden/partition_schema1-2-4fcfc1d26e1de1ce3071f1f93c012988 new file mode 100644 index 0000000000000..8af58e2b29f4a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb b/sql/hive/src/test/resources/golden/partition_schema1-3-fdef2e7e9e40868305d21c1b0df019bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 b/sql/hive/src/test/resources/golden/partition_schema1-4-9b756f83973c37236e72f6927b1c02d7 new file mode 100644 index 0000000000000..82a07522b0af0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 b/sql/hive/src/test/resources/golden/partition_schema1-5-52a518a4f7132598998c4f6781fd7634 new file mode 100644 index 0000000000000..8af58e2b29f4a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d b/sql/hive/src/test/resources/golden/partition_serde_format-0-65b98f7ed3ca5907e7ca5206de94939d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 b/sql/hive/src/test/resources/golden/partition_serde_format-1-37d3a88b6b22de326dbc1f4cba6b7cd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf b/sql/hive/src/test/resources/golden/partition_serde_format-2-47b559b01e389cc9e327a2fd29255acf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 b/sql/hive/src/test/resources/golden/partition_serde_format-3-54d18742b4eab85edd1946ef139771b4 new file mode 100644 index 0000000000000..95f5492558a9b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 b/sql/hive/src/test/resources/golden/partition_special_char-0-6ae3e0805cbfa967f50ad222bc33b772 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 b/sql/hive/src/test/resources/golden/partition_special_char-1-4de8e998198c8df484e9102f60ba05c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_special_char-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_special_char-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_special_char-4-a3e8e57df86f00101d2f4dd0ce8c2bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_special_char-5-f1c90c8f4afd0c2ee66019e017997eb5 new file mode 100644 index 0000000000000..e16053e3110ae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_special_char-6-94d637f6e5cee2771b9844438008a618 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 b/sql/hive/src/test/resources/golden/partition_special_char-7-a3e8e57df86f00101d2f4dd0ce8c2bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 b/sql/hive/src/test/resources/golden/partition_special_char-8-f1c90c8f4afd0c2ee66019e017997eb5 new file mode 100644 index 0000000000000..e16053e3110ae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 b/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 new file mode 100644 index 0000000000000..00750edc07d64 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_special_char-9-94d637f6e5cee2771b9844438008a618 @@ -0,0 +1 @@ +3 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c b/sql/hive/src/test/resources/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_type_check-0-b500f9a7ff3ef4ea3046cbaee22f434c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa b/sql/hive/src/test/resources/golden/partition_type_check-1-e676cfebd53fcc4c86407028dff8ddaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc b/sql/hive/src/test/resources/golden/partition_type_check-10-d51791c8b809ac86dc5b0f493a938fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a b/sql/hive/src/test/resources/golden/partition_type_check-11-a1164f1770d2f787b520fbc3d345911a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f b/sql/hive/src/test/resources/golden/partition_type_check-2-5e857e1536264658caf0df9b7e28652f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 b/sql/hive/src/test/resources/golden/partition_type_check-3-7880e06d5cff6ea961dba5ff533cf346 new file mode 100644 index 0000000000000..c4f5a8a640fd8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d b/sql/hive/src/test/resources/golden/partition_type_check-4-45fb706ff448da1fe609c7ff76a80d4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 b/sql/hive/src/test/resources/golden/partition_type_check-5-4a0b90a674d255ce00dd417dfefc46d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 b/sql/hive/src/test/resources/golden/partition_type_check-6-1f5f161f5e992a6aa7358dcbbffc5939 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 b/sql/hive/src/test/resources/golden/partition_type_check-7-7880e06d5cff6ea961dba5ff533cf346 new file mode 100644 index 0000000000000..c4f5a8a640fd8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d b/sql/hive/src/test/resources/golden/partition_type_check-8-45fb706ff448da1fe609c7ff76a80d4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 b/sql/hive/src/test/resources/golden/partition_type_check-9-57b300095c52fd652e1d414724523587 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 b/sql/hive/src/test/resources/golden/partition_varchar1-0-ec359154c4bad3bfce1b7ab4914d1554 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 b/sql/hive/src/test/resources/golden/partition_varchar1-1-5064ea12af064b5e040d7b9ebd08b940 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 b/sql/hive/src/test/resources/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 new file mode 100644 index 0000000000000..b4de394767536 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-10-1839df813809f21d8e0a0dd0006c7eb1 @@ -0,0 +1 @@ +11 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 b/sql/hive/src/test/resources/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 new file mode 100644 index 0000000000000..64bb6b746dcea --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-11-83bf857460d5f49bf4bf6e480d9ce36 @@ -0,0 +1 @@ +30 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 b/sql/hive/src/test/resources/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-12-299a7b630adb14d0f6a8ea8f2e528489 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c b/sql/hive/src/test/resources/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-13-4b9a64b41647af09b2f420c3b23b811c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 b/sql/hive/src/test/resources/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-14-c07f977014280e92c02f24bb3ea0ec68 @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 b/sql/hive/src/test/resources/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-15-6442628796f33b87ef1ca11945fa4b48 @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d b/sql/hive/src/test/resources/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-16-3300093c71a371aed6225ffa9e05fc3d @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 b/sql/hive/src/test/resources/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 new file mode 100644 index 0000000000000..f599e28b8ab0d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-17-367ed375dd11bf57d5ab4288289a11d2 @@ -0,0 +1 @@ +10 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a b/sql/hive/src/test/resources/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a new file mode 100644 index 0000000000000..209e3ef4b6247 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-18-3df8ffe4c220764e59efb874bab97b9a @@ -0,0 +1 @@ +20 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 b/sql/hive/src/test/resources/golden/partition_varchar1-19-ec359154c4bad3bfce1b7ab4914d1554 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 b/sql/hive/src/test/resources/golden/partition_varchar1-2-fc9381cdfb786fc3b7b9b35df6003474 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 b/sql/hive/src/test/resources/golden/partition_varchar1-3-968759281b7d1bf0a60991ed04953b93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef b/sql/hive/src/test/resources/golden/partition_varchar1-4-deb9b7715610152bda285a3a33b772ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce b/sql/hive/src/test/resources/golden/partition_varchar1-5-f77df9235261b945ca0a55e1afe2e5ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 b/sql/hive/src/test/resources/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 new file mode 100644 index 0000000000000..051ca3d3c28e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-6-ad21f89ac813692cf47343c66e302ea6 @@ -0,0 +1,2 @@ +2000-01-01 +2013-08-08 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e b/sql/hive/src/test/resources/golden/partition_varchar1-7-be5b3ad809173ec5dab8fb3d29b6cb0e new file mode 100644 index 0000000000000..24192eefd2caf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b b/sql/hive/src/test/resources/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b new file mode 100644 index 0000000000000..60d3b2f4a4cd5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-8-6b968247a2c615af03fc6a386d72db9b @@ -0,0 +1 @@ +15 diff --git a/sql/hive/src/test/resources/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 b/sql/hive/src/test/resources/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_varchar1-9-45412c7f1eb448dfd5d5eb636fa1d8a0 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-0-c25482149887659ec66d7534cad22f63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-1-29f193b3e8def419733366c578e8a236 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-2-14033db9e60aa0bc4f64376c96195adf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-3-9bdf636332ca722093413523366efa86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde b/sql/hive/src/test/resources/golden/partition_vs_table_metadata-4-e784348b7221bb26830cc1ebda69bdde new file mode 100644 index 0000000000000..39c80f1b77eab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-0-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-1-9fad934993b77eef15a5d10eb203a378 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-10-f89a3f7c0ee5bc3f6bd04aa0b459dd4a new file mode 100644 index 0000000000000..dcf40c875b9ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-11-606ad10de7caf7e65e09778f2673e712 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-12-c6bf6ee8fdc50f2abb3a71305d1e6882 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-13-c262e8c736b67119b9806f69eb492ef3 new file mode 100644 index 0000000000000..9639a1e84c615 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat-14-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat-15-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-16-7d619408a560b5c8d4a06dcd0ee106e5 new file mode 100644 index 0000000000000..707036ebbc76c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-17-a488cb92e1388a7542d8787a22f4fb55 new file mode 100644 index 0000000000000..06316a924c38b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-18-f89a3f7c0ee5bc3f6bd04aa0b459dd4a new file mode 100644 index 0000000000000..735d41e54e958 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-19-e3b55daf170a032dcc8ed12ee26ccf63 new file mode 100644 index 0000000000000..27dfc622c9bb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-2-7d619408a560b5c8d4a06dcd0ee106e5 new file mode 100644 index 0000000000000..1812e0d53e443 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-20-606ad10de7caf7e65e09778f2673e712 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-21-c6bf6ee8fdc50f2abb3a71305d1e6882 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-22-c56c391659f5701a9d2e8782a60f7f8a new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-23-c262e8c736b67119b9806f69eb492ef3 new file mode 100644 index 0000000000000..fca99e91bbd8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f b/sql/hive/src/test/resources/golden/partition_wise_fileformat-24-fe564b4f68241ec5c5884e44a1306f4f new file mode 100644 index 0000000000000..fca99e91bbd8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-3-a488cb92e1388a7542d8787a22f4fb55 new file mode 100644 index 0000000000000..3532257511613 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-4-606ad10de7caf7e65e09778f2673e712 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-5-c262e8c736b67119b9806f69eb492ef3 new file mode 100644 index 0000000000000..9b9389353dd5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-6-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat-7-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-8-7d619408a560b5c8d4a06dcd0ee106e5 new file mode 100644 index 0000000000000..a02458b88bf52 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 b/sql/hive/src/test/resources/golden/partition_wise_fileformat-9-a488cb92e1388a7542d8787a22f4fb55 new file mode 100644 index 0000000000000..301bff44316ff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-1-198cb7d650a506ec3420b94b82a01375 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-2-f723dedd396bd468107755b4495c1031 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-3-c278fd699aa25809bdef310fb92f510e new file mode 100644 index 0000000000000..001841c8a1cd4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-4-620cb6a290ef5297ac1d3a9ea776e2bf new file mode 100644 index 0000000000000..63f35fd827de3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-5-dd4c3f1636af9a7371edf7142abee088 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-6-620cb6a290ef5297ac1d3a9ea776e2bf new file mode 100644 index 0000000000000..63f35fd827de3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e b/sql/hive/src/test/resources/golden/partition_wise_fileformat10-7-c278fd699aa25809bdef310fb92f510e new file mode 100644 index 0000000000000..f75126345f351 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-1-b30db33650de0545cbcd53769ed7cf40 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-10-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..11542e84a3768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-11-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-2-92bfcf88ca528eb6c9259142bf6541e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-3-61f1abcdc66a64c11df85dded920d167 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-4-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..53aab40f88b50 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-5-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-6-ee7394c912991b8cd4401fb94942351f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-7-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-8-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat11-9-ed1e2dbef8eb8efbf83a50dc646485ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-1-b30db33650de0545cbcd53769ed7cf40 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-10-ed1e2dbef8eb8efbf83a50dc646485ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-11-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..1b97612ce3f5a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-12-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..f8e13d5235028 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-13-26a9a9cafa15d0223b934eba4777aea7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-14-d35f445501407d6fae5c3ad161fb2236 new file mode 100644 index 0000000000000..919a66a94c5cb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-15-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..45c169f0d8330 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-2-92bfcf88ca528eb6c9259142bf6541e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-3-61f1abcdc66a64c11df85dded920d167 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-4-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..53aab40f88b50 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-5-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-6-ee7394c912991b8cd4401fb94942351f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-7-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-8-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d b/sql/hive/src/test/resources/golden/partition_wise_fileformat12-9-b8aed3dd7d7c151f5d96050c4058e71d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-1-9a812f44c4c7a4c72b4be95139e6fd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 new file mode 100644 index 0000000000000..b6a7d89c68e0c --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-10-5bc619cec1d519c1dc6423f841b1c0a4 @@ -0,0 +1 @@ +16 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-2-3b6e28e3c1c071583e9b3b8f1f997f75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-3-9b3e4a236550f1900a36566830b24024 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-4-49cf189a09e11c2c635fbb574b89a2af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-5-2ebe5fe98e830143b9571b13fe59dc0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-6-fa2f56078df18add8e5a77e538756488 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-7-1d822cc037d9281ce172e2d5685b1495 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-8-e4531456a7418952ec1d69e18bc8460b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d new file mode 100644 index 0000000000000..b6a7d89c68e0c --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat13-9-8f6983fda271fba18d218c75b8fb2b3d @@ -0,0 +1 @@ +16 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-10-e5e7913d0875ad01f5d6f5031e86f0a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-11-bbfb0a95274044dea4732e35c1d7ecbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 new file mode 100644 index 0000000000000..425151f3a411f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-12-be84e8199b0a3b9f72e552018854ac15 @@ -0,0 +1 @@ +40 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-13-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-14-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 new file mode 100644 index 0000000000000..425151f3a411f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-15-dd6e9965d271bd35604059540c23d967 @@ -0,0 +1 @@ +40 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-16-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 new file mode 100644 index 0000000000000..425151f3a411f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-17-f4a3e39d5df18861e9fe67ef37af57e1 @@ -0,0 +1 @@ +40 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 new file mode 100644 index 0000000000000..c739b42c4d2ce --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-18-58080916a9f5883121bcaad719be0309 @@ -0,0 +1 @@ +44 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-3-2683f9835169a568c1f03dae859d27d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-4-2d2e1149d2f035017bb37bbfaad4def0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-5-61a8225e20f36e286b4d02501d2c80d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-6-b5165befb75ebeed42f3e69d4d64375c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-7-e438f7ec386b2ab19660e9da7260dd95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-8-f0e3ddaa1e6ea067444b1f540bfac293 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b b/sql/hive/src/test/resources/golden/partition_wise_fileformat14-9-adeaa033260f16c5bc106e761e5fef8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-1-e1cf6c355de3ae8db7564b1676199117 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..11542e84a3768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-10-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-11-662bece7f71ef854ca6e25f0eef5b830 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-12-8250422b5ed16120ee33bd4fa69d3f47 new file mode 100644 index 0000000000000..025abe4ec330b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-13-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..f067c1ed602dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-2-107d7c681b43611df056238be242127b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-3-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..53aab40f88b50 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-4-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-5-ee7394c912991b8cd4401fb94942351f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-6-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-7-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-8-ed1e2dbef8eb8efbf83a50dc646485ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat15-9-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-1-5b9fe6eb0e500ae6bf97e2a34d3d0ad9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..11542e84a3768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-10-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 NULL 1 +238 val_238 NULL 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-11-662bece7f71ef854ca6e25f0eef5b830 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-12-8250422b5ed16120ee33bd4fa69d3f47 new file mode 100644 index 0000000000000..025abe4ec330b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-13-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..f067c1ed602dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-2-107d7c681b43611df056238be242127b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-3-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..53aab40f88b50 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-4-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476.0 val_238 +476.0 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-5-ee7394c912991b8cd4401fb94942351f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-6-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd new file mode 100644 index 0000000000000..6d512a4f0bdc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-7-3fc394a7a3d43a70968282ef1ee21dbd @@ -0,0 +1,2 @@ +238 val_238 1 +238 val_238 1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-8-ed1e2dbef8eb8efbf83a50dc646485ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b new file mode 100644 index 0000000000000..de31196d97c3f --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat16-9-75a0aa2df39cbdc6a470b4c88803b42b @@ -0,0 +1,2 @@ +476 val_238 +476 val_238 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 b/sql/hive/src/test/resources/golden/partition_wise_fileformat17-0-7c49277a7217a147685d30e27822d273 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 b/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat17-1-df4a3bf439eadc5ada3303a9b5e6aa46 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-0-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-1-9fad934993b77eef15a5d10eb203a378 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-10-561ef0dbccfcbfbb0c75f33ebfd4203d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-11-20a02894f5e9340e89b55a30bef252b7 new file mode 100644 index 0000000000000..325e26b3d9737 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-2-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-3-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-4-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-5-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-6-32b36a6c3344c5fcca0ad6c93ffcab62 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-7-561ef0dbccfcbfbb0c75f33ebfd4203d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-8-20a02894f5e9340e89b55a30bef252b7 new file mode 100644 index 0000000000000..325e26b3d9737 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat2-9-50131c0ba7b7a6b65c789a5a8497bada @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-0-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-1-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-10-c6bf6ee8fdc50f2abb3a71305d1e6882 new file mode 100644 index 0000000000000..79c8f8e614a1f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-2-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-3-f89a3f7c0ee5bc3f6bd04aa0b459dd4a new file mode 100644 index 0000000000000..f487b340cd55b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-4-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-5-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-6-e3b55daf170a032dcc8ed12ee26ccf63 new file mode 100644 index 0000000000000..0c7739c2b9fbb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-7-c56c391659f5701a9d2e8782a60f7f8a new file mode 100644 index 0000000000000..79c8f8e614a1f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-8-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a b/sql/hive/src/test/resources/golden/partition_wise_fileformat3-9-f89a3f7c0ee5bc3f6bd04aa0b459dd4a new file mode 100644 index 0000000000000..0c6fbc997de7a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-0-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-1-c561806d8f9ad419dc9b17ae995aab68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-2-b9f8c3b822051854770f61e5ae5b48b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-3-9837451512e92e982f1bd9a12b132e84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-4-58cfa555b061057f559fc6b9c2f6c631 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 b/sql/hive/src/test/resources/golden/partition_wise_fileformat4-5-ac79def5434bb8a926237d0db8db2e84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-1-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-2-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-3-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-4-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-5-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 new file mode 100644 index 0000000000000..f0d140e18c053 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-6-a0e23b26ee1777ccc8947fb5eb1e8745 @@ -0,0 +1,2 @@ +101 25 +102 25 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-7-a0eeded14b3d337a74189a5d02c7a5ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 b/sql/hive/src/test/resources/golden/partition_wise_fileformat5-8-a0e23b26ee1777ccc8947fb5eb1e8745 new file mode 100644 index 0000000000000..6425bae2a6a8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-1-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-2-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-3-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-4-da1b1887eb530c7e9d37667b99c9793f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-5-517aaa22478287fa80eef4a19f2cb9ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-6-e95296c9f7056b0075007c61d4e5e92f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat6-7-4758d41d052eba37a9acd90c2dbc58f0 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-1-c854b607353e810be297d3159be30da4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-2-6c4f7b115f18953dcc7710fa97287459 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-3-f5f427b174dca478c14eddc371c0025a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-4-a34505bd397bb2a66e46408d1dfb6bf2 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat7-5-f2c42f1f32eb3cb300420fb36cbf2362 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-1-22e3d59a0423473051535684bca72b27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-2-55ae9fbf6daa36225dd386e34025dd38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-3-c561806d8f9ad419dc9b17ae995aab68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-4-ae71ce67b5d4a91bce1b34acde830268 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-5-cb3d7c0fec42ef7d68b67c8e4cdeab3e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-6-ae6a5016b6a6ace6b6f2576a4c666b4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-7-d1a5913edaaec9654edb333e8207f57b new file mode 100644 index 0000000000000..5e06930239fea --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd b/sql/hive/src/test/resources/golden/partition_wise_fileformat8-8-624b059dfaa86e2c78f065169de325cd new file mode 100644 index 0000000000000..1bd9063a9c63c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-0-66ee62178e3576fb38cb09800cb610bf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-1-22e3d59a0423473051535684bca72b27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-2-55ae9fbf6daa36225dd386e34025dd38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-3-92bfcf88ca528eb6c9259142bf6541e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-4-ae71ce67b5d4a91bce1b34acde830268 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-5-d1a5913edaaec9654edb333e8207f57b new file mode 100644 index 0000000000000..f259b4eefc608 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd b/sql/hive/src/test/resources/golden/partition_wise_fileformat9-6-624b059dfaa86e2c78f065169de325cd new file mode 100644 index 0000000000000..89afce150e7a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/plan_json-0-74146da55d57b22443140e7fbab3375c b/sql/hive/src/test/resources/golden/plan_json-0-74146da55d57b22443140e7fbab3375c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd1-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd1-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 b/sql/hive/src/test/resources/golden/ppd1-2-ae8aea06b05358ef9c486b61c9e30a69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 b/sql/hive/src/test/resources/golden/ppd1-3-1d3f71876ba45f942e93a68c0be15ef5 new file mode 100644 index 0000000000000..55b2a1b47afa2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd1-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 b/sql/hive/src/test/resources/golden/ppd1-5-ae8aea06b05358ef9c486b61c9e30a69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 b/sql/hive/src/test/resources/golden/ppd1-6-1d3f71876ba45f942e93a68c0be15ef5 new file mode 100644 index 0000000000000..55b2a1b47afa2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd2-1-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 b/sql/hive/src/test/resources/golden/ppd2-2-4d5021216c5bc600a8c5344945f55a4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b b/sql/hive/src/test/resources/golden/ppd2-3-fccdc7a24b626308d9ec17608f36564b new file mode 100644 index 0000000000000..878c86cb46b8a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 b/sql/hive/src/test/resources/golden/ppd2-4-4e457825319166f3bd2ad07d8f7c2f69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 b/sql/hive/src/test/resources/golden/ppd2-5-a2d5e5ec2504041ea1a62856c7086451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd2-6-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd2-7-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 b/sql/hive/src/test/resources/golden/ppd2-8-4d5021216c5bc600a8c5344945f55a4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b b/sql/hive/src/test/resources/golden/ppd2-9-fccdc7a24b626308d9ec17608f36564b new file mode 100644 index 0000000000000..878c86cb46b8a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_clusterby-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_clusterby-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 b/sql/hive/src/test/resources/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 new file mode 100644 index 0000000000000..e2ef71e141c46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_clusterby-10-62979aa9e6b4e6ffb44ec452aabbef65 @@ -0,0 +1 @@ +20 val_20 20 diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 b/sql/hive/src/test/resources/golden/ppd_clusterby-2-16681f9c2bdd44278817d72c138b6ee1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 b/sql/hive/src/test/resources/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 new file mode 100644 index 0000000000000..ae8b7551d9630 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_clusterby-3-707a2295731e0d631a6c5f71c745c8d5 @@ -0,0 +1 @@ +10 val_10 diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 b/sql/hive/src/test/resources/golden/ppd_clusterby-4-a2d5e5ec2504041ea1a62856c7086451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 b/sql/hive/src/test/resources/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 new file mode 100644 index 0000000000000..e2ef71e141c46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_clusterby-5-62979aa9e6b4e6ffb44ec452aabbef65 @@ -0,0 +1 @@ +20 val_20 20 diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_clusterby-6-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 b/sql/hive/src/test/resources/golden/ppd_clusterby-7-16681f9c2bdd44278817d72c138b6ee1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 b/sql/hive/src/test/resources/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 new file mode 100644 index 0000000000000..ae8b7551d9630 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_clusterby-8-707a2295731e0d631a6c5f71c745c8d5 @@ -0,0 +1 @@ +10 val_10 diff --git a/sql/hive/src/test/resources/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 b/sql/hive/src/test/resources/golden/ppd_clusterby-9-a2d5e5ec2504041ea1a62856c7086451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 b/sql/hive/src/test/resources/golden/ppd_constant_expr-2-2ab005b2ee21deedbe7d10904a034468 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 b/sql/hive/src/test/resources/golden/ppd_constant_expr-3-5b20725caf905c6674759fee7873d627 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 b/sql/hive/src/test/resources/golden/ppd_constant_expr-4-aded77ef8dced4717d919a949d109b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c b/sql/hive/src/test/resources/golden/ppd_constant_expr-5-7da8c74586c99d96254f6f031bcaeb1c new file mode 100644 index 0000000000000..c87107a2f1168 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_constant_expr-6-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 b/sql/hive/src/test/resources/golden/ppd_constant_expr-7-5b20725caf905c6674759fee7873d627 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 b/sql/hive/src/test/resources/golden/ppd_constant_expr-8-aded77ef8dced4717d919a949d109b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c b/sql/hive/src/test/resources/golden/ppd_constant_expr-9-7da8c74586c99d96254f6f031bcaeb1c new file mode 100644 index 0000000000000..c87107a2f1168 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 b/sql/hive/src/test/resources/golden/ppd_constant_where-0-345fa30edd72d631fee21c3beeeef3d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 b/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 new file mode 100644 index 0000000000000..e37d32abba426 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_constant_where-1-84c951641740895ca1c8ddc098805da5 @@ -0,0 +1 @@ +1000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_gby-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_gby-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e b/sql/hive/src/test/resources/golden/ppd_gby-2-fea67192aba8704a64a85da47f7f321e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d b/sql/hive/src/test/resources/golden/ppd_gby-3-97b8d0091058060f5f17cf5e81cce73d new file mode 100644 index 0000000000000..e1659e6072577 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_gby-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e b/sql/hive/src/test/resources/golden/ppd_gby-5-fea67192aba8704a64a85da47f7f321e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d b/sql/hive/src/test/resources/golden/ppd_gby-6-97b8d0091058060f5f17cf5e81cce73d new file mode 100644 index 0000000000000..e1659e6072577 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_gby2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_gby2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 b/sql/hive/src/test/resources/golden/ppd_gby2-2-25541db999d8c1d56ba36b63949b6073 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f b/sql/hive/src/test/resources/golden/ppd_gby2-3-321628d4c52f6992f2680a3a162f19f new file mode 100644 index 0000000000000..7e66866e2dd60 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_gby2-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 b/sql/hive/src/test/resources/golden/ppd_gby2-5-25541db999d8c1d56ba36b63949b6073 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f b/sql/hive/src/test/resources/golden/ppd_gby2-6-321628d4c52f6992f2680a3a162f19f new file mode 100644 index 0000000000000..7e66866e2dd60 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_gby_join-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_gby_join-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 b/sql/hive/src/test/resources/golden/ppd_gby_join-2-45e102aabf0e90ac455e2bab3988d8c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_gby_join-3-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 b/sql/hive/src/test/resources/golden/ppd_gby_join-4-45e102aabf0e90ac455e2bab3988d8c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 b/sql/hive/src/test/resources/golden/ppd_join-2-4c4dcd0c288653e39cfe077c19c68570 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 b/sql/hive/src/test/resources/golden/ppd_join-3-aab5d092ca17ed23ee71d3a6ef653998 new file mode 100644 index 0000000000000..fbce9efa766d7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 b/sql/hive/src/test/resources/golden/ppd_join-5-4c4dcd0c288653e39cfe077c19c68570 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 b/sql/hive/src/test/resources/golden/ppd_join-6-aab5d092ca17ed23ee71d3a6ef653998 new file mode 100644 index 0000000000000..fbce9efa766d7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d b/sql/hive/src/test/resources/golden/ppd_join2-2-307ac4f1b13e310dc1c61c05a113945d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 b/sql/hive/src/test/resources/golden/ppd_join2-3-d03c0ad3ab713691cf9d3b27ae1223f9 new file mode 100644 index 0000000000000..9e6c4359e78a8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join2-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d b/sql/hive/src/test/resources/golden/ppd_join2-5-307ac4f1b13e310dc1c61c05a113945d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 b/sql/hive/src/test/resources/golden/ppd_join2-6-d03c0ad3ab713691cf9d3b27ae1223f9 new file mode 100644 index 0000000000000..9e6c4359e78a8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join3-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 b/sql/hive/src/test/resources/golden/ppd_join3-2-5018d137c74aed08cc4da4cbd1904092 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b b/sql/hive/src/test/resources/golden/ppd_join3-3-42cd793c031af1f7961d7b5e237de76b new file mode 100644 index 0000000000000..0e11aea14d61d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join3-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 b/sql/hive/src/test/resources/golden/ppd_join3-5-5018d137c74aed08cc4da4cbd1904092 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b b/sql/hive/src/test/resources/golden/ppd_join3-6-42cd793c031af1f7961d7b5e237de76b new file mode 100644 index 0000000000000..0e11aea14d61d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join_filter-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join_filter-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 b/sql/hive/src/test/resources/golden/ppd_join_filter-10-2c1710aa3e08f618c1930305ebdccc17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 b/sql/hive/src/test/resources/golden/ppd_join_filter-11-41debade389a200de226d8ad0fe47d24 new file mode 100644 index 0000000000000..62127cc213b1d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 b/sql/hive/src/test/resources/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join_filter-12-3bcc51a124f8cbd456620853d808354 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join_filter-13-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 b/sql/hive/src/test/resources/golden/ppd_join_filter-14-2c1710aa3e08f618c1930305ebdccc17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 b/sql/hive/src/test/resources/golden/ppd_join_filter-15-41debade389a200de226d8ad0fe47d24 new file mode 100644 index 0000000000000..62127cc213b1d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 b/sql/hive/src/test/resources/golden/ppd_join_filter-2-2c1710aa3e08f618c1930305ebdccc17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 b/sql/hive/src/test/resources/golden/ppd_join_filter-3-41debade389a200de226d8ad0fe47d24 new file mode 100644 index 0000000000000..62127cc213b1d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join_filter-4-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join_filter-5-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 b/sql/hive/src/test/resources/golden/ppd_join_filter-6-2c1710aa3e08f618c1930305ebdccc17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 b/sql/hive/src/test/resources/golden/ppd_join_filter-7-41debade389a200de226d8ad0fe47d24 new file mode 100644 index 0000000000000..62127cc213b1d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join_filter-8-73819ea1a7c0653a61652b3766afb003 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_join_filter-9-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e b/sql/hive/src/test/resources/golden/ppd_outer_join1-2-a239ac7d9503d93e0859043019f3e02e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 b/sql/hive/src/test/resources/golden/ppd_outer_join1-3-9dfd333c14f82fd71c213e1d39c83524 new file mode 100644 index 0000000000000..997f37b76bedb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join1-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e b/sql/hive/src/test/resources/golden/ppd_outer_join1-5-a239ac7d9503d93e0859043019f3e02e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 b/sql/hive/src/test/resources/golden/ppd_outer_join1-6-9dfd333c14f82fd71c213e1d39c83524 new file mode 100644 index 0000000000000..997f37b76bedb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 b/sql/hive/src/test/resources/golden/ppd_outer_join2-2-b5443e75f4473eb3cbe55bf0d58cc999 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 b/sql/hive/src/test/resources/golden/ppd_outer_join2-3-5340cd2d88dcf41dc18119389a475e36 new file mode 100644 index 0000000000000..997f37b76bedb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join2-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 b/sql/hive/src/test/resources/golden/ppd_outer_join2-5-b5443e75f4473eb3cbe55bf0d58cc999 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 b/sql/hive/src/test/resources/golden/ppd_outer_join2-6-5340cd2d88dcf41dc18119389a475e36 new file mode 100644 index 0000000000000..997f37b76bedb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 b/sql/hive/src/test/resources/golden/ppd_outer_join3-2-8f14853ac6ce5f40c98982ace9fa6221 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 b/sql/hive/src/test/resources/golden/ppd_outer_join3-3-1e3af37cc2d9c2059488b5940a33a1d0 new file mode 100644 index 0000000000000..b3bf95dd32bac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join3-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 b/sql/hive/src/test/resources/golden/ppd_outer_join3-5-8f14853ac6ce5f40c98982ace9fa6221 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 b/sql/hive/src/test/resources/golden/ppd_outer_join3-6-1e3af37cc2d9c2059488b5940a33a1d0 new file mode 100644 index 0000000000000..b3bf95dd32bac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 b/sql/hive/src/test/resources/golden/ppd_outer_join4-2-7b0e117314e5784ba06b7ab69e689c84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe b/sql/hive/src/test/resources/golden/ppd_outer_join4-3-ac57dc2a7151f20029d6a97049d6eebe new file mode 100644 index 0000000000000..d4a363c49aeaf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join4-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 b/sql/hive/src/test/resources/golden/ppd_outer_join4-5-7b0e117314e5784ba06b7ab69e689c84 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe b/sql/hive/src/test/resources/golden/ppd_outer_join4-6-ac57dc2a7151f20029d6a97049d6eebe new file mode 100644 index 0000000000000..d4a363c49aeaf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join5-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_outer_join5-1-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b b/sql/hive/src/test/resources/golden/ppd_outer_join5-10-b12aa87aaf64b573ce0be7013117651b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 b/sql/hive/src/test/resources/golden/ppd_outer_join5-11-c8e68d5a524b965c244f01782bc42e97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 b/sql/hive/src/test/resources/golden/ppd_outer_join5-12-fd1ebd8150ed40b4b91774f6db42df2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee b/sql/hive/src/test/resources/golden/ppd_outer_join5-2-5b859f606230a70698edf52ca814beee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 b/sql/hive/src/test/resources/golden/ppd_outer_join5-3-2823683e3169487b80e882aa9c4e3bd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 b/sql/hive/src/test/resources/golden/ppd_outer_join5-4-ab457e9f757065dbf6bf66a997ab76a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 b/sql/hive/src/test/resources/golden/ppd_outer_join5-5-ad3dc0900226e1ae9674bec0e054a4d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a b/sql/hive/src/test/resources/golden/ppd_outer_join5-6-c669e5c7ae22c4fb96995c700ebadd9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd b/sql/hive/src/test/resources/golden/ppd_outer_join5-7-54d9885a797deb82236499f6bb99ddfd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c b/sql/hive/src/test/resources/golden/ppd_outer_join5-8-e6949a27ee3c56243c81660de7d97f9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 b/sql/hive/src/test/resources/golden/ppd_outer_join5-9-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_random-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_random-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b b/sql/hive/src/test/resources/golden/ppd_random-2-a589a2f51ebb3962202698c79db4a33b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_random-3-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b b/sql/hive/src/test/resources/golden/ppd_random-4-a589a2f51ebb3962202698c79db4a33b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/ppd_repeated_alias-0-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/ppd_repeated_alias-1-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 b/sql/hive/src/test/resources/golden/ppd_repeated_alias-2-588c0c8007b156167714d1cb06d10384 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 b/sql/hive/src/test/resources/golden/ppd_repeated_alias-3-106d8664d84c73f86268cf69e9ba6834 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f b/sql/hive/src/test/resources/golden/ppd_repeated_alias-4-2cd71603d825cddd0d181bd240c0051f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 b/sql/hive/src/test/resources/golden/ppd_repeated_alias-5-941a7af885ec77d91a78e03a6a568220 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 b/sql/hive/src/test/resources/golden/ppd_repeated_alias-6-c3746d419ceaab36dee78f4b4b38eb99 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf b/sql/hive/src/test/resources/golden/ppd_repeated_alias-7-54b70fb31202186b8984ae2feea299bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 b/sql/hive/src/test/resources/golden/ppd_repeated_alias-8-89696914fad2d7b7bfc5b7729a7e7c34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 b/sql/hive/src/test/resources/golden/ppd_repeated_alias-9-a299c8b1a9f8c2772989a5454574f4e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_transform-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_transform-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/sql/hive/src/test/resources/golden/ppd_transform-2-75f6b66b7ad7ca4ca8f1357d0de41bd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 b/sql/hive/src/test/resources/golden/ppd_transform-3-5d0e4236af589d3e63a8dd84e663b745 new file mode 100644 index 0000000000000..0190981db84ed --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_transform-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 b/sql/hive/src/test/resources/golden/ppd_transform-5-75f6b66b7ad7ca4ca8f1357d0de41bd9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 b/sql/hive/src/test/resources/golden/ppd_transform-6-5d0e4236af589d3e63a8dd84e663b745 new file mode 100644 index 0000000000000..0190981db84ed --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_udf_case-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_udf_case-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 b/sql/hive/src/test/resources/golden/ppd_udf_case-2-c44d82e56d50653148bb4f9f98a42137 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_udf_col-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_udf_col-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 b/sql/hive/src/test/resources/golden/ppd_udf_col-10-53bbac80bbacf419ea971ddbb0f48542 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e b/sql/hive/src/test/resources/golden/ppd_udf_col-2-1c26c857485a928462c79e1b38ed1d9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed b/sql/hive/src/test/resources/golden/ppd_udf_col-3-86fd1e8c9491c6ea0d67e111aa6cf9ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae b/sql/hive/src/test/resources/golden/ppd_udf_col-4-c601016163c99a115feaca5fe5b74aae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 b/sql/hive/src/test/resources/golden/ppd_udf_col-5-53bbac80bbacf419ea971ddbb0f48542 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_udf_col-6-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e b/sql/hive/src/test/resources/golden/ppd_udf_col-7-1c26c857485a928462c79e1b38ed1d9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed b/sql/hive/src/test/resources/golden/ppd_udf_col-8-86fd1e8c9491c6ea0d67e111aa6cf9ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae b/sql/hive/src/test/resources/golden/ppd_udf_col-9-c601016163c99a115feaca5fe5b74aae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_union-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 b/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_union-1-f18babdee8d2d4206ce4f2a93b6575f9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 b/sql/hive/src/test/resources/golden/ppd_union-2-fbfb76b9e6f2af48dcd227af03dffa89 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e b/sql/hive/src/test/resources/golden/ppd_union-3-678c8197f458b459171c266f7431683e new file mode 100644 index 0000000000000..4c3919232c73e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 b/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppd_union-4-145c2779dadb5bd921dc2baac608b803 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 b/sql/hive/src/test/resources/golden/ppd_union-5-fbfb76b9e6f2af48dcd227af03dffa89 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e b/sql/hive/src/test/resources/golden/ppd_union-6-678c8197f458b459171c266f7431683e new file mode 100644 index 0000000000000..4c3919232c73e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 b/sql/hive/src/test/resources/golden/ppd_union_view-0-d680c8ac7e5121c6971458a9022c00b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 b/sql/hive/src/test/resources/golden/ppd_union_view-1-bfb5ca0dd2d5f070ce359790f8b91480 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f b/sql/hive/src/test/resources/golden/ppd_union_view-10-a74a5ff32204d842175b3d880477178f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c b/sql/hive/src/test/resources/golden/ppd_union_view-11-745e750f4f4a36af27e87338a979240c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 b/sql/hive/src/test/resources/golden/ppd_union_view-2-ac54e78582e6bd5d5533a0f3bfb51369 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 b/sql/hive/src/test/resources/golden/ppd_union_view-3-f6fd9a5271e172e4a65ea57aaa41d512 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 b/sql/hive/src/test/resources/golden/ppd_union_view-4-19cd3ea4e274befd809e4aad21da8d50 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 b/sql/hive/src/test/resources/golden/ppd_union_view-5-443c0979b586f6a6dfb0dc5d28cd5124 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 b/sql/hive/src/test/resources/golden/ppd_union_view-6-b57b2b4f4cd8012fbfcd0b69c8d95e13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c b/sql/hive/src/test/resources/golden/ppd_union_view-7-aab9d79f65d6edfc1cae88a14b8e106c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 b/sql/hive/src/test/resources/golden/ppd_union_view-8-e3ee46daeac605b05c7ada97b3c43578 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c b/sql/hive/src/test/resources/golden/ppd_union_view-9-8a77c30b6b07717d1c0bee971fb0132c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 b/sql/hive/src/test/resources/golden/ppd_vc-0-cf479fbfecc042e8c9ea63e761da62a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 b/sql/hive/src/test/resources/golden/ppd_vc-1-e442e970ac492f95e5d8e55e21c0d229 new file mode 100644 index 0000000000000..d183a30ddf3b6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 b/sql/hive/src/test/resources/golden/ppd_vc-2-c9e44ddbb494ff7f02027205610bcb65 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 b/sql/hive/src/test/resources/golden/ppd_vc-3-be78760949abf728580442e9e37a3ce7 new file mode 100644 index 0000000000000..5b2461e35b5f4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_allchildsarenull-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d b/sql/hive/src/test/resources/golden/ppr_allchildsarenull-1-965b997838e7cc654cac68baef1be41d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 b/sql/hive/src/test/resources/golden/ppr_allchildsarenull-2-22b7886d627e6266f5f1415ba239c621 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 b/sql/hive/src/test/resources/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_allchildsarenull-3-73819ea1a7c0653a61652b3766afb003 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d b/sql/hive/src/test/resources/golden/ppr_allchildsarenull-4-965b997838e7cc654cac68baef1be41d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 b/sql/hive/src/test/resources/golden/ppr_allchildsarenull-5-22b7886d627e6266f5f1415ba239c621 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b b/sql/hive/src/test/resources/golden/ppr_pushdown-0-855b235f9c760ba9d6d0200bfd1ad08b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 b/sql/hive/src/test/resources/golden/ppr_pushdown-1-f689aaf15b08c433d1e93ce977a9b6d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 b/sql/hive/src/test/resources/golden/ppr_pushdown-10-178be64f122542983ff4384df4bb1530 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 b/sql/hive/src/test/resources/golden/ppr_pushdown-11-3828393aa33a55cf1aea707e1be0a452 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 b/sql/hive/src/test/resources/golden/ppr_pushdown-12-5affd35b94b0de3fb2a43f8729526055 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 b/sql/hive/src/test/resources/golden/ppr_pushdown-13-2c64f5abe8c23957d7f0602e9c257cd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c b/sql/hive/src/test/resources/golden/ppr_pushdown-14-c3188230751166d9d90082cd357ecb0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb b/sql/hive/src/test/resources/golden/ppr_pushdown-15-e52a39743f82af47902341a7ecd38afb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae b/sql/hive/src/test/resources/golden/ppr_pushdown-16-70bef3ba441873327e188ff2ec6d68ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 b/sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 new file mode 100644 index 0000000000000..eb3e1bc1a37f1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-17-fcd10fbfc55a2c0aa843fe618f9613c6 @@ -0,0 +1,2 @@ +1234 1234 +abcd 1234 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 b/sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 new file mode 100644 index 0000000000000..7ba18baf99d20 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-18-ff36e3978571ac05f11e8322c024e4b6 @@ -0,0 +1,2 @@ +1224 1224 +abcd 1224 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a b/sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a new file mode 100644 index 0000000000000..e1d91a4660f36 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-19-481005cf034ef3d7b998da32eb82aa9a @@ -0,0 +1,2 @@ +1214 1214 +abcd 1214 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 b/sql/hive/src/test/resources/golden/ppr_pushdown-2-5eede4874e678021938683fc2f5dc900 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 b/sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 new file mode 100644 index 0000000000000..1872f733e0ee7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-20-9073825e8b9804331f780980d1f9fa92 @@ -0,0 +1,2 @@ +12.4 12.4 +abcd 12.4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d b/sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d new file mode 100644 index 0000000000000..9fb4a9236e0fe --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-21-9cb28f0d3a434c9d1ab039192906ec9d @@ -0,0 +1,2 @@ +12+4 12+4 +abcd 12+4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d b/sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d new file mode 100644 index 0000000000000..1c8af4d81713a --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-22-678f50025924fee7b59f66e2abdb472d @@ -0,0 +1,2 @@ +12:4 12:4 +abcd 12:4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 b/sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 new file mode 100644 index 0000000000000..422b06c2f2841 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-23-4d5bfa800ba434f464b07bf9b5d39f20 @@ -0,0 +1,2 @@ +12%4 12%4 +abcd 12%4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 b/sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 new file mode 100644 index 0000000000000..51b2c2fdb2887 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-24-22663f09ea1c1bc303756067e84df5a7 @@ -0,0 +1,2 @@ +12*4 12*4 +abcd 12*4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b b/sql/hive/src/test/resources/golden/ppr_pushdown-25-e789d9b469aa1fffe4ce0a15a8c1fb9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 b/sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 new file mode 100644 index 0000000000000..e338e5eb13d50 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-26-8065c18e387fd8bffae19a80af8dc1d4 @@ -0,0 +1 @@ +1234 1234 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 b/sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 new file mode 100644 index 0000000000000..ffc2dc5792f54 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-27-b72de558c88ae91460989938000e0d27 @@ -0,0 +1 @@ +1224 1224 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 b/sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 new file mode 100644 index 0000000000000..fcdf68d826db1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-28-7217735d357770af4ffee730e4e9add4 @@ -0,0 +1 @@ +1214 1214 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea b/sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea new file mode 100644 index 0000000000000..18abbb411527e --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-29-630e2f7918b7727fc4ca057fa21e2eea @@ -0,0 +1 @@ +12.4 12.4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 b/sql/hive/src/test/resources/golden/ppr_pushdown-3-c5c542f8ee81cd0afd44e67fc7b4d306 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 b/sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 new file mode 100644 index 0000000000000..b8286f78807d2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-30-2e062414293b643ea4d7f6df92f939e4 @@ -0,0 +1 @@ +12+4 12+4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf b/sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf new file mode 100644 index 0000000000000..6ba2e7177ed23 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-31-5eba4bf10315099129eae319d73636cf @@ -0,0 +1 @@ +12:4 12:4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b b/sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b new file mode 100644 index 0000000000000..7152ea7cecfb3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-32-35af0585a4f98bc222c786688cb6de6b @@ -0,0 +1 @@ +12%4 12%4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 b/sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 new file mode 100644 index 0000000000000..44880cb42d749 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown-33-a5594625510703427ab8bae5d0563f73 @@ -0,0 +1 @@ +12*4 12*4 diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 b/sql/hive/src/test/resources/golden/ppr_pushdown-4-f54bebec398f0fdfdbc0393123dba234 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de b/sql/hive/src/test/resources/golden/ppr_pushdown-5-78af12432bcbf99d4a0d41c25f964de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d b/sql/hive/src/test/resources/golden/ppr_pushdown-6-58724fbe96a0b3deceef20c8cc5e318d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 b/sql/hive/src/test/resources/golden/ppr_pushdown-7-14570f946e75924d7926c809485951d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c b/sql/hive/src/test/resources/golden/ppr_pushdown-8-3ba325662296fc455f07f1c835495e4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 b/sql/hive/src/test/resources/golden/ppr_pushdown-9-4375f17bed264d5554a67d119fa5dd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b b/sql/hive/src/test/resources/golden/ppr_pushdown2-0-855b235f9c760ba9d6d0200bfd1ad08b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 b/sql/hive/src/test/resources/golden/ppr_pushdown2-1-1f31dc385f79c5a7ae6a0d55b49bd583 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c b/sql/hive/src/test/resources/golden/ppr_pushdown2-10-ab3e46183487096745d7d90e3020e94c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b b/sql/hive/src/test/resources/golden/ppr_pushdown2-11-680316eba447eb4649530fdc1c37d95b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a b/sql/hive/src/test/resources/golden/ppr_pushdown2-12-720582e599a974ee9ca46b653678a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 b/sql/hive/src/test/resources/golden/ppr_pushdown2-13-bd9067aeee8884a32db921b6d478f806 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 b/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 new file mode 100644 index 0000000000000..e3e04ee48543d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-14-8d01597374157f2d3d066840983ba1f8 @@ -0,0 +1 @@ +3 1 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 new file mode 100644 index 0000000000000..c458b0f57aba9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-15-5614065e1b8e709f68be4fa67666f41 @@ -0,0 +1 @@ +2 1 1 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb new file mode 100644 index 0000000000000..c458b0f57aba9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-16-3a67618e47c977f58c9dd8f4b9a576eb @@ -0,0 +1 @@ +2 1 1 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe b/sql/hive/src/test/resources/golden/ppr_pushdown2-17-44e5f3ac566f60d8b17ef19c18a11ebe new file mode 100644 index 0000000000000..63511415ddf55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 b/sql/hive/src/test/resources/golden/ppr_pushdown2-18-e2c7e9b01ec95dfcc685827e24d66775 new file mode 100644 index 0000000000000..c592b5d505b0e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 new file mode 100644 index 0000000000000..e3e04ee48543d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-19-bd7e1917f8d2cf50c062a22ef3fa15b5 @@ -0,0 +1 @@ +3 1 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e b/sql/hive/src/test/resources/golden/ppr_pushdown2-2-efd3e4c333d4efb81218df8921e58f9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 new file mode 100644 index 0000000000000..e3e04ee48543d --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-20-ece6fe0efc1e658b36ddc10f0653d229 @@ -0,0 +1 @@ +3 1 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c b/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c new file mode 100644 index 0000000000000..679926f7d738c --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-3-1886675984606b6c972c4a26dca6fd2c @@ -0,0 +1 @@ +2 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 new file mode 100644 index 0000000000000..fcc7be2cb12aa --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-4-33b137b28e7246ec3c2acb937c638910 @@ -0,0 +1 @@ +22 22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 b/sql/hive/src/test/resources/golden/ppr_pushdown2-5-b5a2518af801f95fe52a75dfc1d3e867 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 b/sql/hive/src/test/resources/golden/ppr_pushdown2-6-96059d8042ccb2ce355541daf9893954 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 b/sql/hive/src/test/resources/golden/ppr_pushdown2-7-93922fe8d88643e1b8aa7e3ef4195404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 b/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 new file mode 100644 index 0000000000000..3ebc6d7fa2375 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-8-c86940e70f705f50e9091c257ee2bb40 @@ -0,0 +1 @@ +2 2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 new file mode 100644 index 0000000000000..a4544ab84afa3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown2-9-4fa4ba4c739b6f44975d41e4705d4389 @@ -0,0 +1 @@ +1 1 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a b/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/ppr_pushdown3-0-df2401785dfa257de49c3ad80b0f480a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 b/sql/hive/src/test/resources/golden/ppr_pushdown3-1-cf6ff7dfb16e7d2567e8f83fcc5d08e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 b/sql/hive/src/test/resources/golden/ppr_pushdown3-2-e879051803d0b64139e703e40fb007d0 new file mode 100644 index 0000000000000..e22f840876d2e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d b/sql/hive/src/test/resources/golden/ppr_pushdown3-3-c6fac1670fbd1ee490867efb1913a12d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb b/sql/hive/src/test/resources/golden/ppr_pushdown3-4-a1c18483e5f5d2fe351be09af658acbb new file mode 100644 index 0000000000000..355ed1617e200 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 b/sql/hive/src/test/resources/golden/ppr_pushdown3-5-9b129023196c29071a92e91471872593 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f b/sql/hive/src/test/resources/golden/ppr_pushdown3-6-67118904e6cc8e9f5210fab88e87fb8f new file mode 100644 index 0000000000000..57005044dde38 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 b/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/print_header-0-860e298a0b70e7a531431e9386ddc0e7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a b/sql/hive/src/test/resources/golden/print_header-1-8540676fc16ac91f3629c40f393a890a new file mode 100644 index 0000000000000..943e1be13b615 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 b/sql/hive/src/test/resources/golden/print_header-2-5cff10d4b561206e7e0b2e81d862ff93 new file mode 100644 index 0000000000000..951e74db0fe23 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/print_header-3-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 b/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/progress_1-0-fe903cc5a573e7aced5ee17b35d2fb04 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 b/sql/hive/src/test/resources/golden/progress_1-1-e93fbb4cb074376a645dc22075c0eab6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f b/sql/hive/src/test/resources/golden/progress_1-2-70ba81c09588aa586e62ebaee2af685f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 b/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 new file mode 100644 index 0000000000000..0b3e0a69a8c79 --- /dev/null +++ b/sql/hive/src/test/resources/golden/progress_1-3-43d286eebddaee26cf95f26e62a75fe4 @@ -0,0 +1 @@ +5000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 b/sql/hive/src/test/resources/golden/protectmode-0-44765a9e389cd7be27e038d257ad2c61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 b/sql/hive/src/test/resources/golden/protectmode-1-377270bf028b6a5e8a5a1679421aece6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 b/sql/hive/src/test/resources/golden/protectmode-10-b993dfdb8636411e4e9182a5a1931362 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 b/sql/hive/src/test/resources/golden/protectmode-11-b5aa37f2dd84c363d257ededdcc9a6b6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a b/sql/hive/src/test/resources/golden/protectmode-12-d35a2c6ec13dcba7f4856522d2c781a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c b/sql/hive/src/test/resources/golden/protectmode-13-70e6f4e73e37e6288fafdf6b8da5c62c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b b/sql/hive/src/test/resources/golden/protectmode-14-f817d21ee3a7646bf107c87bb264586b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 b/sql/hive/src/test/resources/golden/protectmode-15-c51a200a345ceca9838507176a4a6dd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b b/sql/hive/src/test/resources/golden/protectmode-16-151259dfc1cea42392a7511991c58a5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 b/sql/hive/src/test/resources/golden/protectmode-17-6fa385b2d2c6fee674dfff38f7338080 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 b/sql/hive/src/test/resources/golden/protectmode-18-968e78b685c3b83bf5eb8a3a786657a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 b/sql/hive/src/test/resources/golden/protectmode-19-b6e156f104768706aa587b762a9d4d18 new file mode 100644 index 0000000000000..eb63636d0bfd4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea b/sql/hive/src/test/resources/golden/protectmode-2-f450722c47e95d5825d0cb920c212aea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b b/sql/hive/src/test/resources/golden/protectmode-20-719ec12056901bb0330fec1f05d4a94b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-21-a31442a7d6c6950c137856ab861d622d new file mode 100644 index 0000000000000..964ede006ad21 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 b/sql/hive/src/test/resources/golden/protectmode-22-8e2cda1a6895512d0c2e72e6b8b283d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-23-a31442a7d6c6950c137856ab861d622d new file mode 100644 index 0000000000000..68b03670f5f80 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 b/sql/hive/src/test/resources/golden/protectmode-24-2281fb42e04f3f3b1ae71e76feeb7733 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 b/sql/hive/src/test/resources/golden/protectmode-25-9095794bb72e75a50f84c7cc79b14df6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-26-a31442a7d6c6950c137856ab861d622d new file mode 100644 index 0000000000000..ec18ce7411535 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 b/sql/hive/src/test/resources/golden/protectmode-27-af002c5db76382ae7619968eaff3cc91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d b/sql/hive/src/test/resources/golden/protectmode-28-a31442a7d6c6950c137856ab861d622d new file mode 100644 index 0000000000000..a6202ac5fc0eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 b/sql/hive/src/test/resources/golden/protectmode-29-6fa385b2d2c6fee674dfff38f7338080 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc b/sql/hive/src/test/resources/golden/protectmode-3-4038060ccc6df535736b4b3cdf722cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 b/sql/hive/src/test/resources/golden/protectmode-30-2cf1d601e4e37fa223cc841c0bf443f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 b/sql/hive/src/test/resources/golden/protectmode-31-97f50f7a49ee261db1888e93cf696c27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 b/sql/hive/src/test/resources/golden/protectmode-32-b6e156f104768706aa587b762a9d4d18 new file mode 100644 index 0000000000000..0a6cebbbd0b82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b b/sql/hive/src/test/resources/golden/protectmode-33-151259dfc1cea42392a7511991c58a5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 b/sql/hive/src/test/resources/golden/protectmode-34-520cc99451f889b26e12308b956f41e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 b/sql/hive/src/test/resources/golden/protectmode-35-67854bfbea7afeca2987ced73b68e671 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 b/sql/hive/src/test/resources/golden/protectmode-36-5b68bd2ad93372f1926166bb1eea7ae1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e b/sql/hive/src/test/resources/golden/protectmode-37-8c07fbbf528fd9180db5a52474f6683e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 b/sql/hive/src/test/resources/golden/protectmode-38-da4a5ad43dbf3df8e31d23979af46423 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 b/sql/hive/src/test/resources/golden/protectmode-39-771f55b9c0dbadc2b8799cb4756c63e9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 b/sql/hive/src/test/resources/golden/protectmode-4-b993dfdb8636411e4e9182a5a1931362 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-40-39024332d020339bd9396c9b15403a0c b/sql/hive/src/test/resources/golden/protectmode-40-39024332d020339bd9396c9b15403a0c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 b/sql/hive/src/test/resources/golden/protectmode-41-7643ff9b92368f2cc44ab8b1f5738d24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb b/sql/hive/src/test/resources/golden/protectmode-42-3a8863ad6fac90ba48555c1042c92ebb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 b/sql/hive/src/test/resources/golden/protectmode-43-377270bf028b6a5e8a5a1679421aece6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c b/sql/hive/src/test/resources/golden/protectmode-5-2af90060e2a14790441e0716e3372e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 b/sql/hive/src/test/resources/golden/protectmode-6-e2d1fc9c94e0d3597f393aec99850510 new file mode 100644 index 0000000000000..a1ff1f8341f30 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 b/sql/hive/src/test/resources/golden/protectmode-7-143fafe70e2ec2d10c6eeef114e7fad1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 b/sql/hive/src/test/resources/golden/protectmode-8-e2d1fc9c94e0d3597f393aec99850510 new file mode 100644 index 0000000000000..9098c9419556b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc b/sql/hive/src/test/resources/golden/protectmode-9-4038060ccc6df535736b4b3cdf722cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 b/sql/hive/src/test/resources/golden/push_or-0-82eac35d076df1ee8020d57832f0f895 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e b/sql/hive/src/test/resources/golden/push_or-1-4c03e0f7fa028897c0caee62167eb68e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 b/sql/hive/src/test/resources/golden/push_or-2-62abd65e444981a96f7cf3dab3e013f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 b/sql/hive/src/test/resources/golden/push_or-3-34a713b9523589f1eb3205037d56a5b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 b/sql/hive/src/test/resources/golden/push_or-4-22888cdef1b94bde983541c1f3d510d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a b/sql/hive/src/test/resources/golden/push_or-5-c94def4b18b9c8c00e7a93eb19ec694a new file mode 100644 index 0000000000000..3a2f20c637883 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 b/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 new file mode 100644 index 0000000000000..2f3fe0189d2c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/query_with_semi-0-3731ce715b60549c17b7993927d48436 @@ -0,0 +1 @@ +aa; \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb b/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb new file mode 100644 index 0000000000000..b5b5773c405b4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/query_with_semi-1-3f53ec3b276b32cf81729433e47010cb @@ -0,0 +1 @@ +bb \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 b/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 new file mode 100644 index 0000000000000..2652f5f42c003 --- /dev/null +++ b/sql/hive/src/test/resources/golden/query_with_semi-2-bf8cb175f9b13fcc23ba46be674b5767 @@ -0,0 +1 @@ +cc \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 b/sql/hive/src/test/resources/golden/quote1-0-54ea0f3c04b185472a38a744d8003c13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 b/sql/hive/src/test/resources/golden/quote1-1-965ec21dbb2cdbfb0e8a8cb20ea96416 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 b/sql/hive/src/test/resources/golden/quote1-2-14d4949bc9215a1004c945885ef10de4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 b/sql/hive/src/test/resources/golden/quote1-3-f36bdb38d0c1b79c6113fa0795d14533 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 b/sql/hive/src/test/resources/golden/quote1-4-582bb2fb4c4ee50eb97bafae452a69a6 new file mode 100644 index 0000000000000..9a0990555b81f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 b/sql/hive/src/test/resources/golden/quote2-0-ea1a1d0c5f9a3248afbb65e6632c5118 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 b/sql/hive/src/test/resources/golden/quote2-1-34f3c423b2fb1f0b11457f45a60042b9 new file mode 100644 index 0000000000000..ed1f53a6588d0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e b/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/quoted alias.attr-0-97b3c408090f758257e4bd20597a525e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d b/sql/hive/src/test/resources/golden/rand_partitionpruner1-0-a7e4414330751eb8ad486bb11643f64d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 b/sql/hive/src/test/resources/golden/rand_partitionpruner1-1-11cdebc422d7a0e7b257279ac9524321 new file mode 100644 index 0000000000000..73b8edab36833 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-0-b9598847d77e2c425423f51d755380e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-1-b7dcf0277eab6c02a7ca47aea7703bf7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-2-db276de57ad86e8880037336886cd557 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 b/sql/hive/src/test/resources/golden/rand_partitionpruner2-3-e4419c33287ca1f48a43f61cca5b5928 new file mode 100644 index 0000000000000..e8738e9c837ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e b/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_bigdata-0-e011be1172043c0c6d0fd2c0e89f361e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 b/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_bigdata-1-6d0d6f4de136f56ab91987e19df8e178 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 b/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_bigdata-2-3688b45adbdb190d58799c0b6d601055 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a b/sql/hive/src/test/resources/golden/rcfile_bigdata-3-fc0c054cdfbf5c130532e139d497866a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 b/sql/hive/src/test/resources/golden/rcfile_columnar-0-4f3c6c5b4a75c97b8cc0432a56dcafd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 b/sql/hive/src/test/resources/golden/rcfile_columnar-1-5d636fc4682aee89328b7c7be17e1013 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b b/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b new file mode 100644 index 0000000000000..d1b3011bffd91 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_columnar-2-b2f56374f2ea8a967b38e77b57138d4b @@ -0,0 +1,2 @@ +key string None +value string None diff --git a/sql/hive/src/test/resources/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff b/sql/hive/src/test/resources/golden/rcfile_columnar-3-4653c703effa4cc65743128fa3c65ff new file mode 100644 index 0000000000000..b7bef323d7df0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-0-82f12a09df66752b05e89d3d12ba1e32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-1-de09b3ac5ba6a77653eb361496863085 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b new file mode 100644 index 0000000000000..96855269dbfa8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-10-fb011a8ee13a82a873039bd28353205b @@ -0,0 +1,5 @@ +255 1 +278 1 +311 1 +409 1 +484 1 diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-11-3708198aac609695b22e19e89306034c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-12-3a4c24fd561f459025264baa3fb6d87 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-2-bc4929594297fa2ee82b8ceff9118e46 new file mode 100644 index 0000000000000..2ba7d9f4f32ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 new file mode 100644 index 0000000000000..1b977ce91e68d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-3-18a2fa22b8cef720cb30d75353b3f276 @@ -0,0 +1,3 @@ +255 val_255 +278 val_278 +311 val_311 diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b new file mode 100644 index 0000000000000..96855269dbfa8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-4-fb011a8ee13a82a873039bd28353205b @@ -0,0 +1,5 @@ +255 1 +278 1 +311 1 +409 1 +484 1 diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-5-68975193b30cb34102b380e647d8d5f4 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-6-ea607fbed28d20e5726f4501285d698d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-7-de09b3ac5ba6a77653eb361496863085 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-8-bc4929594297fa2ee82b8ceff9118e46 new file mode 100644 index 0000000000000..2ba7d9f4f32ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 new file mode 100644 index 0000000000000..1b977ce91e68d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_lazydecompress-9-18a2fa22b8cef720cb30d75353b3f276 @@ -0,0 +1,3 @@ +255 val_255 +278 val_278 +311 val_311 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 b/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge1-0-d877ca1eefa4344eae30ad3ef2039b00 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge1-1-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge1-10-f94fdb0b79dcf73989e6fbce87355753 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 b/sql/hive/src/test/resources/golden/rcfile_merge1-11-3fb2f07dd710f180a3c39dd17beccaa5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c b/sql/hive/src/test/resources/golden/rcfile_merge1-12-150cb190dc2343a747ea89298eb6352c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 b/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 new file mode 100644 index 0000000000000..b141fed81f206 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge1-13-af7b1e79eb40854ea82d61debaafcf40 @@ -0,0 +1 @@ +53278638794 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 b/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 new file mode 100644 index 0000000000000..b141fed81f206 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge1-14-4547f75ed3cb94914c2d025c1e057b5 @@ -0,0 +1 @@ +53278638794 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 b/sql/hive/src/test/resources/golden/rcfile_merge1-15-2f190c7e30999cbdf1ce62e8f31438f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e b/sql/hive/src/test/resources/golden/rcfile_merge1-16-c198c437f48c3844d8d0ace881b3107e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge1-2-690b2898f94ef32f01ded0ddb737a056 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee b/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge1-3-f88c85ce3cbcdc607bce650e1ccff4ee @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 b/sql/hive/src/test/resources/golden/rcfile_merge1-4-2f190c7e30999cbdf1ce62e8f31438f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e b/sql/hive/src/test/resources/golden/rcfile_merge1-5-c198c437f48c3844d8d0ace881b3107e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 b/sql/hive/src/test/resources/golden/rcfile_merge1-6-8adb6d3de3839fc1f0bd2598fdcc3d46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 b/sql/hive/src/test/resources/golden/rcfile_merge1-7-8c76c8d41133f29366359c308b0c9cc0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f b/sql/hive/src/test/resources/golden/rcfile_merge1-8-ed4ca53754ed08a0fab1d788e3c65d6f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 b/sql/hive/src/test/resources/golden/rcfile_merge1-9-780b166bd3fcc932cb217273b0636d63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge2-0-f94fdb0b79dcf73989e6fbce87355753 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge2-1-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 b/sql/hive/src/test/resources/golden/rcfile_merge2-10-5b4fb8038f522877101a4e429f082f11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge2-2-690b2898f94ef32f01ded0ddb737a056 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge2-3-778b659dba30ece782a956d87b1a31eb @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 b/sql/hive/src/test/resources/golden/rcfile_merge2-4-5b4fb8038f522877101a4e429f082f11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 b/sql/hive/src/test/resources/golden/rcfile_merge2-5-78b97f09b49452e054cba2ae461813c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a b/sql/hive/src/test/resources/golden/rcfile_merge2-6-d5a91ca1ff9039b2dc55af8735d0249a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 b/sql/hive/src/test/resources/golden/rcfile_merge2-7-805fc662f9b3b7a587d1043b659e2424 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 b/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 new file mode 100644 index 0000000000000..8d25d618795b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge2-8-afd119f412a66bdf5cd1c74ae8965685 @@ -0,0 +1 @@ +-4208881187 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 b/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 new file mode 100644 index 0000000000000..8d25d618795b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge2-9-351c25ee0ca991ca9e0dbafa873b7592 @@ -0,0 +1 @@ +-4208881187 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge3-0-f94fdb0b79dcf73989e6fbce87355753 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge3-1-690b2898f94ef32f01ded0ddb737a056 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 b/sql/hive/src/test/resources/golden/rcfile_merge3-10-1486feb77f13bb9a0ed606fe795ef686 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 b/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 new file mode 100644 index 0000000000000..11ff946b46f0f --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge3-11-7674266b00c52a2b6755910ea0793b69 @@ -0,0 +1 @@ +14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f b/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f new file mode 100644 index 0000000000000..11ff946b46f0f --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge3-12-56703d58e54ca8b4fb86c92ffd74cc1f @@ -0,0 +1 @@ +14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge3-13-d57efd24ca36f282a37be5d0bf6452e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge3-14-5ccde6b58d36e11d6e3c02a867eced7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge3-2-778b659dba30ece782a956d87b1a31eb @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge3-3-d57efd24ca36f282a37be5d0bf6452e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge3-4-5ccde6b58d36e11d6e3c02a867eced7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 b/sql/hive/src/test/resources/golden/rcfile_merge3-5-bb2196ee46a7719cc88dcd6976a6eb38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b b/sql/hive/src/test/resources/golden/rcfile_merge3-6-e4d1c54cf6a802eef3342bec2505f69b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/sql/hive/src/test/resources/golden/rcfile_merge3-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 b/sql/hive/src/test/resources/golden/rcfile_merge3-8-4891ccd223f2403e235e2c9d6b5fc18 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 b/sql/hive/src/test/resources/golden/rcfile_merge3-9-7c38c42af102069e08a85a27a735eb61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 b/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge4-0-f94fdb0b79dcf73989e6fbce87355753 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 b/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge4-1-690b2898f94ef32f01ded0ddb737a056 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 b/sql/hive/src/test/resources/golden/rcfile_merge4-10-1486feb77f13bb9a0ed606fe795ef686 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 b/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 new file mode 100644 index 0000000000000..11ff946b46f0f --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge4-11-7674266b00c52a2b6755910ea0793b69 @@ -0,0 +1 @@ +14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f b/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f new file mode 100644 index 0000000000000..11ff946b46f0f --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge4-12-56703d58e54ca8b4fb86c92ffd74cc1f @@ -0,0 +1 @@ +14412220296 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge4-13-d57efd24ca36f282a37be5d0bf6452e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge4-14-5ccde6b58d36e11d6e3c02a867eced7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb b/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_merge4-2-778b659dba30ece782a956d87b1a31eb @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 b/sql/hive/src/test/resources/golden/rcfile_merge4-3-d57efd24ca36f282a37be5d0bf6452e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b b/sql/hive/src/test/resources/golden/rcfile_merge4-4-5ccde6b58d36e11d6e3c02a867eced7b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 b/sql/hive/src/test/resources/golden/rcfile_merge4-5-18f216bf1ea33debc2576ac85ac1a449 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 b/sql/hive/src/test/resources/golden/rcfile_merge4-6-3e292c7212dd02d7d6000837e09a42f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 b/sql/hive/src/test/resources/golden/rcfile_merge4-7-f9661455ce2c3bbe3d9cb4dc4ce0eb98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 b/sql/hive/src/test/resources/golden/rcfile_merge4-8-4891ccd223f2403e235e2c9d6b5fc18 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 b/sql/hive/src/test/resources/golden/rcfile_merge4-9-7c38c42af102069e08a85a27a735eb61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 b/sql/hive/src/test/resources/golden/rcfile_null_value-0-69edd47d7cca001865102e3fad925613 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d b/sql/hive/src/test/resources/golden/rcfile_null_value-1-38ac0f806892e3b25b0038c5f227c8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee b/sql/hive/src/test/resources/golden/rcfile_null_value-2-e721b8138774fdefca7171d1051841ee new file mode 100644 index 0000000000000..b0135533064c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a b/sql/hive/src/test/resources/golden/rcfile_null_value-3-3c82d1a57cf295901c6b931278a3760a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 b/sql/hive/src/test/resources/golden/rcfile_null_value-4-8e839adf91e01e3230e15e9a936c9126 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed b/sql/hive/src/test/resources/golden/rcfile_null_value-5-d57d9c8882edbb3c6116d89516e4aed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 b/sql/hive/src/test/resources/golden/rcfile_null_value-6-a7a74aeb8c5cac1f2fd2362398a52d2 new file mode 100644 index 0000000000000..06b2b4d7e6d47 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-0-644898731e5228e863236f388ba1aa07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-1-a9f718201b71fa855fb81a29e1d7bd3b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-2-4a9a9175758ef576804c8b7309b019e8 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-3-ea607fbed28d20e5726f4501285d698d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-4-4df4ea1d1d30dedff582dc5c249dce85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-5-6a3af12e36cec853c876a2cbae61c23a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 b/sql/hive/src/test/resources/golden/rcfile_toleratecorruptions-6-718032defb864225dd664b1719f3b590 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 b/sql/hive/src/test/resources/golden/rcfile_union-0-2a3aaaffda2fb4486871bed84c68fc60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b b/sql/hive/src/test/resources/golden/rcfile_union-1-ea555c16ad460f9a8d17b4e448d7c91b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 b/sql/hive/src/test/resources/golden/rcfile_union-2-e104293998e4d4ea9d69d6b48f9aa6e5 new file mode 100644 index 0000000000000..c0e3764b993b4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 b/sql/hive/src/test/resources/golden/recursive_dir-0-fb096f0f4ecc530357ad76ae0353d338 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 b/sql/hive/src/test/resources/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 new file mode 100644 index 0000000000000..1ab53eed2f15a --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate-10-44d72569675c48e482ff53f45c7526e3 @@ -0,0 +1 @@ +130091 36210398070 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 b/sql/hive/src/test/resources/golden/reduce_deduplicate-11-a9591dfbb3d8c6baa66fd9d56389bc81 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 b/sql/hive/src/test/resources/golden/reduce_deduplicate-12-d6f20dc500809305e7454db01e24ce26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 b/sql/hive/src/test/resources/golden/reduce_deduplicate-13-1583df07fff7a2af0a9cc6a681446036 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe b/sql/hive/src/test/resources/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate-3-45ca2bb223236cce4dcc7d7d7c96edbe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 b/sql/hive/src/test/resources/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate-4-ac9d2110b999fb41f5f0f183742a8b27 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d b/sql/hive/src/test/resources/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate-5-e9d5f1694b887c50ef17ba48545faa4d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 b/sql/hive/src/test/resources/golden/reduce_deduplicate-6-890c5edd947ca276254ef4bd2813b296 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 b/sql/hive/src/test/resources/golden/reduce_deduplicate-7-e6ae500ce884875bd5f8e4e5200764e7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 b/sql/hive/src/test/resources/golden/reduce_deduplicate-8-e15d6f0f5d84bdeb153965cf0cc4fe11 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 b/sql/hive/src/test/resources/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 new file mode 100644 index 0000000000000..da664fda89fd7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate-9-3b77920b01edb72aa3a7fc4f813e5d72 @@ -0,0 +1 @@ +21025334 36210398070 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-0-163aa8a17867cfbd9297cc8825c4f3d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-1-a3f3e279ab0be5093f4a926e265c0211 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-2-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-3-862d1558d2999ca7ee585f8bdc3dacb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_gby-4-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-0-aa047b3a8b40b68b93c4ad11e173c767 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-1-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 b/sql/hive/src/test/resources/golden/reduce_deduplicate_exclude_join-2-f420affa96a8f13a91f6082805cf72a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-0-aa047b3a8b40b68b93c4ad11e173c767 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-1-d0a93f40892e3894460553b443c77428 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-10-5a0cbc5744352a117afb34d1754496fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-11-9a4a8559680a45aad0a54144fb200f5e new file mode 100644 index 0000000000000..4a827d502b934 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-12-4c2ae1bc1c838240e1f355714ed8e785 new file mode 100644 index 0000000000000..af5653b0de758 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-13-ca02f80b104ee3dff445518501d0ee3e new file mode 100644 index 0000000000000..5d9e6bbf9cedb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-14-fd197d657a19ef5cdcb9cc7b5824035c new file mode 100644 index 0000000000000..4a827d502b934 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-15-dbc2b906b6cb99a138175eb4764da6bd new file mode 100644 index 0000000000000..32740bfc546a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-16-22d9cdca0e70becd2ef86db59fc01ebb new file mode 100644 index 0000000000000..cc33012c6e2eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-17-ffeb52326db68aafe4bb801ef6e7db5f new file mode 100644 index 0000000000000..851d5ed155549 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-18-4435a485aa7e3a72b789418e063d4cb8 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-19-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-2-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-20-29dc99f23850e9f19368926d621a3029 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-21-722d6027daf02b2e40f121a4fec362d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-22-f3a586e75df55df355982eac0d4d4a63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-23-241faeae86e3e43e3ed8e35fb24e027d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-24-dd3ed01a7ac656855ce76b4e6fcd27ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-25-bce36b45ce0c67af052bcf0487a53052 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-26-93434017959c1be56608adff9309f02a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-27-5a0cbc5744352a117afb34d1754496fa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-28-9a4a8559680a45aad0a54144fb200f5e new file mode 100644 index 0000000000000..4a827d502b934 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-29-4c2ae1bc1c838240e1f355714ed8e785 new file mode 100644 index 0000000000000..af5653b0de758 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-3-290a9fab18a45f29dd84ac3da0a47fb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-30-ca02f80b104ee3dff445518501d0ee3e new file mode 100644 index 0000000000000..5d9e6bbf9cedb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-31-fd197d657a19ef5cdcb9cc7b5824035c new file mode 100644 index 0000000000000..4a827d502b934 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-32-dbc2b906b6cb99a138175eb4764da6bd new file mode 100644 index 0000000000000..32740bfc546a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-33-22d9cdca0e70becd2ef86db59fc01ebb new file mode 100644 index 0000000000000..cc33012c6e2eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-34-ffeb52326db68aafe4bb801ef6e7db5f new file mode 100644 index 0000000000000..851d5ed155549 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-35-4435a485aa7e3a72b789418e063d4cb8 new file mode 100644 index 0000000000000..8bfbbf1e69f57 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-4-722d6027daf02b2e40f121a4fec362d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-5-f3a586e75df55df355982eac0d4d4a63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-6-8801137a557caa5a75cfacd7aaf56bad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-7-7e350e31bce459f89b322b3356806174 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-8-bce36b45ce0c67af052bcf0487a53052 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 b/sql/hive/src/test/resources/golden/reduce_deduplicate_extended-9-a94cab7395270afa10d73b5e8c2f4a71 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb b/sql/hive/src/test/resources/golden/reducesink_dedup-0-d3f50875bd5dff172cf813fdb7d738eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 b/sql/hive/src/test/resources/golden/reducesink_dedup-1-dda16565b98926fc3587de937b9401c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 b/sql/hive/src/test/resources/golden/reducesink_dedup-2-971f12c90619b02484c8e7db1b32f050 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 b/sql/hive/src/test/resources/golden/regex_col-0-ac78bd83c9aa538c2827598dd6007a69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db b/sql/hive/src/test/resources/golden/regex_col-1-42751bfc3f1e83e7a014db9272d597db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 b/sql/hive/src/test/resources/golden/regex_col-2-21564f64cdfd46098e1254380490701 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 b/sql/hive/src/test/resources/golden/regex_col-3-f0c653593a7a7b701f0614f3f0a7ba61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 b/sql/hive/src/test/resources/golden/regex_col-4-daf9d3ca22b243870a138ba90d1593c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 b/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/remote_script-0-4e2e94d649b1792c5bd3fd879349ef92 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa b/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/remote_script-1-e168f471980470d93b790702a70238fa @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 b/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/remote_script-2-a19a19272149c732977c37e043910505 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d b/sql/hive/src/test/resources/golden/remote_script-3-4eb54a664e549614d56ca088c8867d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 b/sql/hive/src/test/resources/golden/rename_column-0-f7eb4bd6f226be0c13117294be250271 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-1-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..726e0947d8302 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 b/sql/hive/src/test/resources/golden/rename_column-10-7ef160935cece55338bd4d52277b0203 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-11-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..500201be8d922 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 b/sql/hive/src/test/resources/golden/rename_column-12-379d54e3aa66daacff23c75007dfa008 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-13-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..71af16a5fa4d7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 b/sql/hive/src/test/resources/golden/rename_column-14-25bfcf66698b12f82903f72f13fea4e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-15-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..f5b2e72aeccab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-16-d032f4795c1186255acea241387adf93 b/sql/hive/src/test/resources/golden/rename_column-16-d032f4795c1186255acea241387adf93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..ecafeaea5f61a --- /dev/null +++ b/sql/hive/src/test/resources/golden/rename_column-17-9c36cac1372650b703400c60dd29042c @@ -0,0 +1,2 @@ +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 b/sql/hive/src/test/resources/golden/rename_column-18-fe4463a19f61099983f50bb51cfcd335 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f b/sql/hive/src/test/resources/golden/rename_column-19-70b42434913b9d2eb17cd216c4f8039f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a b/sql/hive/src/test/resources/golden/rename_column-2-b2b2dfa681d01296fdacb4f56fb6db3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 b/sql/hive/src/test/resources/golden/rename_column-20-f7eb4bd6f226be0c13117294be250271 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-21-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..726e0947d8302 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a b/sql/hive/src/test/resources/golden/rename_column-22-b2b2dfa681d01296fdacb4f56fb6db3a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-23-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..17127eaec9755 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 b/sql/hive/src/test/resources/golden/rename_column-24-e4bf0dd372b886b2afcca5b2dc089409 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-25-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..bfcefe4d18046 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd b/sql/hive/src/test/resources/golden/rename_column-26-89761e1c7afe3a5b9858f287cb808ccd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-27-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..c436c39a16b8a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 b/sql/hive/src/test/resources/golden/rename_column-28-59388d1eb6b5dc4e81a434bd59bf2cf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-29-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..bb1507e7488f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-3-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..17127eaec9755 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 b/sql/hive/src/test/resources/golden/rename_column-30-7ef160935cece55338bd4d52277b0203 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-31-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..500201be8d922 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 b/sql/hive/src/test/resources/golden/rename_column-32-379d54e3aa66daacff23c75007dfa008 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-33-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..71af16a5fa4d7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 b/sql/hive/src/test/resources/golden/rename_column-34-25bfcf66698b12f82903f72f13fea4e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-35-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..f5b2e72aeccab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-36-d032f4795c1186255acea241387adf93 b/sql/hive/src/test/resources/golden/rename_column-36-d032f4795c1186255acea241387adf93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-37-9c36cac1372650b703400c60dd29042c b/sql/hive/src/test/resources/golden/rename_column-37-9c36cac1372650b703400c60dd29042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 b/sql/hive/src/test/resources/golden/rename_column-4-e4bf0dd372b886b2afcca5b2dc089409 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-5-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..bfcefe4d18046 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd b/sql/hive/src/test/resources/golden/rename_column-6-89761e1c7afe3a5b9858f287cb808ccd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-7-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..c436c39a16b8a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 b/sql/hive/src/test/resources/golden/rename_column-8-59388d1eb6b5dc4e81a434bd59bf2cf4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b b/sql/hive/src/test/resources/golden/rename_column-9-6a3bbeb3203ce4df35275dccc4c4e37b new file mode 100644 index 0000000000000..bb1507e7488f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 b/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/rename_external_partition_location-0-5c73d46fb91e9d4b3dc916622df09290 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/router_join_ppr-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 b/sql/hive/src/test/resources/golden/router_join_ppr-1-f5d1802919e5f9c2afa606f509d8ff58 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 b/sql/hive/src/test/resources/golden/router_join_ppr-2-49b8b038ed8f5513405071c532967c47 new file mode 100644 index 0000000000000..3a57720041fb3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc b/sql/hive/src/test/resources/golden/router_join_ppr-3-a0b1014e173f410923de0d0446dfb4cc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 b/sql/hive/src/test/resources/golden/router_join_ppr-4-baaf33e5c11c65e1aa2b6be37de6eef6 new file mode 100644 index 0000000000000..19492fd335bcb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 b/sql/hive/src/test/resources/golden/router_join_ppr-5-89b4fc284984f51c12f15017707ad312 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f b/sql/hive/src/test/resources/golden/router_join_ppr-6-2edf8e4ca99543089fdacbf506caa94f new file mode 100644 index 0000000000000..3a57720041fb3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 b/sql/hive/src/test/resources/golden/router_join_ppr-7-defc6eae5ce9ac8bc9f57308e1513f93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e b/sql/hive/src/test/resources/golden/router_join_ppr-8-b07ad21f47a74162f438abf579675f8e new file mode 100644 index 0000000000000..19492fd335bcb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/sample2-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 b/sql/hive/src/test/resources/golden/sample2-1-a1d8184eab25b242a961533cc016efd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 b/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 new file mode 100644 index 0000000000000..0bc999a3e6aa2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/script_env_var1-0-16015162957e1d8e0ef586b44b276f64 @@ -0,0 +1,2 @@ +1 +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef b/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/script_env_var2-0-e5c4893b2ff412f9df0632328d404cef @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 b/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 new file mode 100644 index 0000000000000..0bc999a3e6aa2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/script_env_var2-1-81cb3e18ab89c533a253defff35e90f0 @@ -0,0 +1,2 @@ +1 +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 b/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/script_pipe-0-415536ae9ab41c3997f3f31cd52bcbb9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 b/sql/hive/src/test/resources/golden/script_pipe-1-a6827d31c2c855e36bd51a21a16aecf0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 b/sql/hive/src/test/resources/golden/script_pipe-2-3bf368261963ee10883e97c7746796b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 b/sql/hive/src/test/resources/golden/script_pipe-3-afe5db164ccf986c5badd0655e009ea1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f b/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f new file mode 100644 index 0000000000000..9cf5170e82d7d --- /dev/null +++ b/sql/hive/src/test/resources/golden/script_pipe-4-7fe60c2fcca928a497252d99436b513f @@ -0,0 +1 @@ +238 val_238 238 val_238 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/scriptfile1-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 b/sql/hive/src/test/resources/golden/scriptfile1-1-89cf592f66b7276add70eb2c7689aa34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 b/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/scriptfile1-2-b08adb4c792bd84b426a7f0bb9e835a5 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a b/sql/hive/src/test/resources/golden/select_as_omitted-0-39bd00e50d63ee4ff10427838591730a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 b/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 new file mode 100644 index 0000000000000..5f4de85940513 --- /dev/null +++ b/sql/hive/src/test/resources/golden/select_as_omitted-1-99d8c7e0fa02f47d19df6a0d7dabe145 @@ -0,0 +1 @@ +0 val_0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 b/sql/hive/src/test/resources/golden/select_unquote_and-0-7299c6986c06fc4a50cb543701527785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 b/sql/hive/src/test/resources/golden/select_unquote_and-1-24a5b4c39dbdbfad75c08f201f2b7a70 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 b/sql/hive/src/test/resources/golden/select_unquote_and-2-8b4aac4293f2092c0661c26cb54b5619 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e new file mode 100644 index 0000000000000..f35d9602e1a7b --- /dev/null +++ b/sql/hive/src/test/resources/golden/select_unquote_and-3-683007138a712792041ef3c8b84e914e @@ -0,0 +1 @@ +498 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a b/sql/hive/src/test/resources/golden/select_unquote_and-4-8417a7b93132e20d468b7948bf46915a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 b/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/select_unquote_and-5-1a5a68a098bfb9c93b76a458a9faf232 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 b/sql/hive/src/test/resources/golden/select_unquote_and-6-f34f161ad7eda65e2640c1d3ae47cb09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 b/sql/hive/src/test/resources/golden/select_unquote_not-0-7299c6986c06fc4a50cb543701527785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 b/sql/hive/src/test/resources/golden/select_unquote_not-1-24a5b4c39dbdbfad75c08f201f2b7a70 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 b/sql/hive/src/test/resources/golden/select_unquote_not-2-8b4aac4293f2092c0661c26cb54b5619 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e new file mode 100644 index 0000000000000..f35d9602e1a7b --- /dev/null +++ b/sql/hive/src/test/resources/golden/select_unquote_not-3-683007138a712792041ef3c8b84e914e @@ -0,0 +1 @@ +498 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 b/sql/hive/src/test/resources/golden/select_unquote_not-4-a6cbbf81715845f61af8fce977c0bcd5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde b/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/select_unquote_not-5-ceada0d3fa65bb2ec65154e797332cde @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 b/sql/hive/src/test/resources/golden/select_unquote_not-6-f34f161ad7eda65e2640c1d3ae47cb09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 b/sql/hive/src/test/resources/golden/select_unquote_or-0-7299c6986c06fc4a50cb543701527785 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 b/sql/hive/src/test/resources/golden/select_unquote_or-1-24a5b4c39dbdbfad75c08f201f2b7a70 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 b/sql/hive/src/test/resources/golden/select_unquote_or-2-8b4aac4293f2092c0661c26cb54b5619 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e b/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e new file mode 100644 index 0000000000000..f35d9602e1a7b --- /dev/null +++ b/sql/hive/src/test/resources/golden/select_unquote_or-3-683007138a712792041ef3c8b84e914e @@ -0,0 +1 @@ +498 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 b/sql/hive/src/test/resources/golden/select_unquote_or-4-954f604d17e36bbc08c04a0571d33045 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f b/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/select_unquote_or-5-55cd874f705673f9de6ec8e3643c760f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 b/sql/hive/src/test/resources/golden/select_unquote_or-6-f34f161ad7eda65e2640c1d3ae47cb09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 b/sql/hive/src/test/resources/golden/serde_reported_schema-0-765ac345199b944a6136273becf29469 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 b/sql/hive/src/test/resources/golden/serde_reported_schema-1-e8c6de8cd50be582d78c9a8244cd92a1 new file mode 100644 index 0000000000000..d1168556e09d4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d b/sql/hive/src/test/resources/golden/serde_reported_schema-2-4da45ecbf2b2c9c1cb3ea05cb608710d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa b/sql/hive/src/test/resources/golden/serde_reported_schema-3-738e1d72a19c3db37ded87ca2fb148fa new file mode 100644 index 0000000000000..0b8f428b24193 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d b/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/set_variable_sub-0-b0b2ec9de0599f8dc2eb160299a2699d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 b/sql/hive/src/test/resources/golden/set_variable_sub-1-e504c8259e45911765541f605c9a46f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 b/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/set_variable_sub-10-bf5d1e710ce0974a67b4084aaa868f67 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 b/sql/hive/src/test/resources/golden/set_variable_sub-2-5b8cd23606e1a38f61eb6410ac6a09d6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 b/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/set_variable_sub-3-266170978f88a14c20c3944bfb55f5c7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 b/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/set_variable_sub-4-3839b7ac878ed6bdd4f9d242f965ded7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd b/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/set_variable_sub-5-ee3a31bb9bb742f90daea98b290c34cd @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a b/sql/hive/src/test/resources/golden/set_variable_sub-6-ee7a7e3d17bb0009f4cf3ffb001b471a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b b/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/set_variable_sub-7-e25bdc67a6d9ea3f5d65676a92d1217b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d b/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/set_variable_sub-8-f321516e93eb0206453465a9b85cf67d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 b/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/set_variable_sub-9-fbb54d457caeaafce723856429bbc0b2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 new file mode 100644 index 0000000000000..fd33cfcc9ab20 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-0-65fee14fcf58502241f0772b21096780 @@ -0,0 +1 @@ +concat \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d b/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d new file mode 100644 index 0000000000000..fd33cfcc9ab20 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_describe_func_quotes-1-26b98b2901556449d5431d731aaa642d @@ -0,0 +1 @@ +concat \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-2-f795383fcecedf7266cd7aed8283cec3 new file mode 100644 index 0000000000000..a3998eeeee623 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/show_describe_func_quotes-3-7bc53505a4e6587132870d8d0a704d2 new file mode 100644 index 0000000000000..a3998eeeee623 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 b/sql/hive/src/test/resources/golden/show_functions-0-45a7762c39f1b0f26f076220e2764043 new file mode 100644 index 0000000000000..9d40ffaef5862 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 b/sql/hive/src/test/resources/golden/show_functions-1-4a6f611305f58bdbafb2fd89ec62d797 new file mode 100644 index 0000000000000..c62b965cb1559 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c b/sql/hive/src/test/resources/golden/show_functions-2-97cbada21ad9efda7ce9de5891deca7c new file mode 100644 index 0000000000000..a56b5a3766c5c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 b/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 new file mode 100644 index 0000000000000..86605075c3d25 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_functions-3-86945c60aed23626c43b507ee4ee6049 @@ -0,0 +1,3 @@ +log +log10 +log2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 b/sql/hive/src/test/resources/golden/show_functions-4-4deaa213aff83575bbaf859f79bfdd48 new file mode 100644 index 0000000000000..312f6cdbf68c7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 b/sql/hive/src/test/resources/golden/show_functions-5-d6dad716f80a2e99ef9fc2e9782ef138 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-0-9acff7f5096cdafc92aa2ddb0f296f83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-1-3f42728fb0083beb962f07c43dd9c9b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-10-d759a63f08c878456c3401626f253ff5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-11-afe66851d1cdc5d8a8a0d21c5705a59e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-12-f2dd38f0a56cd2137c9e1b870271550b new file mode 100644 index 0000000000000..e772f4a83fecd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-13-a9224a7a0012e407da67041bf680d490 new file mode 100644 index 0000000000000..d68fbfc9c1e0f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-14-9d121385c5ab32d659dc7d0374ae8d6e new file mode 100644 index 0000000000000..94ee57be9dcaf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-15-b032f4869c43d1278a890328d0591d5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-16-8b2dc53c795584e1e23a8d631c82b43f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-17-c93fd07893f47b712165725c78d95555 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-18-9acff7f5096cdafc92aa2ddb0f296f83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-19-3f42728fb0083beb962f07c43dd9c9b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-2-e90c14f9edaeab8cf4540e9a35b11546 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-3-aa5935155586821fb35e17156c8d8460 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-4-6eb587e2751942de625c9229872ca0dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-5-ad307c2c7edb4bb2e6c34ef1eb7b47f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-6-ae97a64481efe733a19007ed400925bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-7-34016fb262ce768de45ec1b7693fd6c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-8-16d39297488db165145e1546c4cb222c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 b/sql/hive/src/test/resources/golden/show_indexes_edge_cases-9-fba02256147a1a753d40f56825449471 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec b/sql/hive/src/test/resources/golden/show_indexes_syntax-0-b6a94a6f5b3938d53ddf419ff97a87ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 b/sql/hive/src/test/resources/golden/show_indexes_syntax-1-fe3da04846e702e0bbe22e05b136a3b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec b/sql/hive/src/test/resources/golden/show_indexes_syntax-10-b6a94a6f5b3938d53ddf419ff97a87ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 b/sql/hive/src/test/resources/golden/show_indexes_syntax-2-8207d7ca034ed62b9fb2c727497c22b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 b/sql/hive/src/test/resources/golden/show_indexes_syntax-3-d687cc8f5d39bfbad3537a6e92788bb7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 b/sql/hive/src/test/resources/golden/show_indexes_syntax-4-187e7dfb63d6b70982db8b4ddeb82ed7 new file mode 100644 index 0000000000000..7e68a8acb1a87 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 b/sql/hive/src/test/resources/golden/show_indexes_syntax-5-47d09f8540512a5f3e21a3e1d4fd2d49 new file mode 100644 index 0000000000000..36d22451eba3e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 b/sql/hive/src/test/resources/golden/show_indexes_syntax-6-de64752733e0dcae32e692d2ad38e3d4 new file mode 100644 index 0000000000000..7e68a8acb1a87 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 b/sql/hive/src/test/resources/golden/show_indexes_syntax-7-991839c8e50c5b4e490ec7faf2228d58 new file mode 100644 index 0000000000000..36d22451eba3e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e b/sql/hive/src/test/resources/golden/show_indexes_syntax-8-8c10f994f51bce851ecb0acee65ced7e new file mode 100644 index 0000000000000..4dddeee9a233f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 b/sql/hive/src/test/resources/golden/show_indexes_syntax-9-f8385127c6405a2c9e48b5988184b515 new file mode 100644 index 0000000000000..76e0434294b4f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a b/sql/hive/src/test/resources/golden/show_partitions-0-73d0fdcea0bd9b828cbc3c2e88acb51a new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 b/sql/hive/src/test/resources/golden/show_partitions-1-e94d4100cb64c67f1127b4e255d28ae0 new file mode 100644 index 0000000000000..19b4a62499762 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca b/sql/hive/src/test/resources/golden/show_partitions-2-a1bde7c2c040b4d45ddceac9983c2ca new file mode 100644 index 0000000000000..f3614273fa8fa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 b/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 new file mode 100644 index 0000000000000..0cdd3e8594c59 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_partitions-3-9e3f80cb6ed9883c715ea8aa3f391d70 @@ -0,0 +1 @@ +ds=2008-04-08/hr=12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 b/sql/hive/src/test/resources/golden/show_tables-0-679cc07f8475a20b49927a5bbbd3d702 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce b/sql/hive/src/test/resources/golden/show_tables-1-ac1c8cca812353544d3f7dead5d033ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 b/sql/hive/src/test/resources/golden/show_tables-10-643b24446d74450c0f83144b1d0ec433 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 b/sql/hive/src/test/resources/golden/show_tables-11-3f9a7f993510123059493826470f78f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c b/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c new file mode 100644 index 0000000000000..60f7943eda4a9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_tables-12-c31d2f4105ec3714cfc55eef68d3f60c @@ -0,0 +1,3 @@ +bar +baz +foo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 b/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 new file mode 100644 index 0000000000000..4ffc580e2b8f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_tables-13-f72d7ab6f3f04161ce2b8f8332244657 @@ -0,0 +1,4 @@ +shtb_test1 +shtb_test2 +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c b/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c new file mode 100644 index 0000000000000..60f7943eda4a9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_tables-14-26ca9b5509544ebac182d8aa4264ff1c @@ -0,0 +1,3 @@ +bar +baz +foo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe b/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe new file mode 100644 index 0000000000000..4ffc580e2b8f3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_tables-15-72a95694f749cb3d5760a69083e9cafe @@ -0,0 +1,4 @@ +shtb_test1 +shtb_test2 +src +srcpart \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 b/sql/hive/src/test/resources/golden/show_tables-16-dfd802554b6764b3105b6fd8dbb8e173 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab b/sql/hive/src/test/resources/golden/show_tables-17-49777c49d2627373ed5e459c4848c9ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 b/sql/hive/src/test/resources/golden/show_tables-18-c22c5af6ef0bcb03fdafe3c4df34ec93 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 b/sql/hive/src/test/resources/golden/show_tables-19-695a68c82308540eba1d0a04e032cf39 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 b/sql/hive/src/test/resources/golden/show_tables-2-c96604d76bcb3721d5e5a327cac0d5e5 new file mode 100644 index 0000000000000..916ac1482c061 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 b/sql/hive/src/test/resources/golden/show_tables-20-691b4e6664e6d435233ea4e8c3b585d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/show_tables-21-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 b/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 new file mode 100644 index 0000000000000..19102815663d2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_tables-22-274454ebeb7f98690a3c152617a0e391 @@ -0,0 +1 @@ +foo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c b/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c new file mode 100644 index 0000000000000..b67b816ee4b45 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_tables-3-a56f51be557c6f636f99fafdbbbbcd9c @@ -0,0 +1,2 @@ +shtb_test1 +shtb_test2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 b/sql/hive/src/test/resources/golden/show_tables-4-743d585ec3da5fbb927581cd0683ae35 new file mode 100644 index 0000000000000..8f06e234b2a6e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d b/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d new file mode 100644 index 0000000000000..b67b816ee4b45 --- /dev/null +++ b/sql/hive/src/test/resources/golden/show_tables-5-c685b358b604bf3ef980a78d9178d87d @@ -0,0 +1,2 @@ +shtb_test1 +shtb_test2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 b/sql/hive/src/test/resources/golden/show_tables-6-1086ed68a5cf2540a72b3e949b9ea5f8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c b/sql/hive/src/test/resources/golden/show_tables-7-a62fc229d241303bffb29b34ad125f8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 b/sql/hive/src/test/resources/golden/show_tables-8-691b4e6664e6d435233ea4e8c3b585d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 b/sql/hive/src/test/resources/golden/show_tables-9-64c9bf0618541518f2ba30ec24a94423 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa b/sql/hive/src/test/resources/golden/show_tablestatus-0-4fa957197c8678b0a3a64d8f4f6da1fa new file mode 100644 index 0000000000000..bec424bb026e9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 b/sql/hive/src/test/resources/golden/show_tablestatus-1-4c31924711bdb64603a14ce57da86ab7 new file mode 100644 index 0000000000000..9392b7dc686f6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 b/sql/hive/src/test/resources/golden/show_tablestatus-2-ecddce523f2af516700677a051581330 new file mode 100644 index 0000000000000..9392b7dc686f6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d b/sql/hive/src/test/resources/golden/show_tablestatus-3-dccfbc8b5a223a9fe47120ca771ee61d new file mode 100644 index 0000000000000..9392b7dc686f6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 b/sql/hive/src/test/resources/golden/show_tablestatus-4-5208ae4e509cb7f10dd4e0d29b5ab346 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 b/sql/hive/src/test/resources/golden/show_tablestatus-5-5bd2196d71aa5308a5d01d9e9df3f59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 b/sql/hive/src/test/resources/golden/show_tablestatus-6-d5e2f1950de4bf0ff77b7c66cddf3eb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c b/sql/hive/src/test/resources/golden/show_tablestatus-7-f7b9148c16045269344c5d74fb8a449c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 b/sql/hive/src/test/resources/golden/show_tablestatus-8-1cd5acb4091d916e5e18948a39979b51 new file mode 100644 index 0000000000000..f8b64f6056ea7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 b/sql/hive/src/test/resources/golden/show_tablestatus-9-e3cc1823000abb51d2635e0c824e81a9 new file mode 100644 index 0000000000000..9392b7dc686f6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 b/sql/hive/src/test/resources/golden/showparts-0-593619bb962b318b82896658deaea1f1 new file mode 100644 index 0000000000000..b590724bca78d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a b/sql/hive/src/test/resources/golden/showparts-1-73d0fdcea0bd9b828cbc3c2e88acb51a new file mode 100644 index 0000000000000..e9c723bbd136e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-0-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-1-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-10-fa00cf008a039908eec64ad5dd415c5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-11-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-12-dd683e148baed6b27642eebacaa87a4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-3-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-4-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-8-daf10744f465e055b35809a528135370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_1-9-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-0-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-1-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-10-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-11-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-12-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-13-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-14-bf8bd6dbf9485c05f8fd4f84e2530724 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-3-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-4-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-8-c64266431d312784ebc7b9ca07ab5188 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoin_union_remove_2-9-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt1-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt1-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt1-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt1-3-9669bca0e2da11221b2e9eb21322e0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt1-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt1-5-fa00cf008a039908eec64ad5dd415c5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt1-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 b/sql/hive/src/test/resources/golden/skewjoinopt1-7-f66c3935651d3cc5fef7d0284e123614 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt10-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt10-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt10-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 b/sql/hive/src/test/resources/golden/skewjoinopt10-3-27fc8f7d7456a761e1d0c2c075b84dc6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt10-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a b/sql/hive/src/test/resources/golden/skewjoinopt10-5-9abe9cb11e3336a689a4116f8804c02a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 b/sql/hive/src/test/resources/golden/skewjoinopt10-6-bc7008e74f5eccab48d820877d83e7e4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 b/sql/hive/src/test/resources/golden/skewjoinopt10-7-f4015c993efd5fc155e2faec784718d0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 b/sql/hive/src/test/resources/golden/skewjoinopt10-8-c9624d1650d395f18d9e510cab4fab79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt11-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt11-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt11-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 b/sql/hive/src/test/resources/golden/skewjoinopt11-3-a079ede4f4245e62b02f624efedfb597 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt11-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt11-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt11-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 b/sql/hive/src/test/resources/golden/skewjoinopt11-7-9e7e6016590d33c617cb568cbd45ef68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt12-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt12-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt12-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 b/sql/hive/src/test/resources/golden/skewjoinopt12-3-da45999e75a09b27161af2c7414c1170 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt12-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd b/sql/hive/src/test/resources/golden/skewjoinopt12-5-e6db5d1ec4694475ab0b8f43eba415cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt12-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 b/sql/hive/src/test/resources/golden/skewjoinopt12-7-8bbc680be8a68053db008789f335c2f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 b/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 new file mode 100644 index 0000000000000..872146532307a --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-10-b81a7fa7b6158dd4d77fa4c62db1f223 @@ -0,0 +1 @@ +2 12 2 22 2 12 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt13-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/skewjoinopt13-3-caf1c5fd299fdbdb655234d01d44caf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt13-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt13-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt13-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 b/sql/hive/src/test/resources/golden/skewjoinopt13-7-c329b937ad1d7cf1c838640ef5f4d135 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt13-8-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d b/sql/hive/src/test/resources/golden/skewjoinopt13-9-ab816627b2137d1850cf05b6f2b59e2d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt14-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt14-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt14-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt14-3-9669bca0e2da11221b2e9eb21322e0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt14-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt14-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt14-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 b/sql/hive/src/test/resources/golden/skewjoinopt14-7-c329b937ad1d7cf1c838640ef5f4d135 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt14-8-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 b/sql/hive/src/test/resources/golden/skewjoinopt14-9-cdf19a17f3295447b66e6e6445742a74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt15-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt15-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 b/sql/hive/src/test/resources/golden/skewjoinopt15-10-7df9fe6075a96bf9849848c93e449469 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 b/sql/hive/src/test/resources/golden/skewjoinopt15-11-3f5ba247cb51c79bacdd56ef3ecbb601 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt15-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 b/sql/hive/src/test/resources/golden/skewjoinopt15-3-32fed3a53e7c15b549a71c0e71d93484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/sql/hive/src/test/resources/golden/skewjoinopt15-4-27dc133d5ad9806d0c8ff0ebf8f9a469 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 b/sql/hive/src/test/resources/golden/skewjoinopt15-5-dff9d122eb83760f08d1d77814c24c91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d b/sql/hive/src/test/resources/golden/skewjoinopt15-6-717b85f496a5cf006cb352f9d884608d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 b/sql/hive/src/test/resources/golden/skewjoinopt15-7-ba43a86694107dd4cb754d676935408 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 b/sql/hive/src/test/resources/golden/skewjoinopt15-8-7381c1f36c997afac91d8f8f29e865f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e b/sql/hive/src/test/resources/golden/skewjoinopt15-9-ccb54093d518eaca1e4644686e9e776e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt16-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt16-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt16-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt16-3-25f94adcba19b899d1db3af93ea1c95b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt16-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt16-5-fa00cf008a039908eec64ad5dd415c5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt16-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 b/sql/hive/src/test/resources/golden/skewjoinopt16-7-b3208400a48223a410b76a4bd1362da5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt17-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt17-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt17-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt17-3-25f94adcba19b899d1db3af93ea1c95b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt17-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 b/sql/hive/src/test/resources/golden/skewjoinopt17-5-2e4b9b1d820a7ef31c51bd4fd2cc28f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt17-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 b/sql/hive/src/test/resources/golden/skewjoinopt17-7-373b8a6363139ca37395b1cc8c9808d9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa b/sql/hive/src/test/resources/golden/skewjoinopt18-10-8c180ce74ca1ac0eefab2d70d38e44fa new file mode 100644 index 0000000000000..ee1bb6b112381 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt18-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 b/sql/hive/src/test/resources/golden/skewjoinopt18-3-32fed3a53e7c15b549a71c0e71d93484 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 b/sql/hive/src/test/resources/golden/skewjoinopt18-4-27dc133d5ad9806d0c8ff0ebf8f9a469 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 b/sql/hive/src/test/resources/golden/skewjoinopt18-5-dff9d122eb83760f08d1d77814c24c91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d b/sql/hive/src/test/resources/golden/skewjoinopt18-6-717b85f496a5cf006cb352f9d884608d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 b/sql/hive/src/test/resources/golden/skewjoinopt18-7-41ca600110e24166325d9426d974fff7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt18-8-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 b/sql/hive/src/test/resources/golden/skewjoinopt18-9-ae751060e656d20aa1afbe3fac16d304 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt19-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt19-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt19-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e b/sql/hive/src/test/resources/golden/skewjoinopt19-3-6eda8acf464a18cfd9909255ddcef37e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt19-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt19-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt19-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 b/sql/hive/src/test/resources/golden/skewjoinopt19-7-1e9c17669089eea559f8fa0b6977b249 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt2-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt2-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt2-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d b/sql/hive/src/test/resources/golden/skewjoinopt2-3-52247b4dd98092bf829254e17424657d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt2-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt2-5-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt2-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 b/sql/hive/src/test/resources/golden/skewjoinopt2-7-2a2e8dfb78c2dfcba51a4cf91da25ae4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt20-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt20-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt20-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 b/sql/hive/src/test/resources/golden/skewjoinopt20-3-4420d88e35aa84327bc95153eed299e0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt20-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt20-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt20-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb b/sql/hive/src/test/resources/golden/skewjoinopt20-7-e209254ae404366e6adca673d666aecb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt3-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt3-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt3-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoinopt3-3-c64266431d312784ebc7b9ca07ab5188 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt3-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt3-5-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt3-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f b/sql/hive/src/test/resources/golden/skewjoinopt3-7-e54bbab48fcc3c41806a101293577e9f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt4-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt4-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt4-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt4-3-9669bca0e2da11221b2e9eb21322e0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt4-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt4-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt4-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 b/sql/hive/src/test/resources/golden/skewjoinopt4-7-c7e2ccaba57a581f27cfdcca76891133 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt5-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt5-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt5-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b b/sql/hive/src/test/resources/golden/skewjoinopt5-3-25f94adcba19b899d1db3af93ea1c95b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt5-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b b/sql/hive/src/test/resources/golden/skewjoinopt5-5-fa00cf008a039908eec64ad5dd415c5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt5-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d b/sql/hive/src/test/resources/golden/skewjoinopt5-7-f38878761b2aeeee0c04387dff60894d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt6-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt6-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt6-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 b/sql/hive/src/test/resources/golden/skewjoinopt6-3-da45999e75a09b27161af2c7414c1170 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt6-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd b/sql/hive/src/test/resources/golden/skewjoinopt6-5-e6db5d1ec4694475ab0b8f43eba415cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt6-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 b/sql/hive/src/test/resources/golden/skewjoinopt6-7-4dd78e79dc6ccab0cf472c8745d1f384 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt7-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt7-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt7-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 b/sql/hive/src/test/resources/golden/skewjoinopt7-3-c64266431d312784ebc7b9ca07ab5188 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt7-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt7-5-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt7-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoinopt7-7-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt7-8-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c b/sql/hive/src/test/resources/golden/skewjoinopt7-9-b54d2a1f5d3bea81680ab06dead952c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt8-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt8-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt8-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 b/sql/hive/src/test/resources/golden/skewjoinopt8-3-caf1c5fd299fdbdb655234d01d44caf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt8-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 b/sql/hive/src/test/resources/golden/skewjoinopt8-5-bebf0a312f3110d0b518153543030f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt8-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e b/sql/hive/src/test/resources/golden/skewjoinopt8-7-fa53198d9eecb9d274b09d4351b9274e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 b/sql/hive/src/test/resources/golden/skewjoinopt8-8-3fda17e4414d191f837631438a19e700 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd b/sql/hive/src/test/resources/golden/skewjoinopt8-9-9b26e8e33d3109e059e7494b53aee6fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-0-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 b/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-1-16a1f74642d7ea4dac66a5ce15469c22 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 b/sql/hive/src/test/resources/golden/skewjoinopt9-10-5c8be465ceef8151f172b82a13e81fa8 new file mode 100644 index 0000000000000..acd4039d35669 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d b/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/skewjoinopt9-2-114600d46ae86edcb66a500b4cac657d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 b/sql/hive/src/test/resources/golden/skewjoinopt9-3-9669bca0e2da11221b2e9eb21322e0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b b/sql/hive/src/test/resources/golden/skewjoinopt9-4-b76bf9f6c92f83c9a5f351f8460d1e3b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 b/sql/hive/src/test/resources/golden/skewjoinopt9-5-744a018c78bae6e09853dd202981e850 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 b/sql/hive/src/test/resources/golden/skewjoinopt9-6-4abc4f450a58ccdd0df2e345f1276979 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 b/sql/hive/src/test/resources/golden/skewjoinopt9-7-350dd4f731e2c4157f83c148c8569203 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d b/sql/hive/src/test/resources/golden/skewjoinopt9-8-446c5e33062b109341add64a9860207d new file mode 100644 index 0000000000000..f608d53f51ebe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 b/sql/hive/src/test/resources/golden/skewjoinopt9-9-83df7254049bc14de8aa53756850ceb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 b/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 new file mode 100644 index 0000000000000..7a442f02e8d7a --- /dev/null +++ b/sql/hive/src/test/resources/golden/small.cartesian-0-e3c7f62795b0e7c14b41b0dc29c47619 @@ -0,0 +1,3 @@ +0 2 +0 2 +0 2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a b/sql/hive/src/test/resources/golden/smb_mapjoin_1-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec b/sql/hive/src/test/resources/golden/smb_mapjoin_1-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 new file mode 100644 index 0000000000000..836ee718649ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-10-b1114520feaa15ad7621b6a0e571c244 @@ -0,0 +1 @@ +51 val_3 51 val_30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed b/sql/hive/src/test/resources/golden/smb_mapjoin_1-11-c9a5aea60b50d31499dc8d8516b3fed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa b/sql/hive/src/test/resources/golden/smb_mapjoin_1-12-8fb6fea440e845ce23b06deed8f519fa new file mode 100644 index 0000000000000..8f358bffec51d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-13-7dc074e59f87c9e10b552c86eef9c96 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-14-b9e32ef49286a471ae2c4e7e66a2f5e1 new file mode 100644 index 0000000000000..6197cf72c3454 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c b/sql/hive/src/test/resources/golden/smb_mapjoin_1-15-4d7cd71c68715335030bd41244a9cb2c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-16-c120e505c143721a36287bf992dbc1a1 new file mode 100644 index 0000000000000..d650d44f42404 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-17-16c336c72785228fa1e6fcda01b20550 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 new file mode 100644 index 0000000000000..836ee718649ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-18-bcd8f7a7a4a77c8d6a39e38b93e5c5a1 @@ -0,0 +1 @@ +51 val_3 51 val_30 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-19-9597d17924805ff4d34398d3a3d0a097 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f b/sql/hive/src/test/resources/golden/smb_mapjoin_1-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-20-d7979e9ab355d8856c2d12e66e7bb838 new file mode 100644 index 0000000000000..8f358bffec51d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec b/sql/hive/src/test/resources/golden/smb_mapjoin_1-21-bce103f1a686915c28581717387cfeec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b b/sql/hive/src/test/resources/golden/smb_mapjoin_1-22-df6bdfe3c5a7927687f4d1fbf0c9c25b new file mode 100644 index 0000000000000..6197cf72c3454 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd b/sql/hive/src/test/resources/golden/smb_mapjoin_1-23-aef0ab90470007f2ae589ca5dddd02fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae b/sql/hive/src/test/resources/golden/smb_mapjoin_1-24-558e9ab6324f8082231b2fdd0e12f7ae new file mode 100644 index 0000000000000..d650d44f42404 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_1-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_1-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea b/sql/hive/src/test/resources/golden/smb_mapjoin_1-9-17098316902a9e99455fa01bd3dae3ea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-0-a2d52f742b171d642bc7b8ec244791c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-1-9385271b1309ae1609b41d8a30dbef24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-10-189aeb9104e89114be759634d51c1a98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-2-de26fe3669e18e6a170da561687fb514 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-3-bfb76fa2eccda1c64a85ea3841202705 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-4-d4746bf376bce0bf561450c75b83fb74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-5-f05f1ef1d08dbe6b02139fe9d0a68ed8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-6-1094dbf800159e1e2382ec238b2466d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_10-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa b/sql/hive/src/test/resources/golden/smb_mapjoin_11-10-3d92573eecd22525a75464ad27b1dcaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-11-685ec4fbbf73330e026fba9b8cc53e92 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-12-c05c09243793df14546e8577ee369d58 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-13-1e0f21b664c1940d10531b3025be7f10 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-14-f13be826d8f11be64c5414d591f70fd6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-15-b62714cb184523454314d19949dba9f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af b/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-16-bee943a85cd82bd199b089fbdd7450af @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_11-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f b/sql/hive/src/test/resources/golden/smb_mapjoin_11-8-beae7266b997c97798631b9dc558534f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 b/sql/hive/src/test/resources/golden/smb_mapjoin_11-9-74936eafc274242beb49bc83d7a8af30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-10-bc8140f238cfd13ea4fc4c4173a05454 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-11-685ec4fbbf73330e026fba9b8cc53e92 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-12-c05c09243793df14546e8577ee369d58 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb b/sql/hive/src/test/resources/golden/smb_mapjoin_12-13-7e7645d5ee7d79991d8fdde072c8dbb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-14-e9715c66355e9bc54155c79a4e82b34f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-15-42b623410c408e09153a773db91c0334 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af b/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af new file mode 100644 index 0000000000000..8975db9a05036 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-16-bee943a85cd82bd199b089fbdd7450af @@ -0,0 +1 @@ +293 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-17-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-18-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-19-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b b/sql/hive/src/test/resources/golden/smb_mapjoin_12-20-1fc1f40252a7e9d2ad5133f358b71f6b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-21-3814ec07d1b074eb0fc44e8f77d1f40e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e new file mode 100644 index 0000000000000..8975db9a05036 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-22-b1e1754efd667aa801b194985d41eb6e @@ -0,0 +1 @@ +293 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_12-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-8-3d016b9a5b0143b7a01b34472b569fb9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 b/sql/hive/src/test/resources/golden/smb_mapjoin_12-9-74936eafc274242beb49bc83d7a8af30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-10-b07488dd0aa9072057d8264271e93495 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-11-c142d1f47152f247d4709b37780c46c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-12-fb171dfe3de091a91e347f53cf68c355 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee b/sql/hive/src/test/resources/golden/smb_mapjoin_13-13-7b9c311217809cbb45a63be3c3e934ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-14-b92cb43f66838319f5d607c431fe1eb3 new file mode 100644 index 0000000000000..b828077157966 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-15-b80660184510c5a283b3c10dcddb9e69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-16-a05f3678364039a4cec1ced0dbda9884 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_13-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-8-721a75e4942a8ea97de6d70e441ce8a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 b/sql/hive/src/test/resources/golden/smb_mapjoin_13-9-71b57a524e9ff17058ab319b18c606f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd b/sql/hive/src/test/resources/golden/smb_mapjoin_14-10-fd4343f466f65fe84350a89ea501a8cd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d b/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-11-23d2ee09b01309b9cfcd0d625afc535d @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-12-2037b594916a80436d571a87d7b5f2e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a b/sql/hive/src/test/resources/golden/smb_mapjoin_14-13-95a09a0af2a77ac6b772c41a0f6a885a new file mode 100644 index 0000000000000..4a9735f855f96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d b/sql/hive/src/test/resources/golden/smb_mapjoin_14-14-5db39db7bbdeae2d87a2dc1d65e6cd1d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 new file mode 100644 index 0000000000000..62f9457511f87 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-15-e0f20915e07acf5ddfdbde40ef924e55 @@ -0,0 +1 @@ +6 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-16-1ae53bf26065f6ae200bd3f8f5702fa2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee b/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-17-5983d1e12e5a2bdd0f41549110e066ee @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-18-c58871e5f7179cd903e24ad22198dea9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-19-163af33279f8d08d747a00ffd1cdfac7 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a b/sql/hive/src/test/resources/golden/smb_mapjoin_14-20-b1651139605264c3b15178ae0543f7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-21-b6c3a3f68f212a966450286e23f59713 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b b/sql/hive/src/test/resources/golden/smb_mapjoin_14-22-f7839f8bcbb99d5731c9d4174c76b4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-23-941d6ef1aaf1b2f16cf9b55eaea49068 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-24-46db62b188fef9c6d636e63d64ef0c20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e b/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-25-23f011143d8db18dd0f3d684adf7b8e @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b b/sql/hive/src/test/resources/golden/smb_mapjoin_14-26-8dd1b03e8d405eb4c0c3a8bba78fca1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-27-ba437062dd6661fc2fdcd41514711093 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-28-db27758cc8bdb4ff80f346ad7e158da3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b b/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-29-d191c9ace84072ef430d7ed36ea8181b @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da b/sql/hive/src/test/resources/golden/smb_mapjoin_14-3-4938d4d724990d16336ee31f0390c7da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb b/sql/hive/src/test/resources/golden/smb_mapjoin_14-30-377c21706143c1b714eeb0b3634343fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 new file mode 100644 index 0000000000000..2ebc6516c7df1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-31-fde2ad19052435bd8c155f18fe579070 @@ -0,0 +1 @@ +56 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-32-edfe76bb21077fec096eb60333cec7f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 new file mode 100644 index 0000000000000..2edeafb09db00 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-33-fb532dd5af8cfe6b2af5c4752a3b0a44 @@ -0,0 +1 @@ +20 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-4-378d42317b39c6519f15bd2f99c5ddc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-5-9140b367b5680860f4c7c0238377583f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-6-70c1d66123d434d3a1e1801e0b19bc3f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_14-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b b/sql/hive/src/test/resources/golden/smb_mapjoin_15-10-fb63a2ba41589835c01ba5f7570f643b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-11-a8c3244fbb17a900a1bd7f634c90c7e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a b/sql/hive/src/test/resources/golden/smb_mapjoin_15-12-b5a588fb78fda8e3c41bbc4d973c1a7a new file mode 100644 index 0000000000000..b828077157966 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-13-5983712d9bdc150d8b1ffe23a555b370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-14-5455eb67634b186d7d8e54cdd32a9909 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-15-268af28b787b5403a164447cbdbb0720 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-16-98537b81d6e139521541753c3c448404 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-17-c2312024b8de8d40faaf65427fa3d499 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c b/sql/hive/src/test/resources/golden/smb_mapjoin_15-18-7b1293b3066a5076e91b3d3dfd74259c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-19-f49ef85423bb1766362f05651c9bb37f new file mode 100644 index 0000000000000..471d725e7bfa3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-20-7577942157bcb84ad24c51306b046a44 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e b/sql/hive/src/test/resources/golden/smb_mapjoin_15-21-af3880637379684acd440830c2361f6e new file mode 100644 index 0000000000000..471d725e7bfa3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-22-55700b43b9e34f4b8370f76d1b1bb7a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-23-e5e54dd17b69773812af376bfec33200 new file mode 100644 index 0000000000000..471d725e7bfa3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-24-5983712d9bdc150d8b1ffe23a555b370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-25-5455eb67634b186d7d8e54cdd32a9909 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_15-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb b/sql/hive/src/test/resources/golden/smb_mapjoin_15-8-c19c008a41a546848f30a27237b124eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 b/sql/hive/src/test/resources/golden/smb_mapjoin_15-9-c11393a5b3fa2eba6e5f388e37e31cd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b b/sql/hive/src/test/resources/golden/smb_mapjoin_16-10-fb63a2ba41589835c01ba5f7570f643b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c b/sql/hive/src/test/resources/golden/smb_mapjoin_16-11-3c64de88fd6329a4346706ea296f7a4c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf b/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf new file mode 100644 index 0000000000000..67d892c80f493 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-12-32f3716e22149e3d0c1f3ac26d414baf @@ -0,0 +1 @@ +1028 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_16-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 b/sql/hive/src/test/resources/golden/smb_mapjoin_16-8-14c221da1d1be2208aec9a641351ed09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e b/sql/hive/src/test/resources/golden/smb_mapjoin_16-9-6223202c291504ede983d8b04eafe08e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-10-516ad5af165e0e93b06b7541d35c96b8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-11-c4daf31f90b01cfac9fb03b8b9de042 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-12-cc5a32c1984378efafe0f05202359b20 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-13-98da74d8dc8c7b722de5d2e25b481581 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-14-672606a44ad31bacc472e7e2825d4fd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-15-39292eb609c10f05f5c3dfacf520f485 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-16-da60c2ca6af8844ed44778f4067f4f27 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-17-bfc16d69eac4464e094e416842550590 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-18-cf2846a07aab193ac997eae616373367 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-19-c11dfa50e4541ba7081fd3c57e0bef56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-2-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c b/sql/hive/src/test/resources/golden/smb_mapjoin_17-20-50d120ef58fd6b737a01fccb3f212e9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f b/sql/hive/src/test/resources/golden/smb_mapjoin_17-21-b1069c2d3f38f78fa5c5635a74b4466f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-22-12d1838d39f395dbb212f322e83c49b0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e b/sql/hive/src/test/resources/golden/smb_mapjoin_17-23-e366aaa64bcf26a714cc9a451a6e8d6e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-24-ce87a96826f6025541ef71fd88c3d370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a b/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a new file mode 100644 index 0000000000000..25398d9017c7f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-25-f066907fca3448b27aab623d05258a9a @@ -0,0 +1 @@ +4378 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-26-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-27-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-28-b149d0fc9d40c63bb9a0807f5a0e04a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 new file mode 100644 index 0000000000000..25398d9017c7f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-29-65d3ae14a785f319658812b51e4620a3 @@ -0,0 +1 @@ +4378 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-3-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e b/sql/hive/src/test/resources/golden/smb_mapjoin_17-30-59009edb2442f58edebf83c8d047cb1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc b/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc new file mode 100644 index 0000000000000..09b5b315bcf45 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-31-783fc1b07f117cd027395cf1c09149bc @@ -0,0 +1 @@ +13126 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff b/sql/hive/src/test/resources/golden/smb_mapjoin_17-32-168e182e2c3feed13de0346cdeade2ff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-4-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-5-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-6-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_17-7-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 b/sql/hive/src/test/resources/golden/smb_mapjoin_17-8-14c221da1d1be2208aec9a641351ed09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e b/sql/hive/src/test/resources/golden/smb_mapjoin_17-9-6223202c291504ede983d8b04eafe08e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/sql/hive/src/test/resources/golden/smb_mapjoin_18-10-75e2e1eb0f45f4fad3e1ce24517dd81e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-11-fb15bd6eceb333302535c0bcdd2d078f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-12-27762aa3d846e8f50d97350eaa7563a1 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-13-91f2af0da94e5a83601d02045980f556 @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-14-1d0b88d6e6f84e485a05c712dd185531 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-15-21a6e09c24697160bf70cb17254ff504 @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-16-35dfd037075aac5a9891bf99ea01e156 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-17-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d b/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-18-ba77d1d4a1754489e2a8d5006b0af54d @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-19-da6eb8ded1f72f518617339f58202cc5 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a b/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-20-940ff79d8f3b401a05b19b9417824d7a @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-21-7ce17310f9fd362e2cc8a80211063264 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-22-a92f50cba6740abb3596c885a8157861 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-23-f0675c53cddf75b012f64e797f5824c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-24-a919b505c34a237ead34eea40b7d136c @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-25-d014ae435859316a1ad43548b72ecb7 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f b/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-26-52d3bbbbef7c580a324d991f52f44e1f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-27-6c5e30c84cb539cbf689a0d4cb4ed0e3 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab b/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-28-c83e75a3e18e68ef10d9970b3b8857ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-29-ecbfca5624b26a8eaa468a6bf46e3189 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-30-76e467313085467a3aa929b3665f9863 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-31-7f1e9114f4db63982985068c4bf36a29 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-32-b59c406dae57fa6ab656a27e310ac54c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe b/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-33-95b42e61bf5eed67123e30d482e0c7fe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-34-14ae716c8c0f47db61b89a2a17e89415 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-35-549b3df453bf939e1c0a38df861245e5 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_18-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c b/sql/hive/src/test/resources/golden/smb_mapjoin_18-7-c248759cecf0e2c223579f5e37f6669c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 b/sql/hive/src/test/resources/golden/smb_mapjoin_18-8-724d37bd4a841f1fa4062f4f3e3eb353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_18-9-c0c82db5bd80edc57f6b3cb0e807f2ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-10-75e2e1eb0f45f4fad3e1ce24517dd81e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-11-fb15bd6eceb333302535c0bcdd2d078f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-12-27762aa3d846e8f50d97350eaa7563a1 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 new file mode 100644 index 0000000000000..dce6588ca1420 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-13-4876f6e3c0ffde24afd82ec462962f19 @@ -0,0 +1 @@ +36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-14-cfc24b330d7c6a11ac2e4f2ea17b3f06 @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab b/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab new file mode 100644 index 0000000000000..d99e90eb9675f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-15-44ad799a82d847054f02d37139cc6aab @@ -0,0 +1 @@ +29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc b/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc new file mode 100644 index 0000000000000..dce6588ca1420 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-16-8ee972ce0d73f28e98f46361082c83dc @@ -0,0 +1 @@ +36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec b/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-17-af5c2dab8b5e2cb53b58fdeee911b7ec @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e new file mode 100644 index 0000000000000..d99e90eb9675f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-18-b2546caccb501fca356930e6cd00ea2e @@ -0,0 +1 @@ +29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-19-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b b/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b new file mode 100644 index 0000000000000..dce6588ca1420 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-20-bceffa38b796fbc2a85daac23357da7b @@ -0,0 +1 @@ +36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-21-a1f4332461d1810334e7ae6d7d43f829 @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f new file mode 100644 index 0000000000000..d99e90eb9675f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-22-ebd323090f771227039cd21b1b8c4f3f @@ -0,0 +1 @@ +29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb b/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb new file mode 100644 index 0000000000000..dce6588ca1420 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-23-90f0bad0606e63e4405eac9ec8c11deb @@ -0,0 +1 @@ +36 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 new file mode 100644 index 0000000000000..86ee83a4a2686 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-24-3ae0277bb7a74cd6bee704313dc102e6 @@ -0,0 +1 @@ +40 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f b/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f new file mode 100644 index 0000000000000..d99e90eb9675f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-25-e621350131d50867015e75677cca031f @@ -0,0 +1 @@ +29 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_19-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d b/sql/hive/src/test/resources/golden/smb_mapjoin_19-7-64b3fd0c215f4b8187866fa7eb55d34d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 b/sql/hive/src/test/resources/golden/smb_mapjoin_19-8-ee04de92100063f9b2063baddb204864 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_19-9-c0c82db5bd80edc57f6b3cb0e807f2ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a b/sql/hive/src/test/resources/golden/smb_mapjoin_2-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec b/sql/hive/src/test/resources/golden/smb_mapjoin_2-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac b/sql/hive/src/test/resources/golden/smb_mapjoin_2-10-1530d7565a22ace89ed16e0e1f9988ac new file mode 100644 index 0000000000000..7b5974818c085 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-11-9fd2e0578c0a8f705929a4f89d3cc97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-12-48e7d8fcb2a4c31c5304045517a3bb89 new file mode 100644 index 0000000000000..0e1d132524064 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d b/sql/hive/src/test/resources/golden/smb_mapjoin_2-13-596594a24f753a66ffd1c87dcd76781d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-14-3e7d435c7a5560abe539918adc4fa922 new file mode 100644 index 0000000000000..32be455fba8c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-15-2674b0d2498ac9392db40b4b83183d45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca b/sql/hive/src/test/resources/golden/smb_mapjoin_2-16-daeabb554f29b77f3c5ff7acff8c58ca new file mode 100644 index 0000000000000..2496c4400b0b9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-17-404d18544be565d251a6b0db1601d663 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-18-411bc909c1c29811d184d1f4aceb25b3 new file mode 100644 index 0000000000000..7b5974818c085 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-19-95e94fc54d417dc08c7617642ee9de90 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f b/sql/hive/src/test/resources/golden/smb_mapjoin_2-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b b/sql/hive/src/test/resources/golden/smb_mapjoin_2-20-43bc9e7e9e4d1bb1f42e7911bd79ea4b new file mode 100644 index 0000000000000..0e1d132524064 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-21-b9609684ffdae8768578849392032ad5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-22-de0b3ef8ef1c5902908dadd06042b84 new file mode 100644 index 0000000000000..32be455fba8c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a b/sql/hive/src/test/resources/golden/smb_mapjoin_2-23-4dec46d167dbc1ac0bc9212e778d090a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-24-709966d157a75ffb1c6ef70d7c72a498 new file mode 100644 index 0000000000000..2496c4400b0b9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_2-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_2-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 b/sql/hive/src/test/resources/golden/smb_mapjoin_2-9-dff8aeed3b5d7577966d965afd6d5408 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-10-82b3bc9314fa0cdb7fa59c58f22fb598 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-11-63ba770ebeff6032b68ba1aabbc1bee8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-12-6bea38b0ee0cccb3dfe7fe47c7c3e9c4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d b/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d new file mode 100644 index 0000000000000..b6e27607fb529 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-13-ba77d1d4a1754489e2a8d5006b0af54d @@ -0,0 +1 @@ +242 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 new file mode 100644 index 0000000000000..ce83bd94b3310 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-14-da6eb8ded1f72f518617339f58202cc5 @@ -0,0 +1 @@ +258 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a b/sql/hive/src/test/resources/golden/smb_mapjoin_20-15-5acb0ec2e1abcc6d57de7529f414a75a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a b/sql/hive/src/test/resources/golden/smb_mapjoin_20-16-5b9583aecebb7480d778dc2a8605434a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-17-e26f212ca34d303036299ba709f65522 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e b/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-18-7fb8778a58cc27dc04d5947df15b250e @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-19-6dd859e98c140df728f858a7a7598462 @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-20-fca1f7361c63f0ba1e9d365cf0743845 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 new file mode 100644 index 0000000000000..34251f6b242e7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-21-28d04e54c0a986079ae9d52788846519 @@ -0,0 +1 @@ +247 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-22-1baa93dd15ab33240255c5d6d5d57366 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-23-5c5eaf9922e7110c6d7260c738b17457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_20-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 b/sql/hive/src/test/resources/golden/smb_mapjoin_20-7-c9981ec081cbf54c9323e0dee977934 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff b/sql/hive/src/test/resources/golden/smb_mapjoin_20-8-5c9994d48509136efd1dcb4e3f3e7aff new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_20-9-c0c82db5bd80edc57f6b3cb0e807f2ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e b/sql/hive/src/test/resources/golden/smb_mapjoin_21-10-75e2e1eb0f45f4fad3e1ce24517dd81e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b b/sql/hive/src/test/resources/golden/smb_mapjoin_21-11-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-12-c3217069b040b92693a575bc077b6f78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf b/sql/hive/src/test/resources/golden/smb_mapjoin_21-13-5e00c7d92c43c135a58a609eb6a4b6cf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b b/sql/hive/src/test/resources/golden/smb_mapjoin_21-14-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd b/sql/hive/src/test/resources/golden/smb_mapjoin_21-15-975f27d5b8171d0b1bd441fa5f2e1ddd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a b/sql/hive/src/test/resources/golden/smb_mapjoin_21-16-c9250f30125e1235cb7ba01d97bbd1a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b b/sql/hive/src/test/resources/golden/smb_mapjoin_21-17-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-18-471c4e00fbbba3b23158aa4bf7595e79 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a b/sql/hive/src/test/resources/golden/smb_mapjoin_21-19-c9250f30125e1235cb7ba01d97bbd1a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b b/sql/hive/src/test/resources/golden/smb_mapjoin_21-20-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab b/sql/hive/src/test/resources/golden/smb_mapjoin_21-21-ee61731107316feaef2efca38bb43fab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-22-1c43fc5ebecae6e400cbd3bea40e3490 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b b/sql/hive/src/test/resources/golden/smb_mapjoin_21-23-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-24-c600a123838779df6a5babea49732659 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-25-bc535bc3d5108cdee7d8b670f0e763d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_21-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c b/sql/hive/src/test/resources/golden/smb_mapjoin_21-7-c248759cecf0e2c223579f5e37f6669c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 b/sql/hive/src/test/resources/golden/smb_mapjoin_21-8-724d37bd4a841f1fa4062f4f3e3eb353 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec b/sql/hive/src/test/resources/golden/smb_mapjoin_21-9-c0c82db5bd80edc57f6b3cb0e807f2ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-0-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-1-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-10-f0def0d77c93f6faebeca9b7a5340dbc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-11-b3e577f3099b5e4acffdd050f4c7f4dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a b/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-12-187c201f61c23833d0d193031926445a @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-13-5c5c521954e5ade621fd0cbff5794c80 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-14-b2534937758d2ff0a08e729f7b3ace4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-15-83d9e3d23d6612a926e53d57a5d07307 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe b/sql/hive/src/test/resources/golden/smb_mapjoin_22-16-abc95b36345d3e37abb196088897c7fe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b b/sql/hive/src/test/resources/golden/smb_mapjoin_22-17-720e78ad8ffc6620cc89a7e03bea3c4b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-18-2c29fc18b24368938f880a1bf3d5eb54 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e b/sql/hive/src/test/resources/golden/smb_mapjoin_22-19-5dc91a74bf4c43e20a8a213ad08c352e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-2-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad b/sql/hive/src/test/resources/golden/smb_mapjoin_22-20-ac918fa1a8e75031a0cc3eef0c9b77ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-21-f0def0d77c93f6faebeca9b7a5340dbc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc b/sql/hive/src/test/resources/golden/smb_mapjoin_22-22-b3e577f3099b5e4acffdd050f4c7f4dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a b/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-23-187c201f61c23833d0d193031926445a @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-24-5c5c521954e5ade621fd0cbff5794c80 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-25-b2534937758d2ff0a08e729f7b3ace4 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 new file mode 100644 index 0000000000000..99bc3d518639f --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-26-83d9e3d23d6612a926e53d57a5d07307 @@ -0,0 +1 @@ +253 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-3-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-4-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-5-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_22-6-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b b/sql/hive/src/test/resources/golden/smb_mapjoin_22-7-8317b719ffcf85da8c618e8f2379a31b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb b/sql/hive/src/test/resources/golden/smb_mapjoin_22-8-8cfa26d100b454c0b0f3443f62389abb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad b/sql/hive/src/test/resources/golden/smb_mapjoin_22-9-ac918fa1a8e75031a0cc3eef0c9b77ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-1-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-10-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_25-11-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-12-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-13-182e8b532775f6e514195adbea3948a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-14-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef b/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-15-442e69416faaea9309bb8c2a3eb73ef @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f b/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-16-beaea10543cfd753458b43d8aeb7571f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-17-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb b/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-18-d0ec6d66ff349db09fd455eec149efdb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-19-16112d7ada2ccc6f4a3b5d627410cb01 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-2-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-20-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d b/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-21-4d3e60a0e9bd8c12fdba4e010493537d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-22-d0a93f40892e3894460553b443c77428 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-23-ae45f6382268c1035c11aa4fc8a23e2 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-24-d5ad76f9178cb787cee037f25b19b270 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-25-a59d19c2438bddb97dc54c08dcf1d8e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-26-5a5aa75e424b6b5e3a2988e52bf3eff6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff b/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-3-bcde511b8d560ca83888265b96a781ff @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-4-8067b7ab260021bc70af9ab47309ee35 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-5-545c0ea2ebd7a141526ee13059a857f6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_25-6-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a b/sql/hive/src/test/resources/golden/smb_mapjoin_25-7-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec b/sql/hive/src/test/resources/golden/smb_mapjoin_25-8-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f b/sql/hive/src/test/resources/golden/smb_mapjoin_25-9-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a b/sql/hive/src/test/resources/golden/smb_mapjoin_3-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec b/sql/hive/src/test/resources/golden/smb_mapjoin_3-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-10-efadb45f09b92d27233601394d53d79 new file mode 100644 index 0000000000000..2c05a72679b22 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-11-52555bc44b4359aacfc42669bc5e4d12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-12-77988b41de4b5a165c93942fbb2220d6 new file mode 100644 index 0000000000000..0d9bce83ea87c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-13-812a0834a0e6546d5d0c8123d35c260f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e b/sql/hive/src/test/resources/golden/smb_mapjoin_3-14-5456a3a43bfabcfdbb57257b633e299e new file mode 100644 index 0000000000000..d4c1adc92802d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-15-adc9940d6a104bca6ec791e57bdb09c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a b/sql/hive/src/test/resources/golden/smb_mapjoin_3-16-fff1acc77150f6ea92fe8eefc04b079a new file mode 100644 index 0000000000000..b62eec8583c92 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe b/sql/hive/src/test/resources/golden/smb_mapjoin_3-17-1f00c63a1df29100d6e5fc4dcb4b0cfe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-18-96a658e934543dd77020ad264ec9b8c1 new file mode 100644 index 0000000000000..2c05a72679b22 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-19-bc0ddeb7b24fc7ed3fbcf77ced76ba07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd b/sql/hive/src/test/resources/golden/smb_mapjoin_3-20-867845ed2cb38b55928f5310b4ae91bd new file mode 100644 index 0000000000000..0d9bce83ea87c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-21-da48411b3e139037e171654fc9861755 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-22-5826f60da3a4b0c731c53340d50b0a06 new file mode 100644 index 0000000000000..d4c1adc92802d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-23-3e229e8219fe44b4a01f64b0967d7b31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-24-c73176ea199d4d20934cf2659d42ec25 new file mode 100644 index 0000000000000..b62eec8583c92 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_3-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_3-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af b/sql/hive/src/test/resources/golden/smb_mapjoin_3-9-8928717d58becd4fd358f3fec65d60af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a b/sql/hive/src/test/resources/golden/smb_mapjoin_4-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec b/sql/hive/src/test/resources/golden/smb_mapjoin_4-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-10-73afe8a2334d1dbb7d4af749875e2f24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-11-59fd75b48de9cbd23accc4c4414c6162 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-12-233250782bea3c7815e4c9949993d4f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-13-1ae07a8abc39f6a4195865ac583c1fb8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d b/sql/hive/src/test/resources/golden/smb_mapjoin_4-14-a6efb153fe9be91c7055a5642e0f642d new file mode 100644 index 0000000000000..46af2ffd2b620 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-15-18da9766adcfcd53820d747d4b573886 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-16-514034935af5348f67f1934cf5429d57 new file mode 100644 index 0000000000000..9047baeb236e5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-17-2a4723deed66d7aca3c272784dd11901 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-18-7f8bfb8ce6c1825708c37daa826fe5 new file mode 100644 index 0000000000000..47626758b4182 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-19-afa62ebf8d41de52dfe732bdc470a995 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-20-be9350a00a523e1758427a1c8e4bdf09 new file mode 100644 index 0000000000000..29616d6368661 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-21-b4e83e2474975db8ed614e671af132f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-22-da023084bbab6727dc3823cfce500308 new file mode 100644 index 0000000000000..6f90cbf839656 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-23-48c28e1909386c95f78cb448bb6272e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-24-7783ab33a26d29a2c0235f52f0e4ad86 new file mode 100644 index 0000000000000..3b195f9dff935 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-25-c12edf424bd7b75fc06a21d6498b89d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e b/sql/hive/src/test/resources/golden/smb_mapjoin_4-26-de14aa74d0da48dff6271410cc5dd98e new file mode 100644 index 0000000000000..e18ae75d14a37 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef b/sql/hive/src/test/resources/golden/smb_mapjoin_4-27-2617dc7a064cbb545c3ea54fe82c45ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-28-f71280b4b556515d068d074378c3a54 new file mode 100644 index 0000000000000..8e51f0864314a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb b/sql/hive/src/test/resources/golden/smb_mapjoin_4-29-239bc4053a7791815b587afef03515fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-30-a0d3abda611809460bcea4101967f21f new file mode 100644 index 0000000000000..51676ca2c94e4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-31-35fa9389f1249c6cd736786522e6d3c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-32-6477640b8a0a96f7a1c9290b8e71b5a8 new file mode 100644 index 0000000000000..4a125291de271 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-33-bc66eac0bf67a5c88f75ca66f0722e07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-34-48869ba51ea36864e720f94b09bf6b3 new file mode 100644 index 0000000000000..660e9044e889a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_4-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_4-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db b/sql/hive/src/test/resources/golden/smb_mapjoin_4-9-137cd40f999272507f72f2faa2fbd7db new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a b/sql/hive/src/test/resources/golden/smb_mapjoin_5-0-91ac1a1471f9d6dd9059bdc6c18f594a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec b/sql/hive/src/test/resources/golden/smb_mapjoin_5-1-ef8d866d66ac801be99efef73664cbec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc b/sql/hive/src/test/resources/golden/smb_mapjoin_5-10-df63d18b19a05a728723ad01d4e526fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-11-656085efab25db5dc019824b6b9a6bf5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-12-a7a6dd05f89679cf3d621ca969a468e4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a b/sql/hive/src/test/resources/golden/smb_mapjoin_5-13-a1bada65e6df26b6e66681c565b41f2a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-14-e20bcd28cfd26259bcde9cffec8d2280 new file mode 100644 index 0000000000000..46af2ffd2b620 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-15-cff7ff4d03450148a945711bad28b34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-16-8a8dc418de3716f5112b9868f472e4b9 new file mode 100644 index 0000000000000..9047baeb236e5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-17-6e755e2d2b23e9bd4073794eb4dbaa4f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc b/sql/hive/src/test/resources/golden/smb_mapjoin_5-18-1f0dd26e6088c4f359fb691a8ef650bc new file mode 100644 index 0000000000000..47626758b4182 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a b/sql/hive/src/test/resources/golden/smb_mapjoin_5-19-dd3d36a83baa3ac6155ad9b2c987657a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-2-31ed4593e624995154d0aedf94aa0f0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d b/sql/hive/src/test/resources/golden/smb_mapjoin_5-20-18fe4e120d3c0d663c360b2504b4f88d new file mode 100644 index 0000000000000..29616d6368661 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-21-7fd638ed676fe06a4970cc0c4a61b6c2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a b/sql/hive/src/test/resources/golden/smb_mapjoin_5-22-6ed2620fe017e454459a83061f25951a new file mode 100644 index 0000000000000..6f90cbf839656 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a b/sql/hive/src/test/resources/golden/smb_mapjoin_5-23-e68f9dc71bb8d5bcaf24d11e21da572a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b b/sql/hive/src/test/resources/golden/smb_mapjoin_5-24-87f758a20c15fa3b97b4ba6703ae541b new file mode 100644 index 0000000000000..3b195f9dff935 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-25-9f57e8a72a99c2e9a9d37bfb379af31 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-26-1c1a9519e207edc3c3a927c986a37177 new file mode 100644 index 0000000000000..e18ae75d14a37 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-27-d29148ce40aca2915f54788d3890b6e2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-28-94440444fff7d2629a23a30fd778fcc7 new file mode 100644 index 0000000000000..8e51f0864314a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-29-8862326985836c72a2b21cb96a151579 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-3-d0e31f8ed366038ca6f4f4955d2bc796 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-30-574d26a5179e5cebbbff5bbb425a9609 new file mode 100644 index 0000000000000..51676ca2c94e4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-31-aa33e936a0d1a5ed534caef75647f0c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc b/sql/hive/src/test/resources/golden/smb_mapjoin_5-32-17c99f827824994cfd21c40dbf4abfc new file mode 100644 index 0000000000000..4a125291de271 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-33-487e89a8efb4dea08105f2fa79de5087 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-34-4db871582cf4f3038d43d0a2d5ae6895 new file mode 100644 index 0000000000000..660e9044e889a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de b/sql/hive/src/test/resources/golden/smb_mapjoin_5-4-3af09654f8d38d21b5d26e6cc21210de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-5-35ab67c91c53dc725f2eab0fb8c9e62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-6-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-7-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_5-8-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df b/sql/hive/src/test/resources/golden/smb_mapjoin_5-9-88a0d8b67f250913ba2be3cddffeb7df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-10-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-11-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-12-39e8889c37ee53d9a0ba48165b080b14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-13-f9de7f57a4c89ccf84d35f47abb40d77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca b/sql/hive/src/test/resources/golden/smb_mapjoin_6-14-43123d2821871aa5b4d4a9e22e03d1ca new file mode 100644 index 0000000000000..c2e3ea8b0c8e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-15-c673bbec2ee476eea9609be3d3555438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e new file mode 100644 index 0000000000000..0e17d179f4167 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-16-449cb60e6537ba0810ea6879a7351a1e @@ -0,0 +1 @@ +278697 278697 101852390308 101852390308 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a new file mode 100644 index 0000000000000..0e17d179f4167 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-17-aa5f237005fb69b3f52808c80b4a276a @@ -0,0 +1 @@ +278697 278697 101852390308 101852390308 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a b/sql/hive/src/test/resources/golden/smb_mapjoin_6-18-a2ed8f3ca5f4a3c2d1377bc82463fe1a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-19-4fd621581c51ad55a7e8389a94c6a411 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-20-f9de7f57a4c89ccf84d35f47abb40d77 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca b/sql/hive/src/test/resources/golden/smb_mapjoin_6-21-43123d2821871aa5b4d4a9e22e03d1ca new file mode 100644 index 0000000000000..c2e3ea8b0c8e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-22-c673bbec2ee476eea9609be3d3555438 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e new file mode 100644 index 0000000000000..0e17d179f4167 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-23-449cb60e6537ba0810ea6879a7351a1e @@ -0,0 +1 @@ +278697 278697 101852390308 101852390308 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a new file mode 100644 index 0000000000000..0e17d179f4167 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-24-aa5f237005fb69b3f52808c80b4a276a @@ -0,0 +1 @@ +278697 278697 101852390308 101852390308 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-25-e3552fabb4c9920d9bb964da5b907b46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-26-5796487563f66866051a86f9c8cee822 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-27-efb64856d7a3cadd61eb1e2c61197c8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-28-9715967e9a70084a4600ccac53dfac60 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-29-495784f95b09cffa052805c6b9af9216 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-3-2bfc6b33655f683fa9bd62ced5dab230 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-30-2cd20a399ce104eb7fb5881e59238196 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa b/sql/hive/src/test/resources/golden/smb_mapjoin_6-4-6eb37be25535293ca377a47c3e08bffa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c b/sql/hive/src/test/resources/golden/smb_mapjoin_6-5-ab8df8b65b35c17c45cac15468a4042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-6-ae6abe879c8ee2b101c7d78da0770dd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-7-d7835ed6fd3add8aa8d65ef196798cc4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 b/sql/hive/src/test/resources/golden/smb_mapjoin_6-8-2e0e607c192151d31a6e6513fba79ef3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_6-9-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-0-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-1-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-10-2e0e607c192151d31a6e6513fba79ef3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-11-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-12-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-13-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-14-5e1546e0e05871eb1a6b8eb957252542 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-15-5e1546e0e05871eb1a6b8eb957252542 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-16-18d2a689883588ae3c24b2810663ab80 new file mode 100644 index 0000000000000..4cd5eefea2a45 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd b/sql/hive/src/test/resources/golden/smb_mapjoin_7-17-2ff32fe07d35a4d44b640f3ded5767fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a b/sql/hive/src/test/resources/golden/smb_mapjoin_7-18-1a4ee3f64c4a9bb1689234d30babb4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a b/sql/hive/src/test/resources/golden/smb_mapjoin_7-19-ab46164010b27950c293f32fb89f9f8a new file mode 100644 index 0000000000000..4cd5eefea2a45 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-20-35dcb5bae00dc0cc7afa6663a5126fe6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e b/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e new file mode 100644 index 0000000000000..ca97f45c90026 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-21-449cb60e6537ba0810ea6879a7351a1e @@ -0,0 +1 @@ +0 130091 0 36210398070 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a b/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a new file mode 100644 index 0000000000000..ca97f45c90026 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-22-aa5f237005fb69b3f52808c80b4a276a @@ -0,0 +1 @@ +0 130091 0 36210398070 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff b/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff new file mode 100644 index 0000000000000..ca97f45c90026 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_7-23-1b6140f49d49eb1195c8551270b97ff @@ -0,0 +1 @@ +0 130091 0 36210398070 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa b/sql/hive/src/test/resources/golden/smb_mapjoin_7-3-e666afe7d9a532114a6133b7dc7df5aa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de b/sql/hive/src/test/resources/golden/smb_mapjoin_7-4-85d6ffbe24f95df3128dee0e21a032de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c b/sql/hive/src/test/resources/golden/smb_mapjoin_7-5-ab8df8b65b35c17c45cac15468a4042c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-6-8613f4d139c1ccdbb3be9b7af0e24ad4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 b/sql/hive/src/test/resources/golden/smb_mapjoin_7-7-ae6abe879c8ee2b101c7d78da0770dd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb b/sql/hive/src/test/resources/golden/smb_mapjoin_7-8-5f141437e3e929cde1b2e5d8458b46eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f b/sql/hive/src/test/resources/golden/smb_mapjoin_7-9-461966a6ca78552b62bbbae4ff5abf1f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-0-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-1-e39f59c35ebbe686a18d45d9d8bf3ab0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-10-401c6c31bc3ae2cbd493a3855b829893 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-11-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-12-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-13-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-14-17529a2129bd485b814dde1ef894f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b b/sql/hive/src/test/resources/golden/smb_mapjoin_8-15-343dfc1301b9f601ae03cb15f790102b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-16-e8f0853543694175095a8642e8f4fa03 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-17-5d2bab62264e462c007eb1a7a2c68866 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-18-17529a2129bd485b814dde1ef894f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b b/sql/hive/src/test/resources/golden/smb_mapjoin_8-19-343dfc1301b9f601ae03cb15f790102b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-20-3af6a47b927b966f80a8f4c973b5f1f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-21-77ac2e23e8a8bc22274952933cd8d252 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-22-17529a2129bd485b814dde1ef894f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b b/sql/hive/src/test/resources/golden/smb_mapjoin_8-23-343dfc1301b9f601ae03cb15f790102b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c b/sql/hive/src/test/resources/golden/smb_mapjoin_8-24-4e7e6b8eb897ed86e28d435c4f522e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-25-722aa3855f98f940fb34368c2789bcc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-26-17529a2129bd485b814dde1ef894f06 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b b/sql/hive/src/test/resources/golden/smb_mapjoin_8-27-343dfc1301b9f601ae03cb15f790102b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c b/sql/hive/src/test/resources/golden/smb_mapjoin_8-28-4e7e6b8eb897ed86e28d435c4f522e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-29-722aa3855f98f940fb34368c2789bcc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/smb_mapjoin_8-3-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e b/sql/hive/src/test/resources/golden/smb_mapjoin_8-30-19b2b8c39155340929c605494826d30e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-31-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c b/sql/hive/src/test/resources/golden/smb_mapjoin_8-32-4e7e6b8eb897ed86e28d435c4f522e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-33-722aa3855f98f940fb34368c2789bcc3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f b/sql/hive/src/test/resources/golden/smb_mapjoin_8-34-c8132556830d31a8803b07273aa1558f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-35-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-36-3af6a47b927b966f80a8f4c973b5f1f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-37-77ac2e23e8a8bc22274952933cd8d252 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-38-5ca28e78b53d14893518057c142f68a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-39-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-4-cd78ebbe3cc259ee6efb946f6b28f8d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b b/sql/hive/src/test/resources/golden/smb_mapjoin_8-40-7aa01ceccf1bb45418145bdfc061206b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c b/sql/hive/src/test/resources/golden/smb_mapjoin_8-41-8cf49829aada9587f04c9a5f334c433c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-42-d656a335f0f2f311717e2d72d0a9b6c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-43-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-44-559ecfd50bcc8c1bf63024c160bcad35 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c b/sql/hive/src/test/resources/golden/smb_mapjoin_8-45-8cf49829aada9587f04c9a5f334c433c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-46-d656a335f0f2f311717e2d72d0a9b6c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-47-3b6920dfb64f41b1b3c3c265275b1e4a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-5-a1f5562186e9e22e18ebd42208943525 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-6-f6541fbe0d06b7ce2479fc4da9d2905a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-7-35e90a6bc292cfd33b48c52460937858 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a b/sql/hive/src/test/resources/golden/smb_mapjoin_8-8-81a6ffb61cbddc9900790d8b08e39e2a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 b/sql/hive/src/test/resources/golden/smb_mapjoin_8-9-bd2b5b7d4eaec7854e3051455e601109 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort-0-eee35bbc888a705ae527625447668032 b/sql/hive/src/test/resources/golden/sort-0-eee35bbc888a705ae527625447668032 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 b/sql/hive/src/test/resources/golden/sort-1-10c8b0a592ed15eff488a71ec5909f45 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-0-6a35b44d4170109f641f8b68efef3719 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-1-a1c5a993c00586e79a161a5a11a6fe5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-10-e4475ba09980280452542a0d595b8217 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-11-3599c6aa6da9420bdcec89c8516ffe89 @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-3-6cc3ecca54fb84949a89240c50d4bd19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-4-a2e980fa0fbe8737cf256e21c784193e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-5-452fb932f6353effc9c458ec24dbeff9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-6-7033dd1c3de2ad70a50a522d88533cf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_1-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-0-6a35b44d4170109f641f8b68efef3719 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-1-a1c5a993c00586e79a161a5a11a6fe5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-10-c0cac985e4115bf4014274b21222cac9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-11-47bc9cb5f049aaca33b394ea78578bdd @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-3-a38a9b74e94a1eb032a52d905d863d46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-4-ac10c36f76f1dcf25783bb796d182f64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-5-452fb932f6353effc9c458ec24dbeff9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-6-7033dd1c3de2ad70a50a522d88533cf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_2-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-0-6a35b44d4170109f641f8b68efef3719 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-1-a1c5a993c00586e79a161a5a11a6fe5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-10-8aa5c2ebf5cb82b372b17954dbdb5f28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-11-47bc9cb5f049aaca33b394ea78578bdd @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-3-ce75a01507b158094bcf4441f3bfea95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-4-ac1b8e89e9977e535e7e499992b6ccf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-5-452fb932f6353effc9c458ec24dbeff9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-6-7033dd1c3de2ad70a50a522d88533cf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_3-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-0-6a35b44d4170109f641f8b68efef3719 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-1-a1c5a993c00586e79a161a5a11a6fe5c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-10-99adddf7dacf8042707a9b36062fe103 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd new file mode 100644 index 0000000000000..8fdd954df9831 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-11-47bc9cb5f049aaca33b394ea78578bdd @@ -0,0 +1 @@ +22 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-2-ffe97dc8c1df3195982e38263fbe8717 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-3-ce75a01507b158094bcf4441f3bfea95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-4-ac10c36f76f1dcf25783bb796d182f64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-5-452fb932f6353effc9c458ec24dbeff9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-6-7033dd1c3de2ad70a50a522d88533cf2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-7-3b0f76816be2c1b18a2058027a19bc9f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_4-9-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-1-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a new file mode 100644 index 0000000000000..67d892c80f493 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-10-cd2f3c647c0821eb53e67d36b1556a4a @@ -0,0 +1 @@ +1028 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-2-1a2fc4c300be7c94681bebc14f7f7deb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-3-ac35b16d0e28472c6079f4ce2ee1438d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-4-f392824f42188aac282046394108ef48 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-5-85d19fc31c2031491560208803fe63bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-6-eac4a4cf29e65d7bd394bbfc57fcbfa7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-7-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-8-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_5-9-8b5783beaca02bba77d82adefe02c51 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-1-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-10-4c7a7ec8806a506693dcb7278c90cba8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a new file mode 100644 index 0000000000000..67d892c80f493 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-11-cd2f3c647c0821eb53e67d36b1556a4a @@ -0,0 +1 @@ +1028 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-3-d651a32232651880ad90428ada99358b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-4-ac35b16d0e28472c6079f4ce2ee1438d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-5-9d2278807f7eef56292afd44b347ff64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-6-85d19fc31c2031491560208803fe63bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-7-e12bb9bc44b0ed7e980ebe47517bc3e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-8-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_6-9-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-0-7b4ad215fc2e75c71c6614a2b6322e8e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-1-365488a703b0640acda73a7d7e6efa06 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-10-5918e837d2ae72cfaa753516c8652f24 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-11-e87a4aa598ea59bfbb7e61879a5b51c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-12-c23ea191ee4d60c0a6252ce763b1beed @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-13-86473a0498e4361e4db0b4a22f2e8571 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-14-afdad72caa2e23071817fe124da07763 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 new file mode 100644 index 0000000000000..8983bf8a289a9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-15-22330850dd2a9d18dedb504ea5879a07 @@ -0,0 +1 @@ +4112 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-2-7cccbdffc32975f8935eeba14a28147 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-3-546c076289ceaf1fbee969aeb30b402b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-4-ac35b16d0e28472c6079f4ce2ee1438d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-5-5f8d9bff48ffdbcc9a18553575bd0c83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-6-7cf7a404374e41059f8c1db181a0e1f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-7-fecc7e5eef32fd38735e82d9d78a324f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-8-85d19fc31c2031491560208803fe63bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec b/sql/hive/src/test/resources/golden/sort_merge_join_desc_7-9-d1cfb738e07e78009bac4881c05853ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats0-0-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats0-1-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c b/sql/hive/src/test/resources/golden/stats0-10-ee1bfc1f0047527d9bd745dcc747ab6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats0-11-82294461be4728b4b191414bf2fb3bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 b/sql/hive/src/test/resources/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 new file mode 100644 index 0000000000000..284da67c9c7c1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats0-12-31ebc4dcaa13c6dc5937c04f603e699 @@ -0,0 +1 @@ +ds=1 diff --git a/sql/hive/src/test/resources/golden/stats0-13-ca0569fd4200d549843deadf84044649 b/sql/hive/src/test/resources/golden/stats0-13-ca0569fd4200d549843deadf84044649 new file mode 100644 index 0000000000000..93381fd6177d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 b/sql/hive/src/test/resources/golden/stats0-14-4f18f4b06db06844920b14e2d19471a9 new file mode 100644 index 0000000000000..688eea009d292 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c b/sql/hive/src/test/resources/golden/stats0-15-4a7ed5b7e3deb4106204f8f950357e1c new file mode 100644 index 0000000000000..4d3a3d8f237b3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats0-16-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats0-17-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 b/sql/hive/src/test/resources/golden/stats0-18-bb8d8522a40a50fb684fabffd2fa7d17 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed b/sql/hive/src/test/resources/golden/stats0-19-eb5c7d6fd7433dfe9684e43e4a3419ed new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats0-2-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats0-20-5f7439f97275da70f633e135205f2095 b/sql/hive/src/test/resources/golden/stats0-20-5f7439f97275da70f633e135205f2095 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a b/sql/hive/src/test/resources/golden/stats0-21-6bcc1360b3d777a8b51ae416ff43898a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 b/sql/hive/src/test/resources/golden/stats0-22-85de3349a12a9fa0536f20ad0377fca1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c b/sql/hive/src/test/resources/golden/stats0-23-247568f4b3ce6b97979ca9d28c2ba05c new file mode 100644 index 0000000000000..a18ab93992cc5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 b/sql/hive/src/test/resources/golden/stats0-24-7f38972bcd18c477a027d881182d83f9 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b b/sql/hive/src/test/resources/golden/stats0-25-98c925a2b2c4de06e270e1b52437a98b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c b/sql/hive/src/test/resources/golden/stats0-26-ee1bfc1f0047527d9bd745dcc747ab6c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats0-27-82294461be4728b4b191414bf2fb3bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 b/sql/hive/src/test/resources/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 new file mode 100644 index 0000000000000..284da67c9c7c1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats0-28-31ebc4dcaa13c6dc5937c04f603e699 @@ -0,0 +1 @@ +ds=1 diff --git a/sql/hive/src/test/resources/golden/stats0-29-ca0569fd4200d549843deadf84044649 b/sql/hive/src/test/resources/golden/stats0-29-ca0569fd4200d549843deadf84044649 new file mode 100644 index 0000000000000..93381fd6177d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats0-3-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 b/sql/hive/src/test/resources/golden/stats0-30-4f18f4b06db06844920b14e2d19471a9 new file mode 100644 index 0000000000000..d861060f13cae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c b/sql/hive/src/test/resources/golden/stats0-31-4a7ed5b7e3deb4106204f8f950357e1c new file mode 100644 index 0000000000000..aa4dfe1e5ba39 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-4-5f7439f97275da70f633e135205f2095 b/sql/hive/src/test/resources/golden/stats0-4-5f7439f97275da70f633e135205f2095 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a b/sql/hive/src/test/resources/golden/stats0-5-6bcc1360b3d777a8b51ae416ff43898a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 b/sql/hive/src/test/resources/golden/stats0-6-85de3349a12a9fa0536f20ad0377fca1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c b/sql/hive/src/test/resources/golden/stats0-7-247568f4b3ce6b97979ca9d28c2ba05c new file mode 100644 index 0000000000000..8a5ae7cda1c57 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 b/sql/hive/src/test/resources/golden/stats0-8-7f38972bcd18c477a027d881182d83f9 new file mode 100644 index 0000000000000..7aae61e5eb82f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b b/sql/hive/src/test/resources/golden/stats0-9-98c925a2b2c4de06e270e1b52437a98b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats2-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats2-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e b/sql/hive/src/test/resources/golden/stats2-10-56dfd0f30574154dc1c2f55c29e2fa0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b b/sql/hive/src/test/resources/golden/stats2-11-912c785dbcef3251dd1d6b7fc56eae5b new file mode 100644 index 0000000000000..f87b67cb2006f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats2-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats2-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats2-4-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 b/sql/hive/src/test/resources/golden/stats2-5-6717e1026e24a38af19b6bce1951e3d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 b/sql/hive/src/test/resources/golden/stats2-6-45d4fb785fc94d266096fc29a9e90d73 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 b/sql/hive/src/test/resources/golden/stats2-7-6436db7a7506b560d0d1759db94a76b9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 b/sql/hive/src/test/resources/golden/stats2-8-72621dba638b15d244850018e9f64d7 new file mode 100644 index 0000000000000..19a5e1e042ae7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e b/sql/hive/src/test/resources/golden/stats2-9-6d93732dc2ca622eb60c171389caee8e new file mode 100644 index 0000000000000..7ddea146475d1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats20-0-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats20-1-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 b/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats20-2-4711e55648c25c86bb526ed50b0c3d09 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b b/sql/hive/src/test/resources/golden/stats20-3-98c925a2b2c4de06e270e1b52437a98b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats20-4-82294461be4728b4b191414bf2fb3bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a b/sql/hive/src/test/resources/golden/stats20-5-76509775cfe11bb98ee088188a07668a new file mode 100644 index 0000000000000..5c8878cba5e71 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 b/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats20-6-1f7f76b2a9d50f86de42edf6ba8a3f84 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 b/sql/hive/src/test/resources/golden/stats20-7-82294461be4728b4b191414bf2fb3bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa b/sql/hive/src/test/resources/golden/stats20-8-300c971de74642118d36d36349bc81aa new file mode 100644 index 0000000000000..d9ba3814de367 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats3-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats3-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 b/sql/hive/src/test/resources/golden/stats3-10-e2e5e7a0378c6f0c28391c447ec9cee9 new file mode 100644 index 0000000000000..bf2a7f452edc3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats3-11-2e8d5cefd9a1b1e8f25b117ca68bfeaa @@ -0,0 +1 @@ +6 diff --git a/sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc b/sql/hive/src/test/resources/golden/stats3-12-892cb7ecc26e84f1c033b95a3ee3edc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/stats3-13-ca5e3149f2b190d7df923a3e5c1cb07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa b/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa new file mode 100644 index 0000000000000..1e8b314962144 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats3-14-2e8d5cefd9a1b1e8f25b117ca68bfeaa @@ -0,0 +1 @@ +6 diff --git a/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 b/sql/hive/src/test/resources/golden/stats3-15-c6493490f898e72dc7ed1bc2d4721aa4 new file mode 100644 index 0000000000000..bf2a7f452edc3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 b/sql/hive/src/test/resources/golden/stats3-16-ca5e3149f2b190d7df923a3e5c1cb07 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 b/sql/hive/src/test/resources/golden/stats3-17-c012b29f0d7720fbc515aa5fe2759ac7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b b/sql/hive/src/test/resources/golden/stats3-18-eefbb3ee8b538aec85c609351e52901b new file mode 100644 index 0000000000000..cb1c9def371df --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 b/sql/hive/src/test/resources/golden/stats3-19-c1bbac06a43b5b00a69356955e4a1dd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 b/sql/hive/src/test/resources/golden/stats3-2-c1bbac06a43b5b00a69356955e4a1dd3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 b/sql/hive/src/test/resources/golden/stats3-20-ca048ad81b3df7159822073d206f0790 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 b/sql/hive/src/test/resources/golden/stats3-3-ca048ad81b3df7159822073d206f0790 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 b/sql/hive/src/test/resources/golden/stats3-4-fa705a031ff5d97558f29c2b5b9de282 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 b/sql/hive/src/test/resources/golden/stats3-5-1b2be7f8bd7bacc8f77a24ffdc0830f1 new file mode 100644 index 0000000000000..8bc7c14574497 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c b/sql/hive/src/test/resources/golden/stats3-6-4bf1504274319c44d370b58092fe016c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d b/sql/hive/src/test/resources/golden/stats3-7-73d7d55d6e5a57aacce8618902904d new file mode 100644 index 0000000000000..4916ef91b0866 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 b/sql/hive/src/test/resources/golden/stats3-8-b0ebbe71c220979b8fd4a36ffa501bf6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 b/sql/hive/src/test/resources/golden/stats3-9-21f4ee91fa1c65e8579e4cbe4777d7a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats4-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats4-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 b/sql/hive/src/test/resources/golden/stats4-10-a33b2c9d962e4921c98e62387f3989f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 b/sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 new file mode 100644 index 0000000000000..0a88e449f625a --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats4-11-ea921e0af59a4940a11c94143b1c4b32 @@ -0,0 +1,2 @@ +ds=2008-04-08/hr=11 +ds=2008-04-08/hr=12 diff --git a/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 b/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 new file mode 100644 index 0000000000000..1634adfc4d70d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats4-12-30bc31441828a053d1a675b225a5d617 @@ -0,0 +1,2 @@ +ds=2008-12-31/hr=11 +ds=2008-12-31/hr=12 diff --git a/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 b/sql/hive/src/test/resources/golden/stats4-13-fca9513ea05bfb8b7e0e6f337d184d66 new file mode 100644 index 0000000000000..d7a8f25b41301 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 b/sql/hive/src/test/resources/golden/stats4-14-9c82167763a771c175c656786d545798 new file mode 100644 index 0000000000000..653516475da22 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c b/sql/hive/src/test/resources/golden/stats4-15-f02b95f20b526fcf2850b07ca6be4f8c new file mode 100644 index 0000000000000..2ac44ee23f45c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 b/sql/hive/src/test/resources/golden/stats4-16-f2bdfe1d8be2ff6f784fcf1c892e1b70 new file mode 100644 index 0000000000000..f0b07ca89fcb3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 b/sql/hive/src/test/resources/golden/stats4-17-746b888d14c1b3fa28aa8549c174f6d9 new file mode 100644 index 0000000000000..f94614124bcf6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 b/sql/hive/src/test/resources/golden/stats4-18-dbe13731de4ab2a3c23343b78525b2f7 new file mode 100644 index 0000000000000..f00fb5b3bd491 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 b/sql/hive/src/test/resources/golden/stats4-19-3d752e1f8748ba98bebb2cef3f473cd5 new file mode 100644 index 0000000000000..7335c8d32fa68 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 b/sql/hive/src/test/resources/golden/stats4-2-463330cf55370dbe92d6ed74ef91302 new file mode 100644 index 0000000000000..8c43153cf66f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 b/sql/hive/src/test/resources/golden/stats4-20-f63000f2c395b935199c9829964f98c1 new file mode 100644 index 0000000000000..674cd9b0b64e5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd b/sql/hive/src/test/resources/golden/stats4-21-c2faa5beb457f3e6da58d4d91b5d1cbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c b/sql/hive/src/test/resources/golden/stats4-22-f709d5c1d4c9f13b7a521867674ac54c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd b/sql/hive/src/test/resources/golden/stats4-3-c2faa5beb457f3e6da58d4d91b5d1cbd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c b/sql/hive/src/test/resources/golden/stats4-4-f709d5c1d4c9f13b7a521867674ac54c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d b/sql/hive/src/test/resources/golden/stats4-5-cd849c4fe1229428da98947e3e43b46d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 b/sql/hive/src/test/resources/golden/stats4-6-9c0d4354b6a9be351fa32a66ff58a177 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats4-7-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats4-8-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f b/sql/hive/src/test/resources/golden/stats4-9-255ad4511130fb8c9ee9d65b7c95743f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats5-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats5-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 b/sql/hive/src/test/resources/golden/stats5-2-6f5d06d6100de19ec398891cb2eae161 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df b/sql/hive/src/test/resources/golden/stats5-3-96d9aa9c32a081518604959dcfac42df new file mode 100644 index 0000000000000..c9a75eb6639d4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 b/sql/hive/src/test/resources/golden/stats5-4-dbf81a12f6c19c14dce831e942870744 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 b/sql/hive/src/test/resources/golden/stats5-5-3980dfc2e4f882dd3bf478e56929361 new file mode 100644 index 0000000000000..e622a0d3704a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats6-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats6-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats6-10-653f748fa2d690502ba4fda407841a20 new file mode 100644 index 0000000000000..bf589ab894312 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats6-11-1c9f833953403596ad50fd32e513642c new file mode 100644 index 0000000000000..e6170a33d7de7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats6-12-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..45f6b344b04ca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats6-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats6-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats6-4-a88c476a632cd92f748967fadb242405 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats6-5-66590e5ed0bd8cd22ae01ecb658c11dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce b/sql/hive/src/test/resources/golden/stats6-6-8926095434b70c83bf88c70559d38dce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 b/sql/hive/src/test/resources/golden/stats6-7-6615049191cfa4026a3a870c3c3749f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats6-8-e15e242124e61ede9196130cb3fb69e7 new file mode 100644 index 0000000000000..9ff40db416cb9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats6-9-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..5c13e8793cc76 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats7-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats7-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats7-10-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..bafa551db56e9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats7-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats7-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats7-4-a88c476a632cd92f748967fadb242405 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats7-5-66590e5ed0bd8cd22ae01ecb658c11dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 b/sql/hive/src/test/resources/golden/stats7-6-5d0c6aa78c9786d904728ff7adab85f2 new file mode 100644 index 0000000000000..d5341825cff70 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 b/sql/hive/src/test/resources/golden/stats7-7-4912a8bbc138ab97ac0983bc90951de4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats7-8-e15e242124e61ede9196130cb3fb69e7 new file mode 100644 index 0000000000000..7085e9397e59b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats7-9-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..a3eb8c6466032 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats8-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats8-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 b/sql/hive/src/test/resources/golden/stats8-10-ce78d029b7764edce3a26336cfac6a8 new file mode 100644 index 0000000000000..d1e0c6243244d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 b/sql/hive/src/test/resources/golden/stats8-11-6615049191cfa4026a3a870c3c3749f4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats8-12-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..27ff9a687f5c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d b/sql/hive/src/test/resources/golden/stats8-13-c728cdab72daf6b0fedcf0a42f0dd07d new file mode 100644 index 0000000000000..ce340cced8844 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 b/sql/hive/src/test/resources/golden/stats8-14-ea9afc1343991ed4d410231803a174f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats8-15-653f748fa2d690502ba4fda407841a20 new file mode 100644 index 0000000000000..649828a63174b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 b/sql/hive/src/test/resources/golden/stats8-16-5179c0a116edc8dc9f7913fd3097bdd6 new file mode 100644 index 0000000000000..e4c8bf3e0005f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 b/sql/hive/src/test/resources/golden/stats8-17-9d896286dccb22d6e1b6abd5c55eaaa4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats8-18-1c9f833953403596ad50fd32e513642c new file mode 100644 index 0000000000000..74aeceec37398 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 b/sql/hive/src/test/resources/golden/stats8-19-dbf72430cff3df0e6e9405ad64531b16 new file mode 100644 index 0000000000000..25d9c86f74634 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats8-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 b/sql/hive/src/test/resources/golden/stats8-20-bff592750d1057448d2cff7694b6dad2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats8-21-e15e242124e61ede9196130cb3fb69e7 new file mode 100644 index 0000000000000..5939c257f2f67 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats8-22-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..8dc22de8e0ee6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats8-23-653f748fa2d690502ba4fda407841a20 new file mode 100644 index 0000000000000..a19f38cc6a364 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats8-24-1c9f833953403596ad50fd32e513642c new file mode 100644 index 0000000000000..aead24acd518c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats8-25-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..5f8bf1774f144 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats8-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 b/sql/hive/src/test/resources/golden/stats8-4-a88c476a632cd92f748967fadb242405 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats8-5-66590e5ed0bd8cd22ae01ecb658c11dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 b/sql/hive/src/test/resources/golden/stats8-6-821e1f061960081b9b379d9bfb47f267 new file mode 100644 index 0000000000000..d30acbf86a295 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce b/sql/hive/src/test/resources/golden/stats8-7-8926095434b70c83bf88c70559d38dce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 b/sql/hive/src/test/resources/golden/stats8-8-e15e242124e61ede9196130cb3fb69e7 new file mode 100644 index 0000000000000..07a61fc1a8bd1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats8-9-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..bd16e76d28d83 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats9-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats9-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-0-6d1832d28f897d0049de053617bd36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-1-887fe99770f53e7e0a0fbdc190118612 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-10-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-11-d58626190cded8d09f0457739a980eb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-12-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-13-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-14-60a231b64a4a0e414d0ddce1c813c614 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-15-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-16-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-2-2ca079278e0de95eecb5df315ce05c6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-3-66e3e0c942759f679c270698b49bfcf1 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-4-d389db66cc7fd9b144445e364dac30e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-5-63abf47101c5097e66a9c3ee92b128e3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-6-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-7-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-8-11f77597cc11fc71e95c0d0d7502c5c3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c b/sql/hive/src/test/resources/golden/stats_aggregator_error_1-9-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-0-9505721cd28fc4ec94ef4baf07029027 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-2-c14f09f88961dbad4d800317079a9105 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-4-cc664530711607c530a2cd384e67a600 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 b/sql/hive/src/test/resources/golden/stats_empty_dyn_part-5-76d56e06108f0c6da75aa821452fa873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 b/sql/hive/src/test/resources/golden/stats_empty_partition-0-42ae9007e347f7dedfc8974d21d84b41 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_empty_partition-1-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 b/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_empty_partition-2-c14f09f88961dbad4d800317079a9105 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 b/sql/hive/src/test/resources/golden/stats_empty_partition-3-868ba7a812c705caa29db49cdcb69c45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 b/sql/hive/src/test/resources/golden/stats_empty_partition-4-aed016ae4b528521874a719a5b129a55 new file mode 100644 index 0000000000000..4475cb837eaa4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_noscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_noscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d b/sql/hive/src/test/resources/golden/stats_noscan_1-10-e6b884de17a29eb476fd6cc502fc615d new file mode 100644 index 0000000000000..5949a76d84a11 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 b/sql/hive/src/test/resources/golden/stats_noscan_1-11-653f748fa2d690502ba4fda407841a20 new file mode 100644 index 0000000000000..97f5929fd816a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c b/sql/hive/src/test/resources/golden/stats_noscan_1-12-1c9f833953403596ad50fd32e513642c new file mode 100644 index 0000000000000..6d08ff47abc2c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 b/sql/hive/src/test/resources/golden/stats_noscan_1-13-fdbe263d925f28d598a71b7a4c577492 new file mode 100644 index 0000000000000..f441c8b483868 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab b/sql/hive/src/test/resources/golden/stats_noscan_1-14-c51fe56935caed4f7ca6f7d9cd6a17ab new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 b/sql/hive/src/test/resources/golden/stats_noscan_1-15-db563e338e4f658e5072cc60aef15480 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 b/sql/hive/src/test/resources/golden/stats_noscan_1-16-e446db2c3ddce173d0a51bf77a489382 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 b/sql/hive/src/test/resources/golden/stats_noscan_1-17-82369b182db851e06bfddb62965e03a3 new file mode 100644 index 0000000000000..f263eed117b10 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 b/sql/hive/src/test/resources/golden/stats_noscan_1-18-eece38f413a0a4f6f5b63cea0751d225 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e b/sql/hive/src/test/resources/golden/stats_noscan_1-19-1e9c23f4b90d83f4e8c5c7f08365928e new file mode 100644 index 0000000000000..6a115f7e6e61a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_noscan_1-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f b/sql/hive/src/test/resources/golden/stats_noscan_1-20-9871c619bb5bf0a28f8d60e6332a614f new file mode 100644 index 0000000000000..1d96413b915da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 b/sql/hive/src/test/resources/golden/stats_noscan_1-21-35a7cb6076ef7bd253ea9c1f5d009709 new file mode 100644 index 0000000000000..4c4380e63b7f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a b/sql/hive/src/test/resources/golden/stats_noscan_1-22-521b49d223a32056025fb8dbd371a72a new file mode 100644 index 0000000000000..ba90d8d3d2477 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 b/sql/hive/src/test/resources/golden/stats_noscan_1-23-fba14d8647e0c8ca214b724486af7763 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_noscan_1-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c b/sql/hive/src/test/resources/golden/stats_noscan_1-4-7938a68104e163566da69ccc70a18f2c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc b/sql/hive/src/test/resources/golden/stats_noscan_1-5-66590e5ed0bd8cd22ae01ecb658c11dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 b/sql/hive/src/test/resources/golden/stats_noscan_1-6-a1fd405e5175757aaa80033162c87670 new file mode 100644 index 0000000000000..b671e68cc4c3b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e b/sql/hive/src/test/resources/golden/stats_noscan_1-7-497861ae04753ffbb63ee43d89eedd9e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae b/sql/hive/src/test/resources/golden/stats_noscan_1-8-623f3701ead20fff786c203d23dd60ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 b/sql/hive/src/test/resources/golden/stats_noscan_1-9-a9ee4584e32fa7b934a1e6ab5403de44 new file mode 100644 index 0000000000000..96499c0fb0f7e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 b/sql/hive/src/test/resources/golden/stats_partscan_1-10-c06f6ce9878b7eededf8c2a085ffb380 new file mode 100644 index 0000000000000..b5fc469438c83 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1-11-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef b/sql/hive/src/test/resources/golden/stats_partscan_1-12-3e423642a5a00dc66cc709f474a3ecef new file mode 100644 index 0000000000000..cc58ef9026786 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 b/sql/hive/src/test/resources/golden/stats_partscan_1-13-2fe3131322b6c82e217f27e95581e681 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c b/sql/hive/src/test/resources/golden/stats_partscan_1-14-2c66f128acea649e8debc3c4b04fcb9c new file mode 100644 index 0000000000000..3243fe9fb497d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 b/sql/hive/src/test/resources/golden/stats_partscan_1-15-c05c5dc6a34b2a55526a43f2c900ad40 new file mode 100644 index 0000000000000..6accd64d06d69 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 b/sql/hive/src/test/resources/golden/stats_partscan_1-16-f93e6c408fcb4570fda5b09e4c7b1040 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1-4-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1-5-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1-6-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1-7-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 b/sql/hive/src/test/resources/golden/stats_partscan_1-8-29279aa84d6ba9dea9e56b493504dd30 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 b/sql/hive/src/test/resources/golden/stats_partscan_1-9-90d41ae72606f9616cb7b1303f997348 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f b/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1_23-0-e7bfbd9422685e9a3a6c9bd4965f828f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1_23-1-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-10-c06f6ce9878b7eededf8c2a085ffb380 new file mode 100644 index 0000000000000..942b7cfe07235 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1_23-11-418ec894d08c33fd712eb358f579b7a0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef b/sql/hive/src/test/resources/golden/stats_partscan_1_23-12-3e423642a5a00dc66cc709f474a3ecef new file mode 100644 index 0000000000000..9c704a6ef4126 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-13-2fe3131322b6c82e217f27e95581e681 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c b/sql/hive/src/test/resources/golden/stats_partscan_1_23-14-2c66f128acea649e8debc3c4b04fcb9c new file mode 100644 index 0000000000000..feee75f095d0d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-15-c05c5dc6a34b2a55526a43f2c900ad40 new file mode 100644 index 0000000000000..f39d366764c95 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-16-f93e6c408fcb4570fda5b09e4c7b1040 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1_23-2-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1_23-3-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf b/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1_23-4-c95dc367df88c9e5cf77157f29ba2daf @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1_23-5-6e53a3ac93113f20db3a12f1dcf30e86 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1_23-6-84967075baa3e56fff2a23f8ab9ba076 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea b/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_partscan_1_23-7-2ee5d706fe3a3bcc38b795f6e94970ea @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca b/sql/hive/src/test/resources/golden/stats_partscan_1_23-8-b158e24051ecb66b8af46743917771ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 b/sql/hive/src/test/resources/golden/stats_partscan_1_23-9-90d41ae72606f9616cb7b1303f997348 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-0-836d3689af091fdebbdf31e6b29a423 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-1-887fe99770f53e7e0a0fbdc190118612 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-10-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-11-50519f196d320ee97be731907409fef9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c b/sql/hive/src/test/resources/golden/stats_publisher_error_1-12-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-13-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-14-f596f33c4a9271a406371baef799e52 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c b/sql/hive/src/test/resources/golden/stats_publisher_error_1-15-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-16-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-2-2ca079278e0de95eecb5df315ce05c6 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-3-66e3e0c942759f679c270698b49bfcf1 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-4-d389db66cc7fd9b144445e364dac30e3 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-5-f39234f16f5819a2f51d3f200293332 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c b/sql/hive/src/test/resources/golden/stats_publisher_error_1-6-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 b/sql/hive/src/test/resources/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 new file mode 100644 index 0000000000000..1b79f38e25b24 --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-7-a31221a0c377c14e11b14484ddaa49a6 @@ -0,0 +1 @@ +500 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b b/sql/hive/src/test/resources/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/stats_publisher_error_1-8-df9d7a7da57e4d207c0bcae2bfa4bf6b @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c b/sql/hive/src/test/resources/golden/stats_publisher_error_1-9-3b7c5bcb71c75e913caaea3acd48530c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 b/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 new file mode 100644 index 0000000000000..9bed96d855476 --- /dev/null +++ b/sql/hive/src/test/resources/golden/str_to_map-0-aefd618b58ad6c37956755b6572cbc73 @@ -0,0 +1 @@ +str_to_map(text, delimiter1, delimiter2) - Creates a map by parsing text diff --git a/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c b/sql/hive/src/test/resources/golden/str_to_map-1-c66590cd4ac0a6f9c2bd88db7efcb16c new file mode 100644 index 0000000000000..1da11fbfa31d5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 b/sql/hive/src/test/resources/golden/str_to_map-10-32997010bba305ec40812df254490730 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a b/sql/hive/src/test/resources/golden/str_to_map-11-d99f1b631bc42a6a57c581025230537a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e b/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e new file mode 100644 index 0000000000000..cd4d4a384163b --- /dev/null +++ b/sql/hive/src/test/resources/golden/str_to_map-12-f793eb7b36a1d6379b90e241df62c72e @@ -0,0 +1,3 @@ +444 +444 +444 diff --git a/sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 b/sql/hive/src/test/resources/golden/str_to_map-13-32997010bba305ec40812df254490730 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec b/sql/hive/src/test/resources/golden/str_to_map-2-28d8e6677d025e1266ce95ae23dfd9ec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 b/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 new file mode 100644 index 0000000000000..e8183f05f5db6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/str_to_map-3-8b76ce17aa29e2eb37a4b953e9c80e66 @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 b/sql/hive/src/test/resources/golden/str_to_map-4-f356516aec917fe0d34df4dc8d9d0f95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda b/sql/hive/src/test/resources/golden/str_to_map-5-269cfcefe2ea6c19ac6c824d3259bbda new file mode 100644 index 0000000000000..62813f9d10491 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf b/sql/hive/src/test/resources/golden/str_to_map-6-53a3d5f99c0fbe17179cb01387a7ccaf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad b/sql/hive/src/test/resources/golden/str_to_map-7-5641678c53ce6ef1dbce3994843cfcad new file mode 100644 index 0000000000000..62813f9d10491 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 b/sql/hive/src/test/resources/golden/str_to_map-8-84121d964faad3547f0e5cce9d268612 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd b/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd new file mode 100644 index 0000000000000..e8183f05f5db6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/str_to_map-9-903f3b3137bfa3f4594fec7b604c62bd @@ -0,0 +1,3 @@ +1 +1 +1 diff --git a/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 b/sql/hive/src/test/resources/golden/string literal-0-ff43f1a7d06479b78622c8bb10e9f9a1 new file mode 100644 index 0000000000000..8938b37682023 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 b/sql/hive/src/test/resources/golden/subq2-0-279fb87476c70e9384874fa6bf77a380 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca b/sql/hive/src/test/resources/golden/subq2-1-235919a7ddb574662158503b8052e7ca new file mode 100644 index 0000000000000..5b3f76a5f7797 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 b/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/subquery-alias.attr-0-fc8183d758151be72b3d75d9df124504 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e b/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/table.attr-0-26c9d24eb6305ea2106c26bdca38012e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 b/sql/hive/src/test/resources/golden/tablename_with_select-0-554898fe9d96fee7b652b8a771d66a25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 b/sql/hive/src/test/resources/golden/tablename_with_select-1-736d6a05e1fe3249a84eb58f7cd806d2 new file mode 100644 index 0000000000000..92cca3b6f1e24 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d b/sql/hive/src/test/resources/golden/tablename_with_select-2-58d48c815413287858e2cfa16e5c6a5d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 b/sql/hive/src/test/resources/golden/tablename_with_select-3-35c08c648a66f09f2cf7cfa6019c2113 new file mode 100644 index 0000000000000..55d9485999072 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 b/sql/hive/src/test/resources/golden/test_boolean_whereclause-0-b38bf01368da26ec9c60e9433a9c59a1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c b/sql/hive/src/test/resources/golden/test_boolean_whereclause-1-3e38e42c5b72364c5461c626f312be8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 b/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/test_boolean_whereclause-2-183920d856ad75e6d1e15121d3cd7364 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 b/sql/hive/src/test/resources/golden/timestamp_1-0-d362501d0176855077e65f8faf067fa8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 b/sql/hive/src/test/resources/golden/timestamp_1-1-1d7cf3a2512fa1876b422b79bbe05426 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 b/sql/hive/src/test/resources/golden/timestamp_1-2-74f477a8b726f5193dd42ef378a793c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 b/sql/hive/src/test/resources/golden/timestamp_2-0-ea7192a4a5a985bcc8aab9aa79d9f028 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 b/sql/hive/src/test/resources/golden/timestamp_2-1-61dbdf6d26c2a3f1143f6fdae999b1b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 b/sql/hive/src/test/resources/golden/timestamp_2-2-a5092ff0f5a3d3b8f4171994932d4d19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 b/sql/hive/src/test/resources/golden/timestamp_3-0-165256158e3db1ce19c3c9db3c8011d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b b/sql/hive/src/test/resources/golden/timestamp_3-1-81edf5107270547641586aa02b4e7d9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 b/sql/hive/src/test/resources/golden/timestamp_3-2-7a012a0d98729da25b5ac374855dcee4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 b/sql/hive/src/test/resources/golden/timestamp_lazy-0-c0b1d928c9dea9b247ace0a056586594 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc b/sql/hive/src/test/resources/golden/timestamp_lazy-1-d15685a0b8e8991b4f8d74568398bcdc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 b/sql/hive/src/test/resources/golden/timestamp_null-0-3bd56783b637a47f6447d247024941a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb b/sql/hive/src/test/resources/golden/timestamp_null-1-4b9c19c1f57557979d3483d548f762cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 b/sql/hive/src/test/resources/golden/timestamp_null-2-b3071984dee42c9e698e947fcbc2a1e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af b/sql/hive/src/test/resources/golden/timestamp_udf-0-79914c5347620c6e62a8e0b9a95984af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d b/sql/hive/src/test/resources/golden/timestamp_udf-1-59fc1842a23369235d42ed040d45fb3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 b/sql/hive/src/test/resources/golden/timestamp_udf-2-9039f474f9a96e9f15ace528faeed923 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f b/sql/hive/src/test/resources/golden/timestamp_udf-3-b0fd4ca3b22eb732a32772399331352f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/touch-0-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-1-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/touch-1-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-10-27ad2962fed131f51ba802596ba37278 b/sql/hive/src/test/resources/golden/touch-10-27ad2962fed131f51ba802596ba37278 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-2-3e95421993ab28d18245ec2340f580a3 b/sql/hive/src/test/resources/golden/touch-2-3e95421993ab28d18245ec2340f580a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-3-c0c18ac884677231a41eea8d980d0451 b/sql/hive/src/test/resources/golden/touch-3-c0c18ac884677231a41eea8d980d0451 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a b/sql/hive/src/test/resources/golden/touch-4-c06da7f8c1e98dc22e3171018e357f6a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c b/sql/hive/src/test/resources/golden/touch-5-af459a0264559a2aeaa1341ce779ab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 b/sql/hive/src/test/resources/golden/touch-6-5c7b24abea435f2628fe618f3a82e115 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-7-3752320b12abae0d138148d56a27c5b1 b/sql/hive/src/test/resources/golden/touch-7-3752320b12abae0d138148d56a27c5b1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-8-3562ab40242756452595cd7eae79b0ce b/sql/hive/src/test/resources/golden/touch-8-3562ab40242756452595cd7eae79b0ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 b/sql/hive/src/test/resources/golden/touch-9-cd46bc635e3010cf1b990a652a584a09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe b/sql/hive/src/test/resources/golden/transform-0-d81d055660f6ef3d9cc60dd673a8c0fe new file mode 100644 index 0000000000000..d23e05acf7ba5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 b/sql/hive/src/test/resources/golden/transform1-0-b6919fc48901e388c869c84ae0211102 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 b/sql/hive/src/test/resources/golden/transform1-1-bb8804b6b511cb7e0c4dbdf2b978f737 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 b/sql/hive/src/test/resources/golden/transform1-2-25d6cab86c36d65fabf5645db3126a19 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 b/sql/hive/src/test/resources/golden/transform1-3-8324a70d533a06a5171c1016b1fea7c3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 b/sql/hive/src/test/resources/golden/transform1-4-65527bae8e73262255ef83082c6968f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 b/sql/hive/src/test/resources/golden/transform1-5-e0037a3f97ce0127a40d163af4c20ad5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f b/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f new file mode 100644 index 0000000000000..c6f628b1a3eef --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform1-6-3b862abd732c9e9f0db50ad0b9dae6f @@ -0,0 +1 @@ +[0,1,2] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform_ppr1-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 b/sql/hive/src/test/resources/golden/transform_ppr1-1-65fe307c21b350846b3c496890b0b619 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 b/sql/hive/src/test/resources/golden/transform_ppr1-2-8de14457978564d5fe4ab9a1c2a87d47 new file mode 100644 index 0000000000000..839efe2d57956 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 b/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/transform_ppr2-0-ae225e86c2ae20519ffdf23190454161 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 b/sql/hive/src/test/resources/golden/transform_ppr2-1-6133f48deec4cdf525b8c1574e7247d1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 b/sql/hive/src/test/resources/golden/transform_ppr2-2-636c4938673a273299c8569295d27c99 new file mode 100644 index 0000000000000..839efe2d57956 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d b/sql/hive/src/test/resources/golden/trivial join ON clause-0-3b6afcbd622aa111ee260bebc763613d new file mode 100644 index 0000000000000..66fafbdf72b63 --- /dev/null +++ b/sql/hive/src/test/resources/golden/trivial 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/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 b/sql/hive/src/test/resources/golden/trivial join where clause-0-25ffeb9d5e570c8b62b6ae2829655fe3 new file mode 100644 index 0000000000000..66fafbdf72b63 --- /dev/null +++ b/sql/hive/src/test/resources/golden/trivial 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/sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 b/sql/hive/src/test/resources/golden/type_widening-0-630ac2c7e7dea4837384ccd572209229 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 b/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/type_widening-1-cfbdf2b6fca84c6e23d4e691d2221bd6 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 b/sql/hive/src/test/resources/golden/type_widening-2-a0ba6952d9bf830d1d1ea7aebd3784a2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 b/sql/hive/src/test/resources/golden/type_widening-3-65da8c67f6903286168acb39ac67fc04 new file mode 100644 index 0000000000000..cf940f4c5faa8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e b/sql/hive/src/test/resources/golden/udaf_collect_set-0-38512a3299e2390dd813e61a0f63f35e new file mode 100644 index 0000000000000..fed3d6802023a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 b/sql/hive/src/test/resources/golden/udaf_collect_set-1-78aa199d061d2ff9ba426849ea1eb449 new file mode 100644 index 0000000000000..fed3d6802023a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-10-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..cedc3068ee4a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-11-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-12-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-13-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..cedc3068ee4a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-2-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-3-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-4-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..cedc3068ee4a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-5-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-6-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db b/sql/hive/src/test/resources/golden/udaf_collect_set-7-c8bc33095e1a195bb7b5e579d8d78db new file mode 100644 index 0000000000000..cedc3068ee4a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-8-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_collect_set-9-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 b/sql/hive/src/test/resources/golden/udaf_corr-0-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 b/sql/hive/src/test/resources/golden/udaf_corr-1-930b47e12b5ea559387ec5c6e8d5d9b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 b/sql/hive/src/test/resources/golden/udaf_corr-10-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_corr-2-c6f2dc536bf105650a461816ae5e330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e b/sql/hive/src/test/resources/golden/udaf_corr-3-ddf417dbc3b551cc8df47d950cec03e new file mode 100644 index 0000000000000..6e5d422b3132e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a b/sql/hive/src/test/resources/golden/udaf_corr-4-8771b2d83d14b3b641e8a77fcdc5081a new file mode 100644 index 0000000000000..fcb49ae69f74f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb b/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_corr-5-8abbd73784728b599f8c2a90f53da9fb @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 b/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_corr-6-4324e1f0a83a7491f3d4e3eef34f8727 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 b/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_corr-7-70e701f50c3418ff91649b2bd8287da3 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 b/sql/hive/src/test/resources/golden/udaf_corr-8-f2f0c7735f8b24266d5aaff96644e369 new file mode 100644 index 0000000000000..3f730875aef8c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 b/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 new file mode 100644 index 0000000000000..5d97236e8b03f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_corr-9-e2a0fa75c43279764ebca015f62bcf16 @@ -0,0 +1 @@ +0.6633880657639323 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 b/sql/hive/src/test/resources/golden/udaf_covar_pop-0-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 b/sql/hive/src/test/resources/golden/udaf_covar_pop-1-930b47e12b5ea559387ec5c6e8d5d9b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 b/sql/hive/src/test/resources/golden/udaf_covar_pop-10-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_covar_pop-2-c6f2dc536bf105650a461816ae5e330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 b/sql/hive/src/test/resources/golden/udaf_covar_pop-3-fe27ea6dae14250e8835baef3c2e20f6 new file mode 100644 index 0000000000000..0fed030c22af3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad b/sql/hive/src/test/resources/golden/udaf_covar_pop-4-7af9231ae293d0c4b84050176e1d73ad new file mode 100644 index 0000000000000..4037062d2da93 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd b/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-5-22004d36f6f3770db284644317770fcd @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b b/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-6-bc03cfbf7ae382ce707bf83e7fb2fb8b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae b/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-7-37e59e993e08216e6c69f88d6ac673ae @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f b/sql/hive/src/test/resources/golden/udaf_covar_pop-8-1e51388408dad651127edf940c11d91f new file mode 100644 index 0000000000000..848e15bc61476 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 b/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 new file mode 100644 index 0000000000000..1a49bf590b346 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_covar_pop-9-b3cc8c5b5b384622e212dbaaf3f09623 @@ -0,0 +1 @@ +3.624999999999999 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 b/sql/hive/src/test/resources/golden/udaf_covar_samp-0-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 b/sql/hive/src/test/resources/golden/udaf_covar_samp-1-930b47e12b5ea559387ec5c6e8d5d9b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 b/sql/hive/src/test/resources/golden/udaf_covar_samp-10-2f783a39471a7f1df9a824d741c443a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 b/sql/hive/src/test/resources/golden/udaf_covar_samp-2-c6f2dc536bf105650a461816ae5e330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf b/sql/hive/src/test/resources/golden/udaf_covar_samp-3-7aa25da7ccb88ba67b100888b6227aaf new file mode 100644 index 0000000000000..97a97c8b5965a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c b/sql/hive/src/test/resources/golden/udaf_covar_samp-4-7e705a637950911e0a18059d8bf1fd2c new file mode 100644 index 0000000000000..e666adba8df68 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b b/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-5-2cfd48c62fcfb58936ea1222ecf6a2b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 b/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-6-8a701e7f4b9ce986f7923ae083bce0f1 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 b/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-7-2702986700ff9a1c962f8b3762c1b5f3 @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a b/sql/hive/src/test/resources/golden/udaf_covar_samp-8-44861ae58cf0951eeda28a5f778f778a new file mode 100644 index 0000000000000..848e15bc61476 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 b/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 new file mode 100644 index 0000000000000..30d98234a23e9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_covar_samp-9-234a5b02085d2228473d7ef15a6af683 @@ -0,0 +1 @@ +4.833333333333332 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-0-86b9fb8bef8a5c51077623f1db3a0251 new file mode 100644 index 0000000000000..3e8bb17f24d6f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-1-5303011455a61171f7eb7eb4bd0ca2a3 new file mode 100644 index 0000000000000..048770d4c8402 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-2-b3b431c36154a1ae022bf01d55a6ecb1 new file mode 100644 index 0000000000000..8b54db8da7167 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 b/sql/hive/src/test/resources/golden/udaf_histogram_numeric-3-ff41f4450d6ae372633fde865ae187c6 new file mode 100644 index 0000000000000..aed3f1e704750 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 b/sql/hive/src/test/resources/golden/udaf_number_format-0-eff4ef3c207d14d5121368f294697964 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 b/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 new file mode 100644 index 0000000000000..2953abcf1e644 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_number_format-1-4a03c4328565c60ca99689239f07fb16 @@ -0,0 +1 @@ +0.0 NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-0-9ce9365f7b3f35a10b5305251c3e81ac new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-1-c7d32089880679d178dea94f1fe118e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-10-b7e588217a3cd184dbbb8d419d3e33ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-11-5034ec132cb8b0a6bd6357a7e1abd755 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-12-914ba18b45a27894bd82302f07efc789 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-13-4bd5703fa32f3283f38841acadc97adb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-14-d861a06b90896a097901d64ab9fbec53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-15-ca796efecd0d064e9e688a17ce75d80f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-16-c838e13d9aafe1212a76d2cf5fe085a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-17-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-18-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-19-b931aec4add0a66c23e444cdd5c33c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-2-ac53a7ba5e8a208255008d3a71fa321a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-20-6f85afbfa98a19d78ab7fd9d46ed3c0c @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-21-3cd4e1282d82d07785051a1cf0e9b4ff @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-22-ed1aec1a908310db90c5f8667631a1df @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-23-333d72e8bce6d11a35fc7a30418f225b @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-24-61903781f5cd75e6f11d85e7e89c1cb3 @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-25-15f40568b41c4505841f5ad13c526f51 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-27-ee4c065e8557632a9ee348dd9223c3a1 new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-28-ba77d1a26f87385f046129b6eb7d2ec3 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-29-956d6b6bc69c8035f80de2e60eda65fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-3-1dae5b2a11507c83b0f76e677a368712 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-30-d196cc7f52bb6ae19a5e66eb2a99577c new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-32-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-33-f28c7b0408737da815493741c806ff80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-34-6f85afbfa98a19d78ab7fd9d46ed3c0c @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-35-3cd4e1282d82d07785051a1cf0e9b4ff @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-36-ed1aec1a908310db90c5f8667631a1df @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-37-333d72e8bce6d11a35fc7a30418f225b @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-38-61903781f5cd75e6f11d85e7e89c1cb3 @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-39-15f40568b41c4505841f5ad13c526f51 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-4-b2e21ffa55342d4f3c243728dfe6b11f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-41-ee4c065e8557632a9ee348dd9223c3a1 new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-42-ba77d1a26f87385f046129b6eb7d2ec3 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-43-956d6b6bc69c8035f80de2e60eda65fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-44-d196cc7f52bb6ae19a5e66eb2a99577c new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-5-8ae1465266d28bc2e5da8d89617873c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-6-69cdebe8b2d4d2bbf2eef64a8c789596 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-7-ab438ea40bc5dddf76fd0a7a2529b8f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-8-7e3cf228c457279965b7414bd05527f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_20-9-5aea8aa95a85c46284f7c1f45978a228 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-0-477a942be95c0616c72f02a0077f9ace new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-1-c7d32089880679d178dea94f1fe118e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-10-b7e588217a3cd184dbbb8d419d3e33ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-11-5034ec132cb8b0a6bd6357a7e1abd755 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-12-914ba18b45a27894bd82302f07efc789 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-13-4bd5703fa32f3283f38841acadc97adb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-14-d861a06b90896a097901d64ab9fbec53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-15-ca796efecd0d064e9e688a17ce75d80f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-16-c838e13d9aafe1212a76d2cf5fe085a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-17-b89ea2173180c8ae423d856f943e061f @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-18-67e864faaff4c6b2a8e1c9fbd188bb66 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-19-b931aec4add0a66c23e444cdd5c33c5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-2-ac53a7ba5e8a208255008d3a71fa321a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-20-6f85afbfa98a19d78ab7fd9d46ed3c0c @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-21-3cd4e1282d82d07785051a1cf0e9b4ff @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-22-ed1aec1a908310db90c5f8667631a1df @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-23-333d72e8bce6d11a35fc7a30418f225b @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-24-61903781f5cd75e6f11d85e7e89c1cb3 @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-25-15f40568b41c4505841f5ad13c526f51 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-26-d1bc3b9a74fbf2ad41ddcd845ca9f0fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-27-ee4c065e8557632a9ee348dd9223c3a1 new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-28-ba77d1a26f87385f046129b6eb7d2ec3 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-29-956d6b6bc69c8035f80de2e60eda65fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-3-1dae5b2a11507c83b0f76e677a368712 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-30-d196cc7f52bb6ae19a5e66eb2a99577c new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-31-fe2c6a36a769f9f88a0ac9be1a4f0c28 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-32-dbcec232623048c7748b708123e18bf0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-33-f28c7b0408737da815493741c806ff80 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-34-6f85afbfa98a19d78ab7fd9d46ed3c0c @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-35-3cd4e1282d82d07785051a1cf0e9b4ff @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-36-ed1aec1a908310db90c5f8667631a1df @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-37-333d72e8bce6d11a35fc7a30418f225b @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 new file mode 100644 index 0000000000000..17c47d308f029 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-38-61903781f5cd75e6f11d85e7e89c1cb3 @@ -0,0 +1 @@ +254.08333333333334 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 new file mode 100644 index 0000000000000..60cbe79310729 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-39-15f40568b41c4505841f5ad13c526f51 @@ -0,0 +1 @@ +255.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-4-b2e21ffa55342d4f3c243728dfe6b11f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-40-d1bc3b9a74fbf2ad41ddcd845ca9f0fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-41-ee4c065e8557632a9ee348dd9223c3a1 new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-42-ba77d1a26f87385f046129b6eb7d2ec3 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-43-956d6b6bc69c8035f80de2e60eda65fb new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-44-d196cc7f52bb6ae19a5e66eb2a99577c new file mode 100644 index 0000000000000..014c315649096 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-45-fe2c6a36a769f9f88a0ac9be1a4f0c28 new file mode 100644 index 0000000000000..a8986e32ff75a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-5-8ae1465266d28bc2e5da8d89617873c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-6-69cdebe8b2d4d2bbf2eef64a8c789596 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-7-ab438ea40bc5dddf76fd0a7a2529b8f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-8-7e3cf228c457279965b7414bd05527f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 b/sql/hive/src/test/resources/golden/udaf_percentile_approx_23-9-5aea8aa95a85c46284f7c1f45978a228 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 b/sql/hive/src/test/resources/golden/udf1-0-7a6f90d9c0931145bee4fe4f5caa0859 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da b/sql/hive/src/test/resources/golden/udf1-1-8281592c818ada269024ac669bec78da new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f b/sql/hive/src/test/resources/golden/udf1-2-c7648c913ec336559fb67e3ab6938c8f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf1-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..493daf5d79c54 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf2-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 b/sql/hive/src/test/resources/golden/udf2-1-f60851dc36f579e83d6848d7d3c589e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf2-2-31c8af064bac42541558a95fad3bca97 b/sql/hive/src/test/resources/golden/udf2-2-31c8af064bac42541558a95fad3bca97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 b/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 new file mode 100644 index 0000000000000..398f517b25b5f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf2-3-c5938fcdd5675b58a4ed54269b5f5591 @@ -0,0 +1 @@ +| abc | abc | abc | \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c b/sql/hive/src/test/resources/golden/udf3-0-66a2b926dd5439d4da9eb58743c18a8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 b/sql/hive/src/test/resources/golden/udf3-1-1d04874d496d05cfe0b9d86de1111 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e b/sql/hive/src/test/resources/golden/udf3-2-25fe77d053e2bad8ae99757ce237052e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..b2890dc4cdb3a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf3-3-adc1ec67836b26b60d8547c4996bfd8f @@ -0,0 +1 @@ +0 NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf6-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 b/sql/hive/src/test/resources/golden/udf6-1-f60851dc36f579e83d6848d7d3c589e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 b/sql/hive/src/test/resources/golden/udf6-2-4d2a0815afe8c050cabf7a2efbce8521 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 b/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf6-3-e579646b969eef49b09656114da52a73 @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 b/sql/hive/src/test/resources/golden/udf6-4-29f45ffe530dd8c27dfb82268017dbb2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e b/sql/hive/src/test/resources/golden/udf6-5-fe336cd9850d6357980bd19139f76e new file mode 100644 index 0000000000000..132dbea91f8a0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf7-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 b/sql/hive/src/test/resources/golden/udf7-1-f60851dc36f579e83d6848d7d3c589e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 b/sql/hive/src/test/resources/golden/udf7-2-645d8fe7ab47806e0427c3deeedb5ec6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 b/sql/hive/src/test/resources/golden/udf7-3-b3afef6eb68f8e29e31d6bb6a7903045 new file mode 100644 index 0000000000000..51958ff45ab8d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf8-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf8-1-63c38297946a2060c0cff4a426b0520c b/sql/hive/src/test/resources/golden/udf8-1-63c38297946a2060c0cff4a426b0520c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 b/sql/hive/src/test/resources/golden/udf8-2-700c31cc5099ea2c3bbb9fbf5c43a32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb b/sql/hive/src/test/resources/golden/udf8-3-72c1a1dd1627491550f6e19581a654cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b b/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b new file mode 100644 index 0000000000000..8a89b039b7151 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf8-4-9f22d5a65353432826a526b1d76eb65b @@ -0,0 +1 @@ +1.0 1.0 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 b/sql/hive/src/test/resources/golden/udf9-0-a05de5714d8ccb2ac31dfe4f178fd358 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c b/sql/hive/src/test/resources/golden/udf9-1-dd0981dc44ac24d445af5412e9f7fa8c new file mode 100644 index 0000000000000..83e2e121222a2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf_10_trims-0-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e b/sql/hive/src/test/resources/golden/udf_10_trims-1-4534b1b2e9101058e8d71756bcb416e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 b/sql/hive/src/test/resources/golden/udf_10_trims-2-7624a192247c9c4c7be6a40d46f13597 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f b/sql/hive/src/test/resources/golden/udf_E-0-33251f00f840de3672f19d353fcfa66f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c b/sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c new file mode 100644 index 0000000000000..83593acfe8f2d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_E-1-d744dcbbb9d70b3dc4722b80ba9d929c @@ -0,0 +1 @@ +2.718281828459045 diff --git a/sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a b/sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a new file mode 100644 index 0000000000000..23a1652370236 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_E-2-72bb4231ea2a877b4d93a53cd7b6b82a @@ -0,0 +1 @@ +E() - returns E diff --git a/sql/hive/src/test/resources/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 b/sql/hive/src/test/resources/golden/udf_E-3-e8924af3bf99d2e01546a965303ffd09 new file mode 100644 index 0000000000000..8ad91668cd245 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f b/sql/hive/src/test/resources/golden/udf_E-4-33251f00f840de3672f19d353fcfa66f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c b/sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c new file mode 100644 index 0000000000000..83593acfe8f2d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_E-5-d744dcbbb9d70b3dc4722b80ba9d929c @@ -0,0 +1 @@ +2.718281828459045 diff --git a/sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a b/sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a new file mode 100644 index 0000000000000..23a1652370236 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_E-6-72bb4231ea2a877b4d93a53cd7b6b82a @@ -0,0 +1 @@ +E() - returns E diff --git a/sql/hive/src/test/resources/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 b/sql/hive/src/test/resources/golden/udf_E-7-e8924af3bf99d2e01546a965303ffd09 new file mode 100644 index 0000000000000..8ad91668cd245 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f b/sql/hive/src/test/resources/golden/udf_PI-0-b28e761e5564b51f98f182f561c1369f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 b/sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 new file mode 100644 index 0000000000000..bd1956ec6788c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_PI-1-1ea1f486385c62adeafcc5c52335bbf7 @@ -0,0 +1 @@ +3.141592653589793 diff --git a/sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 b/sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 new file mode 100644 index 0000000000000..dd554e4ac9c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_PI-2-9c1476a2eab7455594e97b338ee3c188 @@ -0,0 +1 @@ +PI() - returns pi diff --git a/sql/hive/src/test/resources/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 b/sql/hive/src/test/resources/golden/udf_PI-3-890f3c276eff2c459d8dc79d5a71c866 new file mode 100644 index 0000000000000..b66e03b623c75 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f b/sql/hive/src/test/resources/golden/udf_PI-4-b28e761e5564b51f98f182f561c1369f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 b/sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 new file mode 100644 index 0000000000000..bd1956ec6788c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_PI-5-1ea1f486385c62adeafcc5c52335bbf7 @@ -0,0 +1 @@ +3.141592653589793 diff --git a/sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 b/sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 new file mode 100644 index 0000000000000..dd554e4ac9c28 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_PI-6-9c1476a2eab7455594e97b338ee3c188 @@ -0,0 +1 @@ +PI() - returns pi diff --git a/sql/hive/src/test/resources/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 b/sql/hive/src/test/resources/golden/udf_PI-7-890f3c276eff2c459d8dc79d5a71c866 new file mode 100644 index 0000000000000..b66e03b623c75 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 b/sql/hive/src/test/resources/golden/udf_abs-0-6fe2e69c338fc823d3f61c9236eb2234 new file mode 100644 index 0000000000000..b613b3b9fe967 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 b/sql/hive/src/test/resources/golden/udf_abs-1-eeb77ae8a0dcebbc0991923ca0932072 new file mode 100644 index 0000000000000..c2fb6dc49322d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 b/sql/hive/src/test/resources/golden/udf_abs-2-6a0ea751ac70262d478b23888dcbdc96 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 b/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 new file mode 100644 index 0000000000000..f5a78bc6e59b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-3-52f5c6cba1b9d48046073a0c2e106530 @@ -0,0 +1 @@ +0 1 123 9223372036854775807 9223372036854775807 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c b/sql/hive/src/test/resources/golden/udf_abs-4-30cd5a94c13e1619ee18b9551db879c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 b/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 new file mode 100644 index 0000000000000..9e57fa65e8014 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_abs-5-343e899acb67c283391387f02aa7b5c4 @@ -0,0 +1 @@ +0.0 3.14159265 3.14159265 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 b/sql/hive/src/test/resources/golden/udf_acos-0-4f49cb5a5c87efea534d63ed76435f06 new file mode 100644 index 0000000000000..204db415144a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f b/sql/hive/src/test/resources/golden/udf_acos-1-d330511cf4f626cd844b18f57f99a85f new file mode 100644 index 0000000000000..d49af1cbf4800 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b b/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-2-86fca49baf270873b46709c9eaeab87b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 b/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 new file mode 100644 index 0000000000000..5548bdb7cf26a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-3-f7f199e5f3dde8056465d55aca29e884 @@ -0,0 +1 @@ +1.5707963267948966 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a b/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a new file mode 100644 index 0000000000000..1f2d6faad9a2c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-4-e66fd90808b7c0eacbfe7ddd8624d79a @@ -0,0 +1 @@ +2.0943951023931957 0.7208187608700897 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e b/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_acos-5-85869fd1e3a6fccaacd54a4315ae6d2e @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 b/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 new file mode 100644 index 0000000000000..e8f11c444a808 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_add-0-5db764318a918a5f6d7c1d95d9e86045 @@ -0,0 +1 @@ +a + b - Returns a+b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 b/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 new file mode 100644 index 0000000000000..e8f11c444a808 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_add-1-400b238f4e6cdf7120be566b0ef079c5 @@ -0,0 +1 @@ +a + b - Returns a+b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee b/sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee new file mode 100644 index 0000000000000..8c5ef9c86c107 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_array-0-2e9c2a7d9325bd9a53cf9f181b6333ee @@ -0,0 +1 @@ +array(n0, n1...) - Creates an array with the given elements diff --git a/sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a b/sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a new file mode 100644 index 0000000000000..8c5ef9c86c107 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_array-1-570741914bb78300b0233e5f38d7f08a @@ -0,0 +1 @@ +array(n0, n1...) - Creates an array with the given elements diff --git a/sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 b/sql/hive/src/test/resources/golden/udf_array-2-db6d3c6c67faed3ceb019cb1993502f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122 b/sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122 new file mode 100644 index 0000000000000..7b41de31897cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_array-3-a5d12c41277fb158e09281169c905122 @@ -0,0 +1 @@ +[] NULL [1,2,3] 3 ["1","a","2","3"] 2 2 diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a b/sql/hive/src/test/resources/golden/udf_array_contains-0-d9a90108b052b111e8de4433e008b25a new file mode 100644 index 0000000000000..783e7d086a5c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c b/sql/hive/src/test/resources/golden/udf_array_contains-1-eff16c7836252e01f3d8190cd833f79c new file mode 100644 index 0000000000000..0ccae5649af4a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 b/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_array_contains-2-42d966b28e61a465d638bffc20ac7247 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 b/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_array_contains-3-ec0cd851fd8135dd9bc822d9a0432569 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c b/sql/hive/src/test/resources/golden/udf_ascii-0-72924c23459330cca6a54c70b12a542c new file mode 100644 index 0000000000000..e121c27212c66 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 b/sql/hive/src/test/resources/golden/udf_ascii-1-dab8656d7f001e85615442d60df4b6b3 new file mode 100644 index 0000000000000..7c9fb835dfe57 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa b/sql/hive/src/test/resources/golden/udf_ascii-2-a9e207503f52d43903877fb998eabeaa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 b/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 new file mode 100644 index 0000000000000..726fda4c15ed3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_ascii-3-28fc6497c9835c2ef331aba44576f1b1 @@ -0,0 +1 @@ +70 0 33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd b/sql/hive/src/test/resources/golden/udf_asin-0-99da197a53767060e3fa0250254d59cd new file mode 100644 index 0000000000000..3a56bffc8da34 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 b/sql/hive/src/test/resources/golden/udf_asin-1-3d0c3fa6121f8f5158d221074f1d4129 new file mode 100644 index 0000000000000..ef207d3f5da38 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f b/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-2-da1b36ab73c6791eb3c4bd1dd2bec52f @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 b/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-3-b67069d37df3a7fb7a3e2138d8558343 @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 b/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 new file mode 100644 index 0000000000000..a9d9b65be4189 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-4-929dabad86ef0e564802de8f663a9e66 @@ -0,0 +1 @@ +-0.5235987755982989 0.7208187608700897 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 b/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_asin-5-1ee8715cce9831623d0af0031964d284 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-0-c79ed30c2444c8493d0db98c33c9132b new file mode 100644 index 0000000000000..019676ad50bc3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-1-77e7ac1b89a4eac9102176cd73f67a62 new file mode 100644 index 0000000000000..ce9e52716ff44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 b/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-2-bf1c7875364d5518e0ac9c1ac6943764 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 b/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 new file mode 100644 index 0000000000000..37b3e6018d62c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-3-9a6252f36fe5ec1387186bf47d74a139 @@ -0,0 +1 @@ +0.7853981633974483 1.4056476493802699 -0.7853981633974483 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b b/sql/hive/src/test/resources/golden/udf_atan-4-c79ed30c2444c8493d0db98c33c9132b new file mode 100644 index 0000000000000..019676ad50bc3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 b/sql/hive/src/test/resources/golden/udf_atan-5-77e7ac1b89a4eac9102176cd73f67a62 new file mode 100644 index 0000000000000..ce9e52716ff44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 b/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-6-bf1c7875364d5518e0ac9c1ac6943764 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 b/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 new file mode 100644 index 0000000000000..37b3e6018d62c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_atan-7-9a6252f36fe5ec1387186bf47d74a139 @@ -0,0 +1 @@ +0.7853981633974483 1.4056476493802699 -0.7853981633974483 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 b/sql/hive/src/test/resources/golden/udf_avg-0-2d715528b290951fb9874f60d7e9b537 new file mode 100644 index 0000000000000..bd171b0e9a29c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f b/sql/hive/src/test/resources/golden/udf_avg-1-c707c56871a903e4e022b3df5c92fc3f new file mode 100644 index 0000000000000..bd171b0e9a29c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 b/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 new file mode 100644 index 0000000000000..27b6cd42b68cf --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bigint-0-6c5b1e4b9d725caeb786bb18448a7927 @@ -0,0 +1 @@ +There is no documentation for function 'bigint' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 b/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 new file mode 100644 index 0000000000000..27b6cd42b68cf --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bigint-1-4636e4f0083ea54814995a03b7c81202 @@ -0,0 +1 @@ +There is no documentation for function 'bigint' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b b/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b new file mode 100644 index 0000000000000..4790a2c1f9292 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-0-ebbb090f6fa6b322a52bec3ba19dfe5b @@ -0,0 +1 @@ +bin(n) - returns n in binary \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 b/sql/hive/src/test/resources/golden/udf_bin-1-843803a1b6ada107c11397af1a2f55d6 new file mode 100644 index 0000000000000..4d82712e858e4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b b/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b new file mode 100644 index 0000000000000..20fae133f0b4d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-2-5ee3932ab9cd164f1005a4413a68007b @@ -0,0 +1 @@ +1 0 101111101011100001101100101 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 b/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 new file mode 100644 index 0000000000000..76ea1be7f9214 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bin-3-b72fc578a7c677e15b8598248c81901 @@ -0,0 +1 @@ +1111111111111111111111111111111111111111111111111111111111111011 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d b/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d new file mode 100644 index 0000000000000..652a71b48011c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-0-abea2a2780fad42422774174cbfd603d @@ -0,0 +1 @@ +[13,2,4,8589934592,4096,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 new file mode 100644 index 0000000000000..6d061bf1198c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_and-1-414291f11711df40fb8362e0a0156b25 @@ -0,0 +1 @@ +[13,1,4,2,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_and-2-d550d017f9eb9176593719ea34eaae9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 b/sql/hive/src/test/resources/golden/udf_bitmap_and-3-a486db1f5a06f9623a0e5abd7737b0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 b/sql/hive/src/test/resources/golden/udf_bitmap_and-4-6320e1c4829024f819d3883df9a782c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 b/sql/hive/src/test/resources/golden/udf_bitmap_and-5-ff2860a163cbe78d5affac8047199296 new file mode 100644 index 0000000000000..43be09952b09c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_and-6-d550d017f9eb9176593719ea34eaae9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 b/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_empty-0-6c80525a4eab6761596e6ad5aa75bc90 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df b/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df new file mode 100644 index 0000000000000..02e4a84d62c4b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_empty-1-a03987655a167f5b18c37b643391a0df @@ -0,0 +1 @@ +false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c b/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c new file mode 100644 index 0000000000000..652a71b48011c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-0-377e873cf3cc23e9234ce9aa7c235d8c @@ -0,0 +1 @@ +[13,2,4,8589934592,4096,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef new file mode 100644 index 0000000000000..163f4734f4c21 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitmap_or-1-85cab84fba419b436b033e3ea07e02ef @@ -0,0 +1 @@ +[13,2,4,8589934592,4224,0] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_or-2-d550d017f9eb9176593719ea34eaae9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 b/sql/hive/src/test/resources/golden/udf_bitmap_or-3-a486db1f5a06f9623a0e5abd7737b0c6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-4-6320e1c4829024f819d3883df9a782c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 b/sql/hive/src/test/resources/golden/udf_bitmap_or-5-ea92fff4d814552b57535ed342a5dde0 new file mode 100644 index 0000000000000..c977a4ce25c3f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b b/sql/hive/src/test/resources/golden/udf_bitmap_or-6-d550d017f9eb9176593719ea34eaae9b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a b/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a new file mode 100644 index 0000000000000..71cadba7649e8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitwise_and-0-e2374700cd32add926992d5539bd463a @@ -0,0 +1 @@ +a & b - Bitwise and \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b b/sql/hive/src/test/resources/golden/udf_bitwise_and-1-2e63ac31262106160ab043027e356a4b new file mode 100644 index 0000000000000..ead02cba53aa4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f b/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f new file mode 100644 index 0000000000000..8b06e374c708d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitwise_not-0-34abab2f47f90f0f194ef44aed1cdd7f @@ -0,0 +1 @@ +~ n - Bitwise not \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d b/sql/hive/src/test/resources/golden/udf_bitwise_not-1-ccc0c59ea3e29b6d55e1edee7029155d new file mode 100644 index 0000000000000..413ca8a653e1f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 b/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 new file mode 100644 index 0000000000000..bdbd28cad97c4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitwise_or-0-bf5bd0beebdd89b3fbbbbb8195a9bbe6 @@ -0,0 +1 @@ +a | b - Bitwise or \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb b/sql/hive/src/test/resources/golden/udf_bitwise_or-1-272722c23fece2807c08191d3969c3bb new file mode 100644 index 0000000000000..4c116be2acae3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 b/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 new file mode 100644 index 0000000000000..b483f75f1b956 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_bitwise_xor-0-6b05142d0b7e444a3e52a11b59d60a16 @@ -0,0 +1 @@ +a ^ b - Bitwise exclusive or \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f b/sql/hive/src/test/resources/golden/udf_bitwise_xor-1-8fc9b9bf1aced42754419310784e0a9f new file mode 100644 index 0000000000000..03bf261d7cf51 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 b/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 new file mode 100644 index 0000000000000..8134698dfbdcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_boolean-0-a68fc036f5f5b7f9c85d71bb9627c749 @@ -0,0 +1 @@ +There is no documentation for function 'boolean' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e b/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e new file mode 100644 index 0000000000000..8134698dfbdcd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_boolean-1-23178b473a9d25d6be0abe378d44fb0e @@ -0,0 +1 @@ +There is no documentation for function 'boolean' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 b/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 new file mode 100644 index 0000000000000..645bd7302810f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-0-5bcbe4c0e0a75462160814a10b9449f4 @@ -0,0 +1 @@ +There is no documentation for function 'case' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 b/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 new file mode 100644 index 0000000000000..645bd7302810f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-1-54acf006155d8822a50e47729be24004 @@ -0,0 +1 @@ +There is no documentation for function 'case' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f b/sql/hive/src/test/resources/golden/udf_case-2-98ee676f92950375917f09d2e492253f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c b/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c new file mode 100644 index 0000000000000..1ca086d1dca4d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-3-ec7343402fd77807842a0eaf2497a47c @@ -0,0 +1 @@ +2 5 15 NULL 20 24 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 b/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 new file mode 100644 index 0000000000000..b920295f69a53 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_case-4-3b3e7ab775f45d24f39e281b0d2f8d76 @@ -0,0 +1 @@ +yo \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c b/sql/hive/src/test/resources/golden/udf_case_column_pruning-0-dd2d7a075df235f17c26bac8713e939c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 b/sql/hive/src/test/resources/golden/udf_case_thrift-0-e3c6c400a6588fd7b52f9540fd621cd8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 b/sql/hive/src/test/resources/golden/udf_case_thrift-1-406cf6f97996bd921cf22e1ddd2d9f96 new file mode 100644 index 0000000000000..d97a0ef52b2e8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 b/sql/hive/src/test/resources/golden/udf_ceil-0-4b40e67b8ca75729ab07df966d814e06 new file mode 100644 index 0000000000000..4854a3c7773ac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb b/sql/hive/src/test/resources/golden/udf_ceil-1-f410065d893a263f375fcf74072877bb new file mode 100644 index 0000000000000..23685df97892d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd b/sql/hive/src/test/resources/golden/udf_ceiling-0-d5685e38400e68341e36a59671dcbdfd new file mode 100644 index 0000000000000..bf61b7e89d8b3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df b/sql/hive/src/test/resources/golden/udf_ceiling-1-69b4ba577d2f0f18befdaa1ee7a858df new file mode 100644 index 0000000000000..79a06f0ee2a5d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 b/sql/hive/src/test/resources/golden/udf_coalesce-0-8d1c97c292c51447e58606f2cefa87c1 new file mode 100644 index 0000000000000..66313c54cc684 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 b/sql/hive/src/test/resources/golden/udf_coalesce-1-e2c086f5148f10264c09ecbd7284c809 new file mode 100644 index 0000000000000..04d69977c3df4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 b/sql/hive/src/test/resources/golden/udf_coalesce-2-bd78a25868e5598ea275e0be5e4c716 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 b/sql/hive/src/test/resources/golden/udf_coalesce-3-badb02b0dfa13d6a1519e6198bb511d2 new file mode 100644 index 0000000000000..cd5686d89eba4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 b/sql/hive/src/test/resources/golden/udf_coalesce-4-83f323874d7941c463ced2aee6cc5157 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 b/sql/hive/src/test/resources/golden/udf_coalesce-5-4bcad31a47f4bfc3baef75b65baa8082 new file mode 100644 index 0000000000000..98339f97ef2f8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 b/sql/hive/src/test/resources/golden/udf_compare_java_string-0-32b16ab99287db115e8de5214ac24b77 new file mode 100644 index 0000000000000..59d3f01051903 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 b/sql/hive/src/test/resources/golden/udf_concat-0-7bc53505a4e6587132870d8d0a704d2 new file mode 100644 index 0000000000000..a3998eeeee623 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 b/sql/hive/src/test/resources/golden/udf_concat-1-765c520f239cdff1ea3f8d22ba83e031 new file mode 100644 index 0000000000000..0afd542a05033 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 b/sql/hive/src/test/resources/golden/udf_concat-2-2c4f33a0b709a7d00c3083e8aa5fc0d5 new file mode 100644 index 0000000000000..69af4e63792dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 b/sql/hive/src/test/resources/golden/udf_concat_insert1-0-13ab74a58da514fe01dbeda0c3e79883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 b/sql/hive/src/test/resources/golden/udf_concat_insert1-1-ed0d453b9879faed8bf93913f7690545 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_concat_insert1-2-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..aecd5d9f2a5c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea b/sql/hive/src/test/resources/golden/udf_concat_insert2-0-659e06570690cceeb3f37e10e855d2ea new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 b/sql/hive/src/test/resources/golden/udf_concat_insert2-1-39829ebc03d9e047f3eaac80b4428768 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_concat_insert2-2-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..455cef3f57162 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 b/sql/hive/src/test/resources/golden/udf_concat_ws-0-4c8bd7794a527e544c838408313eeaa8 new file mode 100644 index 0000000000000..c901f70500cba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab b/sql/hive/src/test/resources/golden/udf_concat_ws-1-b8b80f7e9bf4348367444c73020b3cab new file mode 100644 index 0000000000000..3e751463e055d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c b/sql/hive/src/test/resources/golden/udf_concat_ws-2-ce7c8205cdc107e1fb865d7d48b84a3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 b/sql/hive/src/test/resources/golden/udf_concat_ws-3-b13a1f7f63d2a54efa331c82bd635d63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f b/sql/hive/src/test/resources/golden/udf_concat_ws-4-a507af4996b13433b0ae100fcb32358f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 b/sql/hive/src/test/resources/golden/udf_concat_ws-5-ca4f051369183cae36fc9a7bec6a9341 new file mode 100644 index 0000000000000..7896fd787f3a2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 b/sql/hive/src/test/resources/golden/udf_concat_ws-6-98276439c0605401ff89c6a5ae54be09 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c b/sql/hive/src/test/resources/golden/udf_concat_ws-7-8f08128276e7e50eeb5a6932c763126c new file mode 100644 index 0000000000000..a0ec688a3b084 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 b/sql/hive/src/test/resources/golden/udf_concat_ws-8-7c9629cc90e72046b61c0b83ebc7bab4 new file mode 100644 index 0000000000000..0c9f2d12ba117 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a b/sql/hive/src/test/resources/golden/udf_conv-0-d552befca345f3396464529cfde9f75a new file mode 100644 index 0000000000000..b8fbe88a19971 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab b/sql/hive/src/test/resources/golden/udf_conv-1-5e5904af10b5d23f20ae28dc4b9a49ab new file mode 100644 index 0000000000000..4f4b0c594c459 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 b/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 new file mode 100644 index 0000000000000..9512cc4241554 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-10-5d38e8d3f2d4c89d57d916c3a5891a52 @@ -0,0 +1,3 @@ +EE 568 +56 134 +137 785 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 b/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 new file mode 100644 index 0000000000000..91e56a11fa628 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-2-6d61a2118b54843716aef87fe539b595 @@ -0,0 +1 @@ +3HL 22 33 116ED2B2FB4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d b/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d new file mode 100644 index 0000000000000..08f70d742b8e4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-3-97161f7a60851d445b23c4ebe4095a1d @@ -0,0 +1 @@ +-641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 b/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 new file mode 100644 index 0000000000000..a2a44daa718cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-4-568e843076f358c404a8634b18541c55 @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 b/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 new file mode 100644 index 0000000000000..6948fe3783119 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-5-3f23d98799b825a2e9594066f973d183 @@ -0,0 +1 @@ +5 NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a b/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a new file mode 100644 index 0000000000000..e1021e50fdcbb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-6-4981b5b92d87fd1000fa0ac26fa6163a @@ -0,0 +1 @@ +3HL 22 33 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e b/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e new file mode 100644 index 0000000000000..08f70d742b8e4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-7-77bd25ad13e3697c80464e4a2682360e @@ -0,0 +1 @@ +-641 B FFFFFFFFFFFFFFFF FFFFFFFFFFFFFFF1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb b/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb new file mode 100644 index 0000000000000..a2a44daa718cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-8-2fae52d392251be476e0c8f6071a4aeb @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFF -1 FFFFFFFFFFFFFFFF -1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 b/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 new file mode 100644 index 0000000000000..6948fe3783119 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_conv-9-2f0098c00c10044522cd23a4a2f54957 @@ -0,0 +1 @@ +5 NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 b/sql/hive/src/test/resources/golden/udf_cos-0-44f411146a7190b89f2bc8b4aa61cae3 new file mode 100644 index 0000000000000..99a7d42bb5b57 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 b/sql/hive/src/test/resources/golden/udf_cos-1-176030bdf43ff83ed8b3112d0c79f2f5 new file mode 100644 index 0000000000000..196294de1f19f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 b/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_cos-2-542c7c9742bdb10b614298a0e9a6fa01 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 b/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 new file mode 100644 index 0000000000000..c0690acff887b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_cos-3-7f30fb51fe862ef46b1ccdb3f5f9a429 @@ -0,0 +1 @@ +0.5570225467662173 7.963267107332633E-4 0.8775825618903728 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 b/sql/hive/src/test/resources/golden/udf_count-0-534a9b25b87d09e418645b1216949560 new file mode 100644 index 0000000000000..e01715295adc8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f b/sql/hive/src/test/resources/golden/udf_count-1-d566feb21bc894b97e6416b65fe5c02f new file mode 100644 index 0000000000000..e01715295adc8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 b/sql/hive/src/test/resources/golden/udf_count-10-455b30e7df33c5eebcaacabe0a578483 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e b/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_count-11-3b201ca546a8b0b5e5afaa1ff6aaee3e @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e b/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_count-12-9f41ac3eb9a6e77b3d612afc2f2b8e0e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 b/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_count-13-9286bc5f08bf4db183719b1b49dc5b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_count-2-3344645abe535426307b9327b381fc85 b/sql/hive/src/test/resources/golden/udf_count-2-3344645abe535426307b9327b381fc85 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 b/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_count-3-e43165f41597d2a1c9e8cf780b99a4a8 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 b/sql/hive/src/test/resources/golden/udf_count-4-93223bab63eada6ece8bc176e89c87a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 b/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 new file mode 100644 index 0000000000000..e0da8ae09ae60 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_count-5-bdee61c35a27bfab974e2ba199d5dfa4 @@ -0,0 +1 @@ +309 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 b/sql/hive/src/test/resources/golden/udf_count-6-f8bcbbf6a5b8eb9b6d79c77264faa241 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 b/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 new file mode 100644 index 0000000000000..e0da8ae09ae60 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_count-7-b975ad0d5f293508ce4832a7b19399b6 @@ -0,0 +1 @@ +309 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 b/sql/hive/src/test/resources/golden/udf_count-8-911e83c85155e1f811ba2c75aabbc6b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 b/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 new file mode 100644 index 0000000000000..eb1f49486af7c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_count-9-75b3d8a0dac332ea00af5ef8971ca643 @@ -0,0 +1 @@ +500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 b/sql/hive/src/test/resources/golden/udf_date_add-0-74d34471bfa0880230d8e3351eb0ab45 new file mode 100644 index 0000000000000..83e0fc4e63a25 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 b/sql/hive/src/test/resources/golden/udf_date_add-1-efb60fcbd6d78ad35257fb1ec39ace2 new file mode 100644 index 0000000000000..83b9851499f71 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 b/sql/hive/src/test/resources/golden/udf_date_sub-0-f8fbf85026da1b0778fd325d9b5dae33 new file mode 100644 index 0000000000000..7dec81f2b6d4e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 b/sql/hive/src/test/resources/golden/udf_date_sub-1-7efeb74367835ade71e5e42b22f8ced4 new file mode 100644 index 0000000000000..105b63424062b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 b/sql/hive/src/test/resources/golden/udf_datediff-0-3bd040a96a4568e7ea4922faa408ada5 new file mode 100644 index 0000000000000..b59d456397d53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 b/sql/hive/src/test/resources/golden/udf_datediff-1-34ae7a68b13c2bc9a89f61acf2edd4c5 new file mode 100644 index 0000000000000..c240df94594d4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 b/sql/hive/src/test/resources/golden/udf_day-0-c4c503756384ff1220222d84fd25e756 new file mode 100644 index 0000000000000..11e32a4509b3f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 b/sql/hive/src/test/resources/golden/udf_day-1-87168babe1110fe4c38269843414ca4 new file mode 100644 index 0000000000000..9da0858ba92a5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 b/sql/hive/src/test/resources/golden/udf_dayofmonth-0-7b2caf942528656555cf19c261a18502 new file mode 100644 index 0000000000000..33e90a2af295f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 b/sql/hive/src/test/resources/golden/udf_dayofmonth-1-ca24d07102ad264d79ff30c64a73a7e8 new file mode 100644 index 0000000000000..ee9911af3248c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae b/sql/hive/src/test/resources/golden/udf_degrees-0-85f4957cd7cd6c517f6941af1289d8ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 b/sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 new file mode 100644 index 0000000000000..36f165be67119 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_degrees-1-aabc6065a03b7da809376cc127af47d7 @@ -0,0 +1 @@ +180.0 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b b/sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b new file mode 100644 index 0000000000000..60a5fff4a87a8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_degrees-2-2fd3a55901fe765f8f154531a7f5fd6b @@ -0,0 +1 @@ +degrees(x) - Converts radians to degrees diff --git a/sql/hive/src/test/resources/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/sql/hive/src/test/resources/golden/udf_degrees-3-42f653c3c3cc3c94bb9ab9c5a4d1ca96 new file mode 100644 index 0000000000000..4a33ce356e237 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae b/sql/hive/src/test/resources/golden/udf_degrees-4-85f4957cd7cd6c517f6941af1289d8ae new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 b/sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 new file mode 100644 index 0000000000000..36f165be67119 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_degrees-5-aabc6065a03b7da809376cc127af47d7 @@ -0,0 +1 @@ +180.0 diff --git a/sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b b/sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b new file mode 100644 index 0000000000000..60a5fff4a87a8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_degrees-6-2fd3a55901fe765f8f154531a7f5fd6b @@ -0,0 +1 @@ +degrees(x) - Converts radians to degrees diff --git a/sql/hive/src/test/resources/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 b/sql/hive/src/test/resources/golden/udf_degrees-7-42f653c3c3cc3c94bb9ab9c5a4d1ca96 new file mode 100644 index 0000000000000..4a33ce356e237 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db b/sql/hive/src/test/resources/golden/udf_div-0-31d31c7d5c544327dabfd874c88314db new file mode 100644 index 0000000000000..d98718752a36f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 b/sql/hive/src/test/resources/golden/udf_div-1-f23a07b1c6b1a98b303863188c10a8d8 new file mode 100644 index 0000000000000..59265a74f9b3f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b b/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_div-2-c71acf88a51fc6d2b23bbb91db2d7b @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 b/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 new file mode 100644 index 0000000000000..8b623e47785f6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_divide-0-1af8b249439ee5b7d4978c31669bc208 @@ -0,0 +1 @@ +a / b - Divide a by b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 b/sql/hive/src/test/resources/golden/udf_divide-1-fa932664bae88683a222b71ac45fb840 new file mode 100644 index 0000000000000..2acf2b6a64648 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 b/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 new file mode 100644 index 0000000000000..400122e60f599 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_divide-2-ce54773b1babe6dde982e4e91ebaeb50 @@ -0,0 +1 @@ +1.5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 b/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 new file mode 100644 index 0000000000000..54de9e9f8edec --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_double-0-aa32d73a99587cae2f1efd9a2094d617 @@ -0,0 +1 @@ +There is no documentation for function 'double' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 b/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 new file mode 100644 index 0000000000000..54de9e9f8edec --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_double-1-79380157cbd6624d760335f8291e6fb4 @@ -0,0 +1 @@ +There is no documentation for function 'double' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f b/sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f new file mode 100644 index 0000000000000..7a22029a2156f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_elt-0-b46b060da76d1772db998c26a62a608f @@ -0,0 +1 @@ +elt(n, str1, str2, ...) - returns the n-th string diff --git a/sql/hive/src/test/resources/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 b/sql/hive/src/test/resources/golden/udf_elt-1-e9f1bf17ad2a6f7bf3e40798ceebdbf4 new file mode 100644 index 0000000000000..61f204dacf9dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a b/sql/hive/src/test/resources/golden/udf_elt-2-5b58f1cfb0392452bf5c28a37d51508a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 b/sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 new file mode 100644 index 0000000000000..d009faddeafa8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_elt-3-f3be980cf4fa166f299c6ec79e981814 @@ -0,0 +1 @@ +defg cc abc 2 12345 123456789012 1.25 16.0 NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a b/sql/hive/src/test/resources/golden/udf_exp-0-814f16e1acabe30740d7b815e4b5cc3a new file mode 100644 index 0000000000000..9b3dbf316d1fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 b/sql/hive/src/test/resources/golden/udf_exp-1-d10d879c74951e9a1f1717cb1a2488c6 new file mode 100644 index 0000000000000..a42c95bb8d213 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 b/sql/hive/src/test/resources/golden/udf_field-0-277b4a4dcb38cabb6df50147c77e0a33 new file mode 100644 index 0000000000000..a30bc26f5ba58 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f b/sql/hive/src/test/resources/golden/udf_field-1-379d8580693493f620a8f4084709324f new file mode 100644 index 0000000000000..bb55c0a1db1f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 b/sql/hive/src/test/resources/golden/udf_field-10-7982ea72163dbc4cd45f53454edf66c8 new file mode 100644 index 0000000000000..275f46482425d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b b/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b new file mode 100644 index 0000000000000..c42823854fb0f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-2-d2c6583a79d77aabe388a52ec164c38b @@ -0,0 +1 @@ +0 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 b/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 new file mode 100644 index 0000000000000..5869234249808 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-3-fea09e934696af40bb604b40225bbc98 @@ -0,0 +1 @@ +1 2 3 4 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 b/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 new file mode 100644 index 0000000000000..5869234249808 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_field-4-b0815d34893d6cba8c07d0a0721c1d29 @@ -0,0 +1 @@ +1 2 3 4 4 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 b/sql/hive/src/test/resources/golden/udf_field-5-2d7c5cbe891c4a9dda34f9842f8e0828 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 b/sql/hive/src/test/resources/golden/udf_field-6-de02aaf3bbb137ba032810bb9ad7a3a3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef b/sql/hive/src/test/resources/golden/udf_field-7-6aa3518e9f55299754521e959e9376ef new file mode 100644 index 0000000000000..a76563207da24 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 b/sql/hive/src/test/resources/golden/udf_field-8-66dc6c81db0ac9b2075783b0d8976083 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 b/sql/hive/src/test/resources/golden/udf_field-9-f053f2d16812aa60b6dd1cab61e90a95 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 b/sql/hive/src/test/resources/golden/udf_find_in_set-0-18d3e88b18c18a00598146a3307276f2 new file mode 100644 index 0000000000000..f14679978b79e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 b/sql/hive/src/test/resources/golden/udf_find_in_set-1-5fb7a7a1725749dc3853eb80fba19246 new file mode 100644 index 0000000000000..1bebc68416bff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c b/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-10-16355c6b7e169b3c0ef506c149c6853c @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a b/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-11-5a8515684c458d3fffea539a3d170e3a @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 b/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-12-3fb21e2befb41ba72a1bbffa645c1e3 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a b/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a new file mode 100644 index 0000000000000..e440e5c842586 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-13-2c986a80620c9238e1f663fc591760a @@ -0,0 +1 @@ +3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 b/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-14-189def133b1871ce8345a8123811a6b5 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb b/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-15-671bff8f50feea55015a8412fc6e5ceb @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 b/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-16-d5d22082588c5fc30ef502237c5797f4 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 b/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 new file mode 100644 index 0000000000000..99f516951ae7d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-17-5b556a29e1685605bcc47bce60cf66c8 @@ -0,0 +1,2 @@ +311 val_311 +128 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 b/sql/hive/src/test/resources/golden/udf_find_in_set-2-b3823bdc04a7f98951b55c3e30d2a772 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e b/sql/hive/src/test/resources/golden/udf_find_in_set-3-132b7bc7812db7683eb3bff607275d0e new file mode 100644 index 0000000000000..5817cbcff62ac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b b/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-4-a35471c87ba597a6d3c7c880704cac0b @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f b/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-5-ddaa3551dffb1169b2fbb671f373b82f @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 b/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 new file mode 100644 index 0000000000000..e440e5c842586 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-6-591e070365f19c65e453b98b88f5f823 @@ -0,0 +1 @@ +3 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 b/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-7-72d05b5cf99388d539adec38c40978c3 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 b/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-8-780771cad9bec96a216aea8ab293c941 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 b/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_find_in_set-9-d59f5aabe1ea0963f9328065c699d175 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 b/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 new file mode 100644 index 0000000000000..9d15b5f5956d7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_float-0-7987032f1c9dcad07001445f3ca1f7a7 @@ -0,0 +1 @@ +There is no documentation for function 'float' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b b/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b new file mode 100644 index 0000000000000..9d15b5f5956d7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_float-1-2abdfb4c67fe3aec2bc9cc128f407e6b @@ -0,0 +1 @@ +There is no documentation for function 'float' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 b/sql/hive/src/test/resources/golden/udf_floor-0-e35abe1d5534243e96d71bd0c28761d6 new file mode 100644 index 0000000000000..c76710bfcc1b5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 b/sql/hive/src/test/resources/golden/udf_floor-1-497a1ddbcf738aead319fde4f90f5248 new file mode 100644 index 0000000000000..3f73eea16e183 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_format_number-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 b/sql/hive/src/test/resources/golden/udf_format_number-1-525f133cfff86d44afdeeda667c365a5 new file mode 100644 index 0000000000000..c981e1726c070 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd b/sql/hive/src/test/resources/golden/udf_format_number-2-591f302d5c1cd24e153a598fa0b352fd new file mode 100644 index 0000000000000..b2aa527ca0ddb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c b/sql/hive/src/test/resources/golden/udf_format_number-3-c89564db1ab953e28b050b9740f2650c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 b/sql/hive/src/test/resources/golden/udf_format_number-4-295d41a2146a27320c2be90499343260 new file mode 100644 index 0000000000000..89e118cc62bf3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 b/sql/hive/src/test/resources/golden/udf_format_number-5-881f33f6727a30629bde6e4b178cf7d9 new file mode 100644 index 0000000000000..33e21fa7dbfc4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 b/sql/hive/src/test/resources/golden/udf_format_number-6-a6720a128716e179e18933992ca899b3 new file mode 100644 index 0000000000000..07b05cd16f5a3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 b/sql/hive/src/test/resources/golden/udf_format_number-7-84a460780828b0b9a2235314cfc24766 new file mode 100644 index 0000000000000..2b600a6a83aa8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e b/sql/hive/src/test/resources/golden/udf_format_number-8-e7eedc849c74ce7d33c559067dd9ca0e new file mode 100644 index 0000000000000..d027b2cb0b94a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 b/sql/hive/src/test/resources/golden/udf_format_number-9-407a0a7c277bb4c5c94ce16533ce1646 new file mode 100644 index 0000000000000..afdec63c8dfca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f b/sql/hive/src/test/resources/golden/udf_from_unixtime-0-c3adaeede5c48d232473d78acf0eed7f new file mode 100644 index 0000000000000..10ce106f121ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 b/sql/hive/src/test/resources/golden/udf_from_unixtime-1-d1a511d2084c7c621b5f638908c8db65 new file mode 100644 index 0000000000000..20f4d2b35c49a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f b/sql/hive/src/test/resources/golden/udf_get_json_object-0-c08e7139c00878b98d396e65d958100f new file mode 100644 index 0000000000000..4e4f3f7255fd3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 b/sql/hive/src/test/resources/golden/udf_get_json_object-1-706bcfd51431ec7f2b80145837f94917 new file mode 100644 index 0000000000000..0e1dcf934a023 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 b/sql/hive/src/test/resources/golden/udf_get_json_object-2-2a18d9570d9b676e240cda76df818c42 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 b/sql/hive/src/test/resources/golden/udf_get_json_object-3-f60851dc36f579e83d6848d7d3c589e6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 b/sql/hive/src/test/resources/golden/udf_get_json_object-4-4f08101fd66fb25d7b322d47773e49f3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 b/sql/hive/src/test/resources/golden/udf_greaterthan-0-99d268829a124103cb3429c53fdc4de4 new file mode 100644 index 0000000000000..54d6096d07bbe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c b/sql/hive/src/test/resources/golden/udf_greaterthan-1-8aab8e39726a986e10e1e572939fd63c new file mode 100644 index 0000000000000..54d6096d07bbe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f b/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f new file mode 100644 index 0000000000000..679b0376125f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthan-2-79ba62f35a9432647e31c6722b44fe6f @@ -0,0 +1 @@ +true false false false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-0-a7214027a91abf6501881e2af313347a new file mode 100644 index 0000000000000..abf7dfdab730c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-1-3669f2008e7f428f365efadbcb5ae451 new file mode 100644 index 0000000000000..abf7dfdab730c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 new file mode 100644 index 0000000000000..1fb1894fc21f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_greaterthanorequal-2-d2690cc7713e91201bb10ef291c95819 @@ -0,0 +1 @@ +true false true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 b/sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 new file mode 100644 index 0000000000000..d1284cf87da3c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hash-0-b9e3a3986320d275982797140edfccf4 @@ -0,0 +1 @@ +hash(a1, a2, ...) - Returns a hash value of the arguments diff --git a/sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a b/sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a new file mode 100644 index 0000000000000..d1284cf87da3c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hash-1-a18646b51501d0b1beb967dc79afbd1a @@ -0,0 +1 @@ +hash(a1, a2, ...) - Returns a hash value of the arguments diff --git a/sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 b/sql/hive/src/test/resources/golden/udf_hash-2-cc121f3c38a7a522abd824940fe04285 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae b/sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae new file mode 100644 index 0000000000000..d50944d3d0c0f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hash-3-23991312391d518aacf3d4469c816eae @@ -0,0 +1 @@ +1 2 3 -1097262584 1067450368 1076887552 51508 96354 1 0 1026 diff --git a/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 b/sql/hive/src/test/resources/golden/udf_hex-0-c8b923c23d5eb31446780f28acbd4e16 new file mode 100644 index 0000000000000..f87a6117eacf7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c b/sql/hive/src/test/resources/golden/udf_hex-1-d55348c0ccd133b7abb690f6949b520c new file mode 100644 index 0000000000000..e8cee4afc3a80 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 b/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 new file mode 100644 index 0000000000000..34eb75a6c784a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-2-332aa8b643b3f6bebd32c558ad4b1559 @@ -0,0 +1 @@ +46616365626F6F6B 00 71776572747975696F706173646667686A6B6C \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf b/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf new file mode 100644 index 0000000000000..2a45250c67a63 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-3-963ad47b5fa3898a71f3c62f592e34bf @@ -0,0 +1 @@ +1 0 FACEB005 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea b/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea new file mode 100644 index 0000000000000..50d9557967ac4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hex-4-e07a02365ad1f5b47e0a08fec64aebea @@ -0,0 +1 @@ +FFFFFFFFFFFFFFFB \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce b/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce new file mode 100644 index 0000000000000..48911456dc339 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hour-0-ba1c46a403b807fe0a28b85e62d869ce @@ -0,0 +1 @@ +hour(date) - Returns the hour of date diff --git a/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 b/sql/hive/src/test/resources/golden/udf_hour-1-3db41b9fe9966a45e663bc42cf182c04 new file mode 100644 index 0000000000000..be1a966635202 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a b/sql/hive/src/test/resources/golden/udf_hour-2-ace1054795b20abd5ae829814cfe15a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 b/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 new file mode 100644 index 0000000000000..1a35f9f158133 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_hour-3-415b0842ab0818c82baf9fbf07180613 @@ -0,0 +1 @@ +13 13 NULL diff --git a/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c b/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c new file mode 100644 index 0000000000000..2cf0d9d61882e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_if-0-b7ffa85b5785cccef2af1b285348cc2c @@ -0,0 +1 @@ +There is no documentation for function 'if' diff --git a/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a b/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a new file mode 100644 index 0000000000000..2cf0d9d61882e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_if-1-30cf7f51f92b5684e556deff3032d49a @@ -0,0 +1 @@ +There is no documentation for function 'if' diff --git a/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 b/sql/hive/src/test/resources/golden/udf_if-2-f2b010128e922d0096a65ddd9ae1d0b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f b/sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f new file mode 100644 index 0000000000000..a29e96cbd1db7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_if-3-20206f17367ff284d67044abd745ce9f @@ -0,0 +1 @@ +1 1 1 1 NULL 2 diff --git a/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca b/sql/hive/src/test/resources/golden/udf_if-4-174dae8a1eb4cad6ccf6f67203de71ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc b/sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc new file mode 100644 index 0000000000000..f0669b86989d0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_if-5-a7db13aec05c97792f9331d63709d8cc @@ -0,0 +1 @@ +128 1.1 ABC 12.3 diff --git a/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 b/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 new file mode 100644 index 0000000000000..ee6e42ce6a83d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_in-0-b21369b3d0dd47d347e0e0af25f06ce4 @@ -0,0 +1 @@ +true false true true false true true true NULL NULL true diff --git a/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 b/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 new file mode 100644 index 0000000000000..993d93304f95e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_in-1-ce5f3a3da5f3602a23fc107325dd13d7 @@ -0,0 +1,3 @@ +238 +86 +238 diff --git a/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 b/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 new file mode 100644 index 0000000000000..cd15a08a539ab --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_in_file-0-1775b929e50cae8b3e957d99c5378f59 @@ -0,0 +1 @@ +in_file(str, filename) - Returns true if str appears in the file diff --git a/sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 b/sql/hive/src/test/resources/golden/udf_in_file-1-2f23153970a569a4643574dde8d78a58 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 b/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 new file mode 100644 index 0000000000000..df5a0561fb8f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_index-0-a277ac394cae40cb55d1ef3aa5add260 @@ -0,0 +1 @@ +Function '`index`' does not exist. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 b/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 new file mode 100644 index 0000000000000..df5a0561fb8f5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_index-1-1f5e109131b0c67ebea521fa4902a8f6 @@ -0,0 +1 @@ +Function '`index`' does not exist. \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 b/sql/hive/src/test/resources/golden/udf_instr-0-2e76f819563dbaba4beb51e3a130b922 new file mode 100644 index 0000000000000..ae27b5efea045 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 b/sql/hive/src/test/resources/golden/udf_instr-1-32da357fc754badd6e3898dcc8989182 new file mode 100644 index 0000000000000..35de2f0d8653f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 b/sql/hive/src/test/resources/golden/udf_instr-2-10147893f38fc08dad4fa7f2bc843fc2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 b/sql/hive/src/test/resources/golden/udf_instr-3-2cb7f5ef9ec07402e3cae7b5279ebe12 new file mode 100644 index 0000000000000..4d34e6df7039b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba b/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba new file mode 100644 index 0000000000000..cda33a8e2e555 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_int-0-c24c3d4e15b5cdf081fee0a8c8ea13ba @@ -0,0 +1 @@ +There is no documentation for function 'int' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 b/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 new file mode 100644 index 0000000000000..cda33a8e2e555 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_int-1-3f0405ff93adfe8b3402b118567867d7 @@ -0,0 +1 @@ +There is no documentation for function 'int' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 b/sql/hive/src/test/resources/golden/udf_isnotnull-0-44584503014c378bb916b38e1879bfb6 new file mode 100644 index 0000000000000..723b5aaf2af6b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 b/sql/hive/src/test/resources/golden/udf_isnotnull-1-6ca2ea7938c7dac331c603ad343c1a7 new file mode 100644 index 0000000000000..723b5aaf2af6b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 b/sql/hive/src/test/resources/golden/udf_isnull-0-ac8e7827d760108923509f9ea1691d53 new file mode 100644 index 0000000000000..99510c6f1db15 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 b/sql/hive/src/test/resources/golden/udf_isnull-1-55d9d04204f30cde4aa2667db88db262 new file mode 100644 index 0000000000000..99510c6f1db15 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-0-ac8e7827d760108923509f9ea1691d53 new file mode 100644 index 0000000000000..99510c6f1db15 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-1-55d9d04204f30cde4aa2667db88db262 new file mode 100644 index 0000000000000..99510c6f1db15 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-2-44584503014c378bb916b38e1879bfb6 new file mode 100644 index 0000000000000..723b5aaf2af6b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-3-6ca2ea7938c7dac331c603ad343c1a7 new file mode 100644 index 0000000000000..723b5aaf2af6b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-4-3dd03048c0152f565b21b6d3b7b010f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 new file mode 100644 index 0000000000000..eedfbc67d50c7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-5-253ed8f6f8659120af927871f57d81a1 @@ -0,0 +1 @@ +true true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-6-9daf0ab0e3b8315018341d6f72bd3174 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b new file mode 100644 index 0000000000000..eedfbc67d50c7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_isnull_isnotnull-7-bb1030dea18d2a0c2c00a6e2de835d6b @@ -0,0 +1 @@ +true true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 b/sql/hive/src/test/resources/golden/udf_java_method-0-991b98a25032b21802bc2a1efde606c7 new file mode 100644 index 0000000000000..b703a30abffba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a b/sql/hive/src/test/resources/golden/udf_java_method-1-a3b94d9f2c2caf85a588b6686a64630a new file mode 100644 index 0000000000000..07375dc92d625 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 b/sql/hive/src/test/resources/golden/udf_java_method-2-69e6b8725086a8fb8f55721705442112 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd b/sql/hive/src/test/resources/golden/udf_java_method-3-c526dfd4d9eac718ced9afb3cf9a62fd new file mode 100644 index 0000000000000..9b93703dae806 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad b/sql/hive/src/test/resources/golden/udf_lcase-0-649df2b742e6a03d0e0e364f5bee76ad new file mode 100644 index 0000000000000..ec6779df2818d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e b/sql/hive/src/test/resources/golden/udf_lcase-1-d947c47e03bedbfd4954853cc134c66e new file mode 100644 index 0000000000000..4da30f6c23a1b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b b/sql/hive/src/test/resources/golden/udf_length-0-38364b60c3a2409f53c9aa2dae19903b new file mode 100644 index 0000000000000..d4d2bd508b44f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 b/sql/hive/src/test/resources/golden/udf_length-1-f183e1f8ae516bb483132ed106289b67 new file mode 100644 index 0000000000000..79c1c54639266 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 b/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 new file mode 100644 index 0000000000000..d8263ee986059 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_length-10-f3a9bd30540345db0f69b6847014b333 @@ -0,0 +1 @@ +2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 b/sql/hive/src/test/resources/golden/udf_length-2-af46cb6887618240836eaf5be8afbba6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d b/sql/hive/src/test/resources/golden/udf_length-3-dcd6404afce1103d5054527e6c216d6d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 b/sql/hive/src/test/resources/golden/udf_length-4-ba49ba4e6505c74bc33b5003f3930c43 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f b/sql/hive/src/test/resources/golden/udf_length-5-adc1ec67836b26b60d8547c4996bfd8f new file mode 100644 index 0000000000000..6ecb2baabb297 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a b/sql/hive/src/test/resources/golden/udf_length-6-460dffb0f1ab0ac0ebc4fd545809aa9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f b/sql/hive/src/test/resources/golden/udf_length-7-8f28e6c488df47d89dca670f04a7563f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf b/sql/hive/src/test/resources/golden/udf_length-8-5e0fe761b7520651c3446ce7f9179caf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 b/sql/hive/src/test/resources/golden/udf_length-9-de456a5765db4a06110d9483985aa4a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 b/sql/hive/src/test/resources/golden/udf_lessthan-0-a0d9e8b51e3d13685b3889db38f22427 new file mode 100644 index 0000000000000..b43707d550c4b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 b/sql/hive/src/test/resources/golden/udf_lessthan-1-952c655a1092a410e5346f1205cb8142 new file mode 100644 index 0000000000000..b43707d550c4b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 b/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 new file mode 100644 index 0000000000000..4364012093724 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthan-2-92fa47f7af4a03ce1a965472eaad23a7 @@ -0,0 +1 @@ +false true false false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b b/sql/hive/src/test/resources/golden/udf_lessthanorequal-0-b3b021456c85da76d1879191886a425b new file mode 100644 index 0000000000000..c36acd7ce80bc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 b/sql/hive/src/test/resources/golden/udf_lessthanorequal-1-869d983466744ad73b109211e1638869 new file mode 100644 index 0000000000000..c36acd7ce80bc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c new file mode 100644 index 0000000000000..38e013b6914c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lessthanorequal-2-56775013e20ecf2287e07e83eccf2e0c @@ -0,0 +1 @@ +false true true true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 b/sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 new file mode 100644 index 0000000000000..b63094711df29 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_like-0-e0ba9a953e50554bdcbc55585cffde09 @@ -0,0 +1 @@ +like(str, pattern) - Checks if str matches pattern diff --git a/sql/hive/src/test/resources/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a b/sql/hive/src/test/resources/golden/udf_like-1-9781f89d352c506e972ad2a1d58ec03a new file mode 100644 index 0000000000000..640be910d969d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 b/sql/hive/src/test/resources/golden/udf_like-2-dbc46cb33f0dd356af03006d9492f8b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc b/sql/hive/src/test/resources/golden/udf_like-3-bef03784eab9d5e8404fd24960dea4fc new file mode 100644 index 0000000000000..b186fa011a671 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 b/sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 new file mode 100644 index 0000000000000..6cff4eb100134 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_like-4-af5fe5d5d176f751747bf14055d00a12 @@ -0,0 +1 @@ +true true false true false diff --git a/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 b/sql/hive/src/test/resources/golden/udf_ln-0-779eed5722a0efaa85efe24c559072b4 new file mode 100644 index 0000000000000..0b7372cb446d9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 b/sql/hive/src/test/resources/golden/udf_ln-1-60e3541b3c703d6413869d774df9b7e4 new file mode 100644 index 0000000000000..bdf2f49ab17ca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e b/sql/hive/src/test/resources/golden/udf_locate-0-6e41693c9c6dceea4d7fab4c02884e4e new file mode 100644 index 0000000000000..63b152162407b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 b/sql/hive/src/test/resources/golden/udf_locate-1-d9b5934457931447874d6bb7c13de478 new file mode 100644 index 0000000000000..ea5465b67683e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c b/sql/hive/src/test/resources/golden/udf_locate-2-849e7058dbbd7d422a9f3eb08d85b15c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 b/sql/hive/src/test/resources/golden/udf_locate-3-2a260e4b8e909eb5e848bf31a07f2531 new file mode 100644 index 0000000000000..cd97bbb17fa81 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a b/sql/hive/src/test/resources/golden/udf_log-0-ca773bc1afa66218f3c13dee676bd87a new file mode 100644 index 0000000000000..e0dba6b06d9a4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 b/sql/hive/src/test/resources/golden/udf_log-1-db9dd44bf8c6225f835819a8cdf20d70 new file mode 100644 index 0000000000000..3dedfb4ddfd88 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f b/sql/hive/src/test/resources/golden/udf_log10-0-35028570b378a2c7ea25b6bf6a4fac1f new file mode 100644 index 0000000000000..61749c48f158e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 b/sql/hive/src/test/resources/golden/udf_log10-1-abf1173290ef905d24d422faf7801fe3 new file mode 100644 index 0000000000000..68ce39dd2b084 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 b/sql/hive/src/test/resources/golden/udf_log2-0-6c9ae9d4deb1b42500ad2796a99e2bc6 new file mode 100644 index 0000000000000..177008a543627 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 b/sql/hive/src/test/resources/golden/udf_log2-1-a79f0dce2cfc000b11a3b5299f02db56 new file mode 100644 index 0000000000000..4b856f91a7a14 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 b/sql/hive/src/test/resources/golden/udf_logic_java_boolean-0-2e7b9484514a049bbf72a4a0af5ee127 new file mode 100644 index 0000000000000..d9d0dc035c9ab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 b/sql/hive/src/test/resources/golden/udf_lower-0-257a0065c0e0df1d0b35a0c6eb30a668 new file mode 100644 index 0000000000000..9bfc44922ea38 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c b/sql/hive/src/test/resources/golden/udf_lower-1-550f0a6da388596a775d921b9da995c new file mode 100644 index 0000000000000..da912776ab009 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e b/sql/hive/src/test/resources/golden/udf_lower-2-467230d9c8442c726e5377b70257ff5e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 b/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 new file mode 100644 index 0000000000000..9640d1794dcbc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lower-3-61b2e3e72180c80d52cf9bed18125e08 @@ -0,0 +1 @@ +abc 123 ABC 123 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef b/sql/hive/src/test/resources/golden/udf_lpad-0-995646acf1e23cea7825412915921bef new file mode 100644 index 0000000000000..672d2d07f68b5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 b/sql/hive/src/test/resources/golden/udf_lpad-1-f58bb0fd11cb70cf197c01555ac924a8 new file mode 100644 index 0000000000000..66db6624a376a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 b/sql/hive/src/test/resources/golden/udf_lpad-2-e779f6358f58919e49fcd6e4b4d007b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae b/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae new file mode 100644 index 0000000000000..63a7235050df3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_lpad-3-5b04264ae9ada1304acd0410ce31f2ae @@ -0,0 +1 @@ +h ...hi 1231hi \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f b/sql/hive/src/test/resources/golden/udf_ltrim-0-398a623504c47bcd64fe8d200c41402f new file mode 100644 index 0000000000000..0c9f6fd1c269e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 b/sql/hive/src/test/resources/golden/udf_ltrim-1-658d495908097792a0e33a77becac2 new file mode 100644 index 0000000000000..3ee2e690d28c1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e b/sql/hive/src/test/resources/golden/udf_map-0-9feb9f29115f94b3bb4f6a36160bd17e new file mode 100644 index 0000000000000..2f31644f4c1cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf b/sql/hive/src/test/resources/golden/udf_map-1-1f8cd98df9bf7b2528506551fef87dcf new file mode 100644 index 0000000000000..2f31644f4c1cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 b/sql/hive/src/test/resources/golden/udf_map-2-a3f90085abab46205e732b4c27b18340 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 b/sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 new file mode 100644 index 0000000000000..9f6becdf7d60f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_map-3-75348d588d3452e6cb35f5197f4ebeb1 @@ -0,0 +1 @@ +{} {1:"a",2:"b",3:"c"} {"1":"2","a":"b"} b b a diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_map_keys-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa b/sql/hive/src/test/resources/golden/udf_map_keys-1-9a5714f8790219e9a9708a2c45fc87aa new file mode 100644 index 0000000000000..b88b74b978b8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f b/sql/hive/src/test/resources/golden/udf_map_keys-2-731b529a9a234473312b7d1db15be75f new file mode 100644 index 0000000000000..b88b74b978b8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e b/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e new file mode 100644 index 0000000000000..3a26a2e5e94d5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_map_keys-3-a3d6a9c2dd5db33c58513ffba1a4103e @@ -0,0 +1 @@ +[1,2,3] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a b/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a new file mode 100644 index 0000000000000..9b4c194f58a8e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_map_keys-4-10cb8da631c1c058dacbbe4834a5928a @@ -0,0 +1 @@ +["b","a","c"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_map_values-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 b/sql/hive/src/test/resources/golden/udf_map_values-1-a1d9810d9793d4ca2d17f539ca72bd08 new file mode 100644 index 0000000000000..e811d1976a24c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 b/sql/hive/src/test/resources/golden/udf_map_values-2-ed39a40cbe55bb33d2bc19f0941dae69 new file mode 100644 index 0000000000000..e811d1976a24c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 b/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 new file mode 100644 index 0000000000000..9d804e48987c6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_map_values-3-ea2d6bed07d285bc8a7d7cab4cbf69d1 @@ -0,0 +1 @@ +["a","b","c"] diff --git a/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e b/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e new file mode 100644 index 0000000000000..be56bd1673eaa --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_map_values-4-a000d06dd3941756b4bb9ccc46f3620e @@ -0,0 +1 @@ +[2,1,3] diff --git a/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f b/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f new file mode 100644 index 0000000000000..c6cd9573ed8ed --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-0-ac7d002a46f773ab680ed8c1ac97821f @@ -0,0 +1 @@ +max(expr) - Returns the maximum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 b/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 new file mode 100644 index 0000000000000..c6cd9573ed8ed --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-1-14afa1f14687893233a662f0f32a40c9 @@ -0,0 +1 @@ +max(expr) - Returns the maximum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 new file mode 100644 index 0000000000000..21096f0f69749 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-10-1e9209f5b2ba926df36b692a4dcd09f6 @@ -0,0 +1 @@ +{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-11-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-12-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 new file mode 100644 index 0000000000000..21096f0f69749 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-13-1e9209f5b2ba926df36b692a4dcd09f6 @@ -0,0 +1 @@ +{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-2-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-3-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 new file mode 100644 index 0000000000000..21096f0f69749 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-4-1e9209f5b2ba926df36b692a4dcd09f6 @@ -0,0 +1 @@ +{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-5-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-6-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 b/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 new file mode 100644 index 0000000000000..21096f0f69749 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-7-1e9209f5b2ba926df36b692a4dcd09f6 @@ -0,0 +1 @@ +{"col1":498,"col2":"val_498"} {"col1":498,"col2":"val_498"} diff --git a/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-8-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_max-9-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 b/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 new file mode 100644 index 0000000000000..0d59cabcde8fb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-0-1a67398a47b4232c3786485b451d1ff8 @@ -0,0 +1 @@ +min(expr) - Returns the minimum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 b/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 new file mode 100644 index 0000000000000..0d59cabcde8fb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-1-69d749d0bca0ebe56e930009e30f4f19 @@ -0,0 +1 @@ +min(expr) - Returns the minimum value of expr diff --git a/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 new file mode 100644 index 0000000000000..b26a7e93c5d9f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-10-191613d4d46d1884d0694fcd8c5fb802 @@ -0,0 +1 @@ +{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-11-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-12-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 new file mode 100644 index 0000000000000..b26a7e93c5d9f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-13-191613d4d46d1884d0694fcd8c5fb802 @@ -0,0 +1 @@ +{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-2-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-3-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 new file mode 100644 index 0000000000000..b26a7e93c5d9f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-4-191613d4d46d1884d0694fcd8c5fb802 @@ -0,0 +1 @@ +{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-5-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-6-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 b/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 new file mode 100644 index 0000000000000..b26a7e93c5d9f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-7-191613d4d46d1884d0694fcd8c5fb802 @@ -0,0 +1 @@ +{"col1":0,"col2":"val_0"} {"col1":0,"col2":"val_0"} diff --git a/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-8-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 b/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_min-9-a00d1791b7fa7ac5a0505d95c3d12257 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee b/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee new file mode 100644 index 0000000000000..b6bd6049a90b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_minute-0-9a38997c1f41f4afe00faa0abc471aee @@ -0,0 +1 @@ +minute(date) - Returns the minute of date \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 b/sql/hive/src/test/resources/golden/udf_minute-1-16995573ac4f4a1b047ad6ee88699e48 new file mode 100644 index 0000000000000..64c3cda697e1e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 b/sql/hive/src/test/resources/golden/udf_minute-2-2bf0b45e6608d66c7d7a5b320f662d75 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c b/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c new file mode 100644 index 0000000000000..5bd5433375d5c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_minute-3-270055c684846e87444b037226cf554c @@ -0,0 +1 @@ +14 14 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 b/sql/hive/src/test/resources/golden/udf_modulo-0-4e06551d4aa9464492e0f53374a280d5 new file mode 100644 index 0000000000000..1bcf0c71008b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 b/sql/hive/src/test/resources/golden/udf_modulo-1-cc0579c83ca1b36fa8a3a1622d19e877 new file mode 100644 index 0000000000000..1bcf0c71008b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee b/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee new file mode 100644 index 0000000000000..b6bd6049a90b9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_month-0-9a38997c1f41f4afe00faa0abc471aee @@ -0,0 +1 @@ +minute(date) - Returns the minute of date \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 b/sql/hive/src/test/resources/golden/udf_month-1-16995573ac4f4a1b047ad6ee88699e48 new file mode 100644 index 0000000000000..64c3cda697e1e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 b/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 new file mode 100644 index 0000000000000..3142f7f94be9e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-0-1b770ec6fb07bb771af2231a9723ec8 @@ -0,0 +1 @@ +negative a - Returns -a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf b/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf new file mode 100644 index 0000000000000..3142f7f94be9e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-1-5f64266721b1ed31cfe84ee2f2377bdf @@ -0,0 +1 @@ +negative a - Returns -a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c b/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c new file mode 100644 index 0000000000000..0f4a4dce9dd3d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-2-a6863d2c5fc8c3131fe70080a011392c @@ -0,0 +1 @@ +a - b - Returns the difference a-b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 new file mode 100644 index 0000000000000..0f4a4dce9dd3d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-3-b90eec030fee9cbd177f9615b782d722 @@ -0,0 +1 @@ +a - b - Returns the difference a-b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c b/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-4-e27bf3f44ccb2e051877da8a4455f50c @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e b/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-5-93d7dd808d4af59bda601faf249a9e @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 b/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-6-6758b00c5acc7aac320238accf299219 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b b/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-7-6d8783f0ed7a4b7058c95f90da3fb4b @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 b/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-8-634af0478ed9ed44b851cd7ef834a489 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa b/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_negative-9-80b4c1fe00f7997838bba64a2cb5f8aa @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc b/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc new file mode 100644 index 0000000000000..1ad92784d47ac --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_not-0-9ddee9ccebe4acbf903da1f42fe55dbc @@ -0,0 +1 @@ +not a - Logical not \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 b/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 new file mode 100644 index 0000000000000..6520d74f02566 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_not-1-efefc8302b02224d20f4bb0f159a6911 @@ -0,0 +1,2 @@ +not a - Logical not +Synonyms: ! \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 b/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 new file mode 100644 index 0000000000000..c0e84e95d01c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_not-2-7e63750d3027ced0e3452ad4eb0df117 @@ -0,0 +1 @@ +! a - Logical not \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a b/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a new file mode 100644 index 0000000000000..05cb1f6ab03d7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_not-3-aa0c674f9ce0feba86448448a211bd2a @@ -0,0 +1,2 @@ +! a - Logical not +Synonyms: not \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e b/sql/hive/src/test/resources/golden/udf_notequal-0-27c0d57f7c7c48ceb7bc671f7160254e new file mode 100644 index 0000000000000..9e2108ff72b18 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a b/sql/hive/src/test/resources/golden/udf_notequal-1-a7f0d1064f8f398ef504008015fddf9a new file mode 100644 index 0000000000000..162f808ed5b67 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 b/sql/hive/src/test/resources/golden/udf_notequal-2-7d815b1218c85e4cf69d6780cab17520 new file mode 100644 index 0000000000000..2dab01a3b2765 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 b/sql/hive/src/test/resources/golden/udf_notequal-3-b72baeb22fad61bb31ce2d2e69375f57 new file mode 100644 index 0000000000000..1f73486ba2896 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 b/sql/hive/src/test/resources/golden/udf_notequal-4-eb04e5ee00967799c913e8a5b424a332 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 b/sql/hive/src/test/resources/golden/udf_notequal-5-e361b9cf294c4aa25018b081a2c05e07 new file mode 100644 index 0000000000000..f82286d8bb069 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 b/sql/hive/src/test/resources/golden/udf_notequal-6-46a6514f2d7e6a097035ec1559df0096 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd b/sql/hive/src/test/resources/golden/udf_notequal-7-a71fea4e9514cda0da9542a7701613dd new file mode 100644 index 0000000000000..f82286d8bb069 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 b/sql/hive/src/test/resources/golden/udf_notop-0-825431072651228a5a9de7f85a0498d6 new file mode 100644 index 0000000000000..a55e3339049e8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 b/sql/hive/src/test/resources/golden/udf_nvl-0-df7383141df0bb71ccb57f3eef9775b4 new file mode 100644 index 0000000000000..5ffcb851d56f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 b/sql/hive/src/test/resources/golden/udf_nvl-1-6ec6c4b23c742fc604c9937a25b0b092 new file mode 100644 index 0000000000000..5afff220da695 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def b/sql/hive/src/test/resources/golden/udf_nvl-2-175ed7006e8907b65e0e5357f00a0def new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 b/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 new file mode 100644 index 0000000000000..2087e17494459 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_nvl-3-7dd762d0da31b9bb0edbabaad1a4dce0 @@ -0,0 +1 @@ +1 5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 b/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 new file mode 100644 index 0000000000000..9138f44ad2a41 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_or-0-c404aa929eb0dd87269121f8f99ada70 @@ -0,0 +1 @@ +a or b - Logical or \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 b/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 new file mode 100644 index 0000000000000..9138f44ad2a41 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_or-1-e21a84ffd26beabb958518ca5e46d6e0 @@ -0,0 +1 @@ +a or b - Logical or \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 b/sql/hive/src/test/resources/golden/udf_parse_url-0-7571c0423df7bf158ea9ca98142b26b8 new file mode 100644 index 0000000000000..0535b085e50c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab b/sql/hive/src/test/resources/golden/udf_parse_url-1-67adfb10d4a35c4d031f26adde9f61ab new file mode 100644 index 0000000000000..7178e07a07e48 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d b/sql/hive/src/test/resources/golden/udf_parse_url-2-ed7adf45f58ad014cf9f3d653b380a4d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 b/sql/hive/src/test/resources/golden/udf_parse_url-3-3a43b1f94ffb8082419bd7cc0b371ce2 new file mode 100644 index 0000000000000..f5483d4e3dafe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 b/sql/hive/src/test/resources/golden/udf_percentile-0-8f99f54ff944f252e47d0af1f4ed1553 new file mode 100644 index 0000000000000..2025042f5d493 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a b/sql/hive/src/test/resources/golden/udf_percentile-1-c0825a744cd14917d2c904d014449a4a new file mode 100644 index 0000000000000..2025042f5d493 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 b/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_percentile-2-1d351f7e821fcaf66c6f7503e42fb291 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e b/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_percentile-3-a7dc16cb82c595b18d4258a38a304b1e @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 b/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 new file mode 100644 index 0000000000000..5d2fc352ee060 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-0-ed67184beaf84c0542117c26651938e1 @@ -0,0 +1 @@ +a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae b/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae new file mode 100644 index 0000000000000..5d2fc352ee060 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-1-90f75e01dcee85253a501d53b8562dae @@ -0,0 +1 @@ +a pmod b - Compute the positive modulo diff --git a/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 b/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 new file mode 100644 index 0000000000000..0b46af11c4516 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-10-b2c7b3ae343b0a21037fe089c1348bf2 @@ -0,0 +1 @@ +6.89 51.7 18.09 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 b/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-2-8ac9813b27801704082c6e9ea4cdc312 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 b/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 new file mode 100644 index 0000000000000..5eb0813b60eb6 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-3-26d9546f030281a29a50a3e8e5858234 @@ -0,0 +1 @@ +8 51 15 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 b/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 new file mode 100644 index 0000000000000..e21e4b08e7a62 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-4-7695df16d24a821224676e6bad3d66d1 @@ -0,0 +1 @@ +5 50 0 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 b/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-5-cf5311d51d44afb8d73f588e27d5e029 @@ -0,0 +1 @@ +8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 b/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-6-3c09a8da2f5645e732c22a45d055125 @@ -0,0 +1 @@ +8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 b/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 new file mode 100644 index 0000000000000..e0bc2a844fb46 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-7-a5fcbb9c74f9ee98e65b74197b10f618 @@ -0,0 +1 @@ +8 51 16 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd b/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd new file mode 100644 index 0000000000000..48371142e9b5d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-8-f49d1f1fab1d9bc19be787efbe6036dd @@ -0,0 +1 @@ +6.8899984 51.700005 18.089996 diff --git a/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 b/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 new file mode 100644 index 0000000000000..ab842acd48b3c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_pmod-9-e7280393102077442aa1d10eb69a6d57 @@ -0,0 +1 @@ +6.890000000000011 51.699999999999996 18.090000000000003 diff --git a/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 b/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 new file mode 100644 index 0000000000000..7c98729cc195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_positive-0-50ee5b92ad14e9f49d7ce6df7847c9b2 @@ -0,0 +1 @@ +positive a - Returns a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c b/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c new file mode 100644 index 0000000000000..7c98729cc195c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_positive-1-1b98434a841d2248ed985c5f6ba2cc3c @@ -0,0 +1 @@ +positive a - Returns a \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 b/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 new file mode 100644 index 0000000000000..e8f11c444a808 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_positive-2-610d421e590f035c24e29694a68b0d23 @@ -0,0 +1 @@ +a + b - Returns a+b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 b/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 new file mode 100644 index 0000000000000..e8f11c444a808 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_positive-3-400b238f4e6cdf7120be566b0ef079c5 @@ -0,0 +1 @@ +a + b - Returns a+b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 b/sql/hive/src/test/resources/golden/udf_pow-0-c7f5178951dd45dc2a41c16729314d81 new file mode 100644 index 0000000000000..43f197e360c82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 b/sql/hive/src/test/resources/golden/udf_pow-1-3c22c000c35144135aedbc7052f10803 new file mode 100644 index 0000000000000..afc2e4462a1a8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 b/sql/hive/src/test/resources/golden/udf_power-0-57001d802c281743322d28bbc520cd4 new file mode 100644 index 0000000000000..5e3a6a8f31fd3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 b/sql/hive/src/test/resources/golden/udf_power-1-ebd0398b2cb03f382a16382ddac13426 new file mode 100644 index 0000000000000..c3414e29768e3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 b/sql/hive/src/test/resources/golden/udf_radians-0-f899daf93b02ca681e0230a792c65e86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 b/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 new file mode 100644 index 0000000000000..116b6bc461ed0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-1-eaaa62dd3935ff3152323dfafd136e93 @@ -0,0 +1 @@ +1.000000357564167 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 b/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 new file mode 100644 index 0000000000000..aaf3b31fef488 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-2-bcaca433f704f71cf9d44c238a33c7b3 @@ -0,0 +1 @@ +2.4999991485811655 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 new file mode 100644 index 0000000000000..73ad88be4ef3d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-3-65e16c7b13de48a5d36793d0c7d35e14 @@ -0,0 +1 @@ +radians(x) - Converts degrees to radians \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-4-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 new file mode 100644 index 0000000000000..e0237c0058f55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 b/sql/hive/src/test/resources/golden/udf_radians-5-f899daf93b02ca681e0230a792c65e86 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 b/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 new file mode 100644 index 0000000000000..116b6bc461ed0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-6-eaaa62dd3935ff3152323dfafd136e93 @@ -0,0 +1 @@ +1.000000357564167 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 b/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 new file mode 100644 index 0000000000000..aaf3b31fef488 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-7-bcaca433f704f71cf9d44c238a33c7b3 @@ -0,0 +1 @@ +2.4999991485811655 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 b/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 new file mode 100644 index 0000000000000..73ad88be4ef3d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_radians-8-65e16c7b13de48a5d36793d0c7d35e14 @@ -0,0 +1 @@ +radians(x) - Converts degrees to radians \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 b/sql/hive/src/test/resources/golden/udf_radians-9-9f1b8ddf9be2e5f9b9b7ff4f00cfb66 new file mode 100644 index 0000000000000..e0237c0058f55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 b/sql/hive/src/test/resources/golden/udf_rand-0-f6d991f4e0dfea517dfc3dcaf1ff6af2 new file mode 100644 index 0000000000000..d19c5b50fe5c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e b/sql/hive/src/test/resources/golden/udf_rand-1-c6229b8f2ca3001663229cfb8ee4763e new file mode 100644 index 0000000000000..d19c5b50fe5c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 b/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 new file mode 100644 index 0000000000000..1d7658151cd62 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_reflect-0-904138e2a1f831c308b7f0aacc859ae1 @@ -0,0 +1 @@ +reflect(class,method[,arg1[,arg2..]]) calls method with reflection diff --git a/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee b/sql/hive/src/test/resources/golden/udf_reflect-1-21ec7caa253c7f95b7cf60191140e2ee new file mode 100644 index 0000000000000..ddf986c01e5b1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 b/sql/hive/src/test/resources/golden/udf_reflect-2-b868357466bab2f04685c2dc73604cf0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 b/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 new file mode 100644 index 0000000000000..cd35e5b290db5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_reflect2-0-7bec330c7bc6f71cbaf9bf1883d1b184 @@ -0,0 +1 @@ +reflect2(arg0,method[,arg1[,arg2..]]) calls method of arg0 with reflection diff --git a/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 b/sql/hive/src/test/resources/golden/udf_reflect2-1-c5a05379f482215a5a484bed0299bf19 new file mode 100644 index 0000000000000..48ef97292ab62 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca b/sql/hive/src/test/resources/golden/udf_reflect2-2-effc057c78c00b0af26a4ac0f5f116ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 b/sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 new file mode 100644 index 0000000000000..3e2abd349cef7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_regexp-0-19917611f74aedc0922560f7f2595948 @@ -0,0 +1 @@ +str regexp regexp - Returns true if str matches regexp and false otherwise diff --git a/sql/hive/src/test/resources/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 b/sql/hive/src/test/resources/golden/udf_regexp-1-f7f0527cd47612d7f256edd5f8963800 new file mode 100644 index 0000000000000..d6b5d52cd1be5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 b/sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 new file mode 100644 index 0000000000000..3c4f7ed353b35 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_regexp-2-5e1798db3ba058e7b202d8a98f228b11 @@ -0,0 +1 @@ +true true true true false false false false diff --git a/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 b/sql/hive/src/test/resources/golden/udf_regexp_extract-0-e251e1a4b1e191814f26c54b14ab6cd9 new file mode 100644 index 0000000000000..429057caf71f0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb b/sql/hive/src/test/resources/golden/udf_regexp_extract-1-8add879ab5904bd805412ef8723276fb new file mode 100644 index 0000000000000..30e237490811f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e b/sql/hive/src/test/resources/golden/udf_regexp_replace-0-3ef9cc6da51dd1f5c6b71cf8a464ca0e new file mode 100644 index 0000000000000..1a38701d68875 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc b/sql/hive/src/test/resources/golden/udf_regexp_replace-1-e79b45aa220d3c4c3b4523ac9c897bc new file mode 100644 index 0000000000000..bebfabaf637fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b b/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b new file mode 100644 index 0000000000000..694c367436f3c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_repeat-0-fdeae3e88f27ef148281d69ac8c4b23b @@ -0,0 +1 @@ +repeat(str, n) - repeat str n times \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b b/sql/hive/src/test/resources/golden/udf_repeat-1-836be47190989d8975a09a545ecbfe0b new file mode 100644 index 0000000000000..5df19ba1c2cbf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a b/sql/hive/src/test/resources/golden/udf_repeat-2-e1dbea7182ec1653e1123b5b67a6d20a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 b/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 new file mode 100644 index 0000000000000..45425cf087c09 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_repeat-3-ba9dd02f59c74d63d60d60b6231a0365 @@ -0,0 +1 @@ +FacebookFacebookFacebook \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 b/sql/hive/src/test/resources/golden/udf_rlike-0-6ec6ef55ac041208627454e16b501d38 new file mode 100644 index 0000000000000..fd9ac3081a1cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca b/sql/hive/src/test/resources/golden/udf_rlike-1-829611a596e0c87431006f7247d25eca new file mode 100644 index 0000000000000..43b06945caa54 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 b/sql/hive/src/test/resources/golden/udf_round-0-10b53ca1f15fd7879365926f86512d15 new file mode 100644 index 0000000000000..e4586b2e73a93 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 b/sql/hive/src/test/resources/golden/udf_round-1-2367bcc43510dedc80bdb6707e434da8 new file mode 100644 index 0000000000000..c0d5b480e9751 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 b/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 new file mode 100644 index 0000000000000..0924e3869076d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-2-9ffa2b573360cd879338de46d91ab374 @@ -0,0 +1 @@ +NULL NULL NULL Infinity NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c b/sql/hive/src/test/resources/golden/udf_round-3-42a221909d3f7ed51bed01a65670461c new file mode 100644 index 0000000000000..3b9c30929a240 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 b/sql/hive/src/test/resources/golden/udf_round-4-dc80ec5189a4c6ce07688df3debe7de4 new file mode 100644 index 0000000000000..1f243f6cbc6db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a b/sql/hive/src/test/resources/golden/udf_round-5-a2414e7845ffafc61f75465508a1850a new file mode 100644 index 0000000000000..918404cec8047 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b b/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b new file mode 100644 index 0000000000000..af105563af144 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round-6-48439efa5c34e7589ab5003ed916f12b @@ -0,0 +1 @@ +1809242.315111134 -1809242.315111134 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 b/sql/hive/src/test/resources/golden/udf_round_2-0-91afaf77ef4061fa20005a3c87dfef32 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce b/sql/hive/src/test/resources/golden/udf_round_2-1-5e44354af73865d03e9088c0232f16ce new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 b/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_2-2-83f91f60dcb2036b61b8b21f18281298 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 b/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_2-3-c62cf7b74a91f605cf669e34a9315f93 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 b/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_2-4-797fa8ed05cb953327d0f6306b30d6c3 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b b/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b new file mode 100644 index 0000000000000..f40e633f703c9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_2-5-e41b862db8cc76c1fe004cf006fad50b @@ -0,0 +1 @@ +Infinity Infinity Infinity Infinity \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c b/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c new file mode 100644 index 0000000000000..3714de0db18dc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-0-8415af605db167315e4d9d3c69d89e6c @@ -0,0 +1 @@ +-128 127 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 b/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 new file mode 100644 index 0000000000000..a9265d7b8a1b7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-1-3ecc9de58a7ea3af5b2315ca119403d0 @@ -0,0 +1 @@ +-32768 32767 -129 128 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 b/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 new file mode 100644 index 0000000000000..2d25f54073df7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-2-5e7ada966f726ceb748f98c20eab4f10 @@ -0,0 +1 @@ +-2147483648 2147483647 -32769 32768 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 b/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 new file mode 100644 index 0000000000000..41a2624c6cfeb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_round_3-3-e94ab3326df006c7203ead86752f16a9 @@ -0,0 +1 @@ +-9223372036854775808 9223372036854775807 -2147483649 2147483648 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 b/sql/hive/src/test/resources/golden/udf_round_3-4-c3227c37fbbddd79a423bd6abe67a6d7 new file mode 100644 index 0000000000000..98d3f53dfc442 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c b/sql/hive/src/test/resources/golden/udf_rpad-0-56de896c5fe8b40b22a9ed55ed79889c new file mode 100644 index 0000000000000..7cb2d71d4b80d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 b/sql/hive/src/test/resources/golden/udf_rpad-1-48d17e5d5d6188300d048f987fab2ca0 new file mode 100644 index 0000000000000..2b198b8dc96c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 b/sql/hive/src/test/resources/golden/udf_rpad-2-48835c7f49d78f8a6da2a90e11514b3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 b/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 new file mode 100644 index 0000000000000..b2645e365c8a9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_rpad-3-dcad885650ea575ab1ac5cfeb76cf871 @@ -0,0 +1 @@ +h hi... hi1231 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a b/sql/hive/src/test/resources/golden/udf_rtrim-0-7acca21c725995febdf2a9c1fdf0535a new file mode 100644 index 0000000000000..94d6aeed8f533 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd b/sql/hive/src/test/resources/golden/udf_rtrim-1-66d61255134c09d37cbfedd757ae47fd new file mode 100644 index 0000000000000..957e608a7c732 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 b/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 new file mode 100644 index 0000000000000..577c90254cb5a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_second-0-e004a6f20da3fa6db868ec847b217ff2 @@ -0,0 +1 @@ +second(date) - Returns the second of date \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 b/sql/hive/src/test/resources/golden/udf_second-1-3525f55f4f13253c42b3abaa53d77888 new file mode 100644 index 0000000000000..4b48294e5b9ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f b/sql/hive/src/test/resources/golden/udf_second-2-d678372e3837a16be245d2e33482f17f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf b/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf new file mode 100644 index 0000000000000..4d5ef5cf4a699 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_second-3-2496e4d3c64ca028184431c2930d82cf @@ -0,0 +1 @@ +15 15 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f b/sql/hive/src/test/resources/golden/udf_sentences-0-ec0dad44fa033691a731f6e4c6b5cf7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 b/sql/hive/src/test/resources/golden/udf_sentences-1-2dc07f4f0f0cb20d08c424e067ed8f69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 b/sql/hive/src/test/resources/golden/udf_sentences-10-ce188a53f69129c14cbf378d2c3f6630 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 b/sql/hive/src/test/resources/golden/udf_sentences-11-3c8672506e23434187caf4e0064a8a80 new file mode 100644 index 0000000000000..fea4f860c8465 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 b/sql/hive/src/test/resources/golden/udf_sentences-12-d55c04a079ca97402509868f24921685 new file mode 100644 index 0000000000000..c49ca6bb5a1b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 b/sql/hive/src/test/resources/golden/udf_sentences-2-1f218343f90e698fb9ed81c4f51d3d14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 b/sql/hive/src/test/resources/golden/udf_sentences-3-ce188a53f69129c14cbf378d2c3f6630 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 b/sql/hive/src/test/resources/golden/udf_sentences-4-3c8672506e23434187caf4e0064a8a80 new file mode 100644 index 0000000000000..b798628e56686 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 b/sql/hive/src/test/resources/golden/udf_sentences-5-60823f4912be26bee1a0b52a0a9588a9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de b/sql/hive/src/test/resources/golden/udf_sentences-6-27b7eeae683a87507f35e61fd4ce67de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f b/sql/hive/src/test/resources/golden/udf_sentences-7-ec0dad44fa033691a731f6e4c6b5cf7f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 b/sql/hive/src/test/resources/golden/udf_sentences-8-2dc07f4f0f0cb20d08c424e067ed8f69 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 b/sql/hive/src/test/resources/golden/udf_sentences-9-68c61b4882802e416d5adaa2de440b59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e b/sql/hive/src/test/resources/golden/udf_sign-0-14f3c3034ac8039fc81681003bbf5e0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 b/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-1-e6cbbd93f7d4fa596ecb7ef44d12c159 @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa new file mode 100644 index 0000000000000..3345fbb8f2c35 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-10-fc7341f89c3cd3c43e436242d8aa61fa @@ -0,0 +1 @@ +sign(x) - returns the sign of x ) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-11-74237f5ecc497813cf9738b21647847a new file mode 100644 index 0000000000000..bdacec8810e77 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef b/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef new file mode 100644 index 0000000000000..31a5b0b81dc51 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-2-85b743f0eed93904789cde4d1b5eafef @@ -0,0 +1 @@ +-1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d b/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d new file mode 100644 index 0000000000000..9f8e9b69a33f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-3-9eeb3985359429abba5d1dd702c66b0d @@ -0,0 +1 @@ +1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa b/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa new file mode 100644 index 0000000000000..3345fbb8f2c35 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-4-fc7341f89c3cd3c43e436242d8aa61fa @@ -0,0 +1 @@ +sign(x) - returns the sign of x ) \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a b/sql/hive/src/test/resources/golden/udf_sign-5-74237f5ecc497813cf9738b21647847a new file mode 100644 index 0000000000000..bdacec8810e77 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e b/sql/hive/src/test/resources/golden/udf_sign-6-14f3c3034ac8039fc81681003bbf5e0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 b/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-7-e6cbbd93f7d4fa596ecb7ef44d12c159 @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef b/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef new file mode 100644 index 0000000000000..31a5b0b81dc51 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-8-85b743f0eed93904789cde4d1b5eafef @@ -0,0 +1 @@ +-1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d b/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d new file mode 100644 index 0000000000000..9f8e9b69a33f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sign-9-9eeb3985359429abba5d1dd702c66b0d @@ -0,0 +1 @@ +1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee b/sql/hive/src/test/resources/golden/udf_sin-0-40b50393869eb0bcde66e36fe41078ee new file mode 100644 index 0000000000000..6155d9a2eae55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 b/sql/hive/src/test/resources/golden/udf_sin-1-2f867f432fb322e21dce353d7eb50c63 new file mode 100644 index 0000000000000..a39ed8840b916 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab b/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sin-2-3f8e9d5a3e69f39a99090840be2b5cab @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 b/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 new file mode 100644 index 0000000000000..ee21925b0ccc5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sin-3-e3b2b18e530eb504ea5017ca348f12e3 @@ -0,0 +1 @@ +0.8304973704919705 0.9999996829318346 -0.479425538604203 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 b/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 new file mode 100644 index 0000000000000..d8952629349a9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_size-0-9c49a52514b1b940a7e8bb93c35eda62 @@ -0,0 +1 @@ +size(a) - Returns the size of a diff --git a/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 b/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 new file mode 100644 index 0000000000000..d8952629349a9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_size-1-3608160636eaa7e053171bdcefc0b1a8 @@ -0,0 +1 @@ +size(a) - Returns the size of a diff --git a/sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 b/sql/hive/src/test/resources/golden/udf_size-2-96d41c9f054662827d1b6b63f5dd8db7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd b/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd new file mode 100644 index 0000000000000..08a708b090282 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_size-3-63df892215b6ce189d6c3e20cfc4bdbd @@ -0,0 +1 @@ +3 1 1 -1 diff --git a/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 b/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 new file mode 100644 index 0000000000000..572ecb0462eb7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_smallint-0-f28e857ef74c967303855c21dc60c042 @@ -0,0 +1 @@ +There is no documentation for function 'smallint' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 b/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 new file mode 100644 index 0000000000000..572ecb0462eb7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_smallint-1-37d8db74267ae370d6a076b3057c5ed6 @@ -0,0 +1 @@ +There is no documentation for function 'smallint' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_sort_array-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 b/sql/hive/src/test/resources/golden/udf_sort_array-1-976cd8b6b50a2748bbc768aa5e11cf82 new file mode 100644 index 0000000000000..d514df4191b89 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 b/sql/hive/src/test/resources/golden/udf_sort_array-2-c429ec85a6da60ebd4bc6f0f266e8b93 new file mode 100644 index 0000000000000..43e36513de881 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de b/sql/hive/src/test/resources/golden/udf_sort_array-3-55c4cdaf8438b06675d60848d68f35de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 b/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 new file mode 100644 index 0000000000000..2e9458debfd0f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sort_array-4-3edb0151fae0622cb79cd04156cb4c44 @@ -0,0 +1 @@ +["a","b","c","d","e","f","g"] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e b/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e new file mode 100644 index 0000000000000..94f18d09863a7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sort_array-5-5d7dfaa9e4137938559eb536e28f8d0e @@ -0,0 +1 @@ +["enterprise databases","hadoop distributed file system","hadoop map-reduce"] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 b/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 new file mode 100644 index 0000000000000..e1968ef44a2eb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_sort_array-6-f754ac1296d16e168abb3d0ebcc35bd3 @@ -0,0 +1 @@ +[1,2,3,4,5,6,7,8,9] diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 b/sql/hive/src/test/resources/golden/udf_sort_array-7-a9c52a8684cefc095470f5a93f63d2a8 new file mode 100644 index 0000000000000..b6c12ace4162b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 b/sql/hive/src/test/resources/golden/udf_sort_array-8-d79f0084177230a7a2845c4791c22d25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a b/sql/hive/src/test/resources/golden/udf_sort_array-9-45ef2679e195a269878527d5f264488a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 b/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 new file mode 100644 index 0000000000000..ed5bda787df23 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-0-91e879c9f01d90eab7bf12fcef256010 @@ -0,0 +1 @@ +space(n) - returns n spaces \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 b/sql/hive/src/test/resources/golden/udf_space-1-e4eaf5e96807e122548cb43be9a26754 new file mode 100644 index 0000000000000..5713d4b0464be --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 b/sql/hive/src/test/resources/golden/udf_space-2-a23a06eef37709e8587647a74bbfa7e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 b/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 new file mode 100644 index 0000000000000..85a16b2abe5ef --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-3-59903e27d8188d6209e007ff643d5956 @@ -0,0 +1 @@ +10 0 1 0 0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 b/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 new file mode 100644 index 0000000000000..8dfaf2745f666 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_space-4-7adb35ad867ba3e980d937a0038ac1a5 @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 b/sql/hive/src/test/resources/golden/udf_split-0-7accac7fc71ba74d61c01a69d3978338 new file mode 100644 index 0000000000000..6b183ccfb17b4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 b/sql/hive/src/test/resources/golden/udf_split-1-ebe303dfc3db504deffe5c355efd0fd1 new file mode 100644 index 0000000000000..9e5c522da155b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 b/sql/hive/src/test/resources/golden/udf_split-2-7bba11f8fc359f2d4b863cda11c643f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 b/sql/hive/src/test/resources/golden/udf_split-3-c08ccb3ccd2bdc89c5c3745b63305f23 new file mode 100644 index 0000000000000..9174f1a92557e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 b/sql/hive/src/test/resources/golden/udf_sqrt-0-d5e4a818c2b3255ef0e32876cd8ed240 new file mode 100644 index 0000000000000..eadd6eeb8b149 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 b/sql/hive/src/test/resources/golden/udf_sqrt-1-323cb8b7f5f2c93bdb1b5aed77c08e77 new file mode 100644 index 0000000000000..74df32f221278 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b b/sql/hive/src/test/resources/golden/udf_std-0-e3613484de2b3fa707995720ec3f8a5b new file mode 100644 index 0000000000000..2966c01c1b7b2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 b/sql/hive/src/test/resources/golden/udf_std-1-6759bde0e50a3607b7c3fd5a93cbd027 new file mode 100644 index 0000000000000..388eaa1c7439a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 b/sql/hive/src/test/resources/golden/udf_stddev-0-ad7627185d89a60b83ce19966eddbc92 new file mode 100644 index 0000000000000..74c6e1eacc379 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d b/sql/hive/src/test/resources/golden/udf_stddev-1-18e1d598820013453fad45852e1a303d new file mode 100644 index 0000000000000..1c461b68c1440 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 b/sql/hive/src/test/resources/golden/udf_stddev_pop-0-96788538f1f20eb879a1add4bb5f9d12 new file mode 100644 index 0000000000000..741771707f280 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 b/sql/hive/src/test/resources/golden/udf_stddev_pop-1-6286ef94de26050617bf69c17a3b4a10 new file mode 100644 index 0000000000000..741771707f280 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-0-29e22949ef00f5ece1b5fd6bb1923a4 new file mode 100644 index 0000000000000..c1fb091d2c425 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-1-ac7c0f92fe3b72287df2c7a719280bc4 new file mode 100644 index 0000000000000..c1fb091d2c425 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-2-29e22949ef00f5ece1b5fd6bb1923a4 new file mode 100644 index 0000000000000..c1fb091d2c425 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 b/sql/hive/src/test/resources/golden/udf_stddev_samp-3-ac7c0f92fe3b72287df2c7a719280bc4 new file mode 100644 index 0000000000000..c1fb091d2c425 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 b/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 new file mode 100644 index 0000000000000..c61c4e1135410 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_string-0-17412ad1c1a827411caa7b5e891b6ac3 @@ -0,0 +1 @@ +There is no documentation for function 'string' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d b/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d new file mode 100644 index 0000000000000..c61c4e1135410 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_string-1-53b00551846b7f8bb27874b3a466e68d @@ -0,0 +1 @@ +There is no documentation for function 'string' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd b/sql/hive/src/test/resources/golden/udf_substr-0-20fb50d79b45264548b953e37d837fcd new file mode 100644 index 0000000000000..c4efdaebc8aab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 b/sql/hive/src/test/resources/golden/udf_substr-1-2af34de7fd584c5f1ead85b3891b0920 new file mode 100644 index 0000000000000..44958996269c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 b/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 new file mode 100644 index 0000000000000..9516b839a3956 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_substr-2-d0268ad73ab5d94af15d9e1703b424d1 @@ -0,0 +1 @@ +NULL NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 b/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 new file mode 100644 index 0000000000000..7f6c057b5eebd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_substr-3-fe643fe7e2fd8b65e3ca0dbf6c98a122 @@ -0,0 +1 @@ + \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a b/sql/hive/src/test/resources/golden/udf_substr-4-f7933c0bb28e9a815555edfa3764524a new file mode 100644 index 0000000000000..042ca15005869 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 b/sql/hive/src/test/resources/golden/udf_substr-5-4f29c1273b8010ce2d05882fc44793d8 new file mode 100644 index 0000000000000..85346dc923b4a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 b/sql/hive/src/test/resources/golden/udf_substr-6-ebbb2e30ccf4a2f3a9525325529a9fe6 new file mode 100644 index 0000000000000..9d607f4d84214 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 b/sql/hive/src/test/resources/golden/udf_substr-7-1742c72a1a8a6c6fd3eb2449d4747496 new file mode 100644 index 0000000000000..042ca15005869 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece b/sql/hive/src/test/resources/golden/udf_substring-0-8297700b238f417dea2bd60ba72a6ece new file mode 100644 index 0000000000000..b0ade52e69a9d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 b/sql/hive/src/test/resources/golden/udf_substring-1-e5df65419ecd9e837dadfcdd7f9074f8 new file mode 100644 index 0000000000000..1d61dc4518087 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 b/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 new file mode 100644 index 0000000000000..0f4a4dce9dd3d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_subtract-0-8f69db6aad14e23c9c32bf21bd0b3bf3 @@ -0,0 +1 @@ +a - b - Returns the difference a-b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 b/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 new file mode 100644 index 0000000000000..0f4a4dce9dd3d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_subtract-1-b90eec030fee9cbd177f9615b782d722 @@ -0,0 +1 @@ +a - b - Returns the difference a-b \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 b/sql/hive/src/test/resources/golden/udf_sum-0-d5b30a6a291025b1f9334ce8944dab31 new file mode 100644 index 0000000000000..744935026110c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b b/sql/hive/src/test/resources/golden/udf_sum-1-ddae1a511d3371122ab79918be5b495b new file mode 100644 index 0000000000000..744935026110c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 b/sql/hive/src/test/resources/golden/udf_sum-2-d5b30a6a291025b1f9334ce8944dab31 new file mode 100644 index 0000000000000..744935026110c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b b/sql/hive/src/test/resources/golden/udf_sum-3-ddae1a511d3371122ab79918be5b495b new file mode 100644 index 0000000000000..744935026110c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-0-c21aa640b4edabf6613dd705d029c878 new file mode 100644 index 0000000000000..c91072e69740b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-1-a8c8eaa832aa9a4345b2fb9cd5e1d505 new file mode 100644 index 0000000000000..4cc0d466f00f4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 b/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-2-27a29c68f846824990e9e443ac95da85 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 b/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 new file mode 100644 index 0000000000000..04ec33c11c3b4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-3-77bedd6c76bdc33428d13a894f468a97 @@ -0,0 +1 @@ +1.5574077246549023 -0.29100619138474915 -1.5574077246549023 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 b/sql/hive/src/test/resources/golden/udf_tan-4-c21aa640b4edabf6613dd705d029c878 new file mode 100644 index 0000000000000..c91072e69740b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 b/sql/hive/src/test/resources/golden/udf_tan-5-a8c8eaa832aa9a4345b2fb9cd5e1d505 new file mode 100644 index 0000000000000..4cc0d466f00f4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 b/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 new file mode 100644 index 0000000000000..fe3a0735d98b8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-6-27a29c68f846824990e9e443ac95da85 @@ -0,0 +1 @@ +NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 b/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 new file mode 100644 index 0000000000000..04ec33c11c3b4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tan-7-77bedd6c76bdc33428d13a894f468a97 @@ -0,0 +1 @@ +1.5574077246549023 -0.29100619138474915 -1.5574077246549023 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad b/sql/hive/src/test/resources/golden/udf_testlength-0-b91edca0cfe8a8a8ff3b7a7fbc6e23ad new file mode 100644 index 0000000000000..89617d8af0085 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 b/sql/hive/src/test/resources/golden/udf_testlength2-0-3da4fe901124f2bbf3d02822652c4e55 new file mode 100644 index 0000000000000..6d967660b3d41 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d b/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d new file mode 100644 index 0000000000000..1b13d42908539 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tinyint-0-4a2b3d48ca445776d4a2d8c445f9633d @@ -0,0 +1 @@ +There is no documentation for function 'tinyint' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 b/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 new file mode 100644 index 0000000000000..1b13d42908539 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_tinyint-1-417de1aeb44510aa9746729f9ff3b426 @@ -0,0 +1 @@ +There is no documentation for function 'tinyint' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 b/sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-0-35e75ee310b66710e2e88cf0fecca670 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 b/sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-1-cbead694a25ec357d69fd008776e19c9 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 b/sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-2-7f24ad5f9bdc0afb6bade7c85490c845 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c b/sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-3-266b9601a9154438ab95550fcd36494c @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d b/sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-4-2602c9a6c910ec7fdd439212c648333d @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad b/sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-5-a9fe3bd1d2c99c89f019e92e5e8a7cad @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f b/sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-6-1e29d35d9b02f6800c8939910a1e4c2f @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a b/sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-7-93d8249c035f34bfca2fa041ae97f55a @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 b/sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_byte-0-df3f6ff9fdf525a7c617e4c33d5c81a4 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 b/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_byte-1-86bb7f03311e7ea8bda76f24412bc9f3 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc b/sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc new file mode 100644 index 0000000000000..6cc8a61f8f6ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_byte-2-108a5c01e12e8823f6affc3d1664a1fc @@ -0,0 +1 @@ +-18 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b b/sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b new file mode 100644 index 0000000000000..c75acbe2ff105 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_byte-3-78e5f8f149031d91a1aa3ae3372a8b @@ -0,0 +1 @@ +127 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 b/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 new file mode 100644 index 0000000000000..3a2e3f4984a0e --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_byte-4-a8b51dcda7b83f0fb8fb05d572b17579 @@ -0,0 +1 @@ +-1 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 b/sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_byte-5-2adc360a8c6a9f4eac05f186ced2d67 @@ -0,0 +1 @@ +-3 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 b/sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_byte-6-75067817a1fe70c8c03c18ec7ed2f404 @@ -0,0 +1 @@ +-3 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 b/sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_byte-7-4b9de0160a65b212cbe42059d98aa718 @@ -0,0 +1 @@ +-3 diff --git a/sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 b/sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 new file mode 100644 index 0000000000000..91ff6743335bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_byte-8-7bf107e0fcc94ab00a5481a9f67aec25 @@ -0,0 +1 @@ +-38 diff --git a/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 b/sql/hive/src/test/resources/golden/udf_to_date-0-ab5c4edc1825010642bd24f4cfc26166 new file mode 100644 index 0000000000000..54a99d2a94ef4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e b/sql/hive/src/test/resources/golden/udf_to_date-1-da3c817bc5f4458078c6199390ac915e new file mode 100644 index 0000000000000..81ddc59fcb531 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 b/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-0-cbe030be095a93a9ae810ce7e66bdca7 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a b/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-1-98d2ce732277c3a7fb4827d8b221a43a @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe b/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe new file mode 100644 index 0000000000000..319fde05380bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-2-69bf8a5a4cb378bbd54c20cb8aa97abe @@ -0,0 +1 @@ +-7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f b/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f new file mode 100644 index 0000000000000..8c1c4fe62b6c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-3-ab23099412d24154ff369d8bd6bde89f @@ -0,0 +1 @@ +-18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 b/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 new file mode 100644 index 0000000000000..1b650de78904f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-4-293a639a2b61a11da6ca798c04624f68 @@ -0,0 +1 @@ +-129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 b/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 new file mode 100644 index 0000000000000..3a3bd0df03b5b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-5-42d1e80bb3324030c62a23c6d1b786a8 @@ -0,0 +1 @@ +-1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 b/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 new file mode 100644 index 0000000000000..38f7ad5afa0ab --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-6-5bac1a7db00d788fd7d82e3a78e60be6 @@ -0,0 +1 @@ +-3.140000104904175 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 b/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-7-97080ab9cd416f8acd8803291e9dc9e5 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 b/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 new file mode 100644 index 0000000000000..f45d1f04dc920 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_double-8-df51146f6ef960c77cd1722191e4b982 @@ -0,0 +1 @@ +-38.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 b/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-0-7646eca02448547eedf84a81bf42be89 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c b/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c new file mode 100644 index 0000000000000..d3827e75a5cad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-1-eac0237ee0294d635c2b538f6e2f0a5c @@ -0,0 +1 @@ +1.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 b/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 new file mode 100644 index 0000000000000..319fde05380bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-2-39a67183b6d2a4da005baed849c5e971 @@ -0,0 +1 @@ +-7.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce b/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce new file mode 100644 index 0000000000000..8c1c4fe62b6c2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-3-5c1ae08cfd9ffd4d3e57b7a6ec4e39ce @@ -0,0 +1 @@ +-18.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 b/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 new file mode 100644 index 0000000000000..1b650de78904f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-4-f3e0ab8ed691a386e6be4ce6993be507 @@ -0,0 +1 @@ +-129.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 b/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 new file mode 100644 index 0000000000000..3a3bd0df03b5b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-5-75f364708c01b5e31f988f19e52b2201 @@ -0,0 +1 @@ +-1025.0 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 b/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-6-1d9b690354f7b04df660a9e3c448a002 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 b/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-7-cdfefa5173854b647a76383300f8b9d1 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c b/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c new file mode 100644 index 0000000000000..f45d1f04dc920 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_float-8-5b0a785185bcaa98b581c5b3dbb3e12c @@ -0,0 +1 @@ +-38.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a b/sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_long-0-ebe447e8fb03de8844951250fe50320a @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 b/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_long-1-61e6679e5a37889bc596590bde0228f0 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 b/sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 new file mode 100644 index 0000000000000..17bdab103828b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_long-2-5efd128fd5cbfbf1794c7da16d4b18d5 @@ -0,0 +1 @@ +-7 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 b/sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 new file mode 100644 index 0000000000000..6cc8a61f8f6ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_long-3-cb3318ba365833316645e1b9890f4613 @@ -0,0 +1 @@ +-18 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 b/sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 new file mode 100644 index 0000000000000..9828ff22b667b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_long-4-55b7b7d046541d63e30f76111734c0f6 @@ -0,0 +1 @@ +-129 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d b/sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_long-5-a0d382d243a226f8d4381970b6831c3d @@ -0,0 +1 @@ +-3 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 b/sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_long-6-47ab11eae68329cc80232fc4089479f0 @@ -0,0 +1 @@ +-3 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e b/sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_long-7-b87c0cabb166f33984cc1b191694918e @@ -0,0 +1 @@ +-3 diff --git a/sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 b/sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 new file mode 100644 index 0000000000000..91ff6743335bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_long-8-52a24d7040db321a842f9201d245ee9 @@ -0,0 +1 @@ +-38 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a b/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_short-0-8a246f07cfa1a8e96a2bf0600f8bd95a @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b b/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_short-1-32c4a61f9166c2a9d7038553d589bd9b @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 b/sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 new file mode 100644 index 0000000000000..6cc8a61f8f6ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_short-2-a4ab375c99efbfe0d5a45941552509c8 @@ -0,0 +1 @@ +-18 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce b/sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce new file mode 100644 index 0000000000000..9828ff22b667b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_short-3-8d5108d9585d8ca6ca3eefac05afc3ce @@ -0,0 +1 @@ +-129 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 b/sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 new file mode 100644 index 0000000000000..450a6125550e5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_short-4-8d76a5a410e5533f2ae625504478cda8 @@ -0,0 +1 @@ +-1025 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 b/sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_short-5-d467dc39319796ff5cbe600ff6530ad5 @@ -0,0 +1 @@ +-3 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 b/sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_short-6-c3b304122e874868f37c6fbaeccf0397 @@ -0,0 +1 @@ +-3 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d b/sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d new file mode 100644 index 0000000000000..a83d1d524c8bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_short-7-827f3181c216fd2e990637c9a091bf0d @@ -0,0 +1 @@ +-3 diff --git a/sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 b/sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 new file mode 100644 index 0000000000000..91ff6743335bd --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_short-8-d35f9502fc7b4205e18ecdf53189bdc4 @@ -0,0 +1 @@ +-38 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc b/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-0-a032eb8f819689a374852c20336d5cc @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 b/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 new file mode 100644 index 0000000000000..ef2f5130b8575 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-1-b461f0e6e98991aede40c7c68561dc44 @@ -0,0 +1 @@ +TRUE diff --git a/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 b/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 new file mode 100644 index 0000000000000..d00491fd7e5bb --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-2-bc6b37e1d5a8d28df8a957c88104c9a5 @@ -0,0 +1 @@ +1 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 b/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 new file mode 100644 index 0000000000000..6cc8a61f8f6ad --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-3-ddbb829eedc6aa1e27169303a7957d15 @@ -0,0 +1 @@ +-18 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 b/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 new file mode 100644 index 0000000000000..9828ff22b667b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-4-c20301e9bbf10143bb9bf67cd7367c21 @@ -0,0 +1 @@ +-129 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 b/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 new file mode 100644 index 0000000000000..450a6125550e5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-5-1040b37847d20ef29d545934316303 @@ -0,0 +1 @@ +-1025 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 b/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-6-4181d264a7af3c6488da2f1db20dc384 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 b/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-7-567bc77f96e7dc8c89bae912e9a3af15 @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea b/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea new file mode 100644 index 0000000000000..01e913dbfe725 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-8-a70b03d79ebd989c8141f9d70dbca8ea @@ -0,0 +1 @@ +-3.14 diff --git a/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b b/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b new file mode 100644 index 0000000000000..bc56c4d89448a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_string-9-51677fbf5d2fc5478853cec1df039e3b @@ -0,0 +1 @@ +Foo diff --git a/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 b/sql/hive/src/test/resources/golden/udf_translate-0-7fe940a2b26fa19a3cfee39e56fb1241 new file mode 100644 index 0000000000000..4255dc76f501e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 b/sql/hive/src/test/resources/golden/udf_translate-1-f7aa38a33ca0df73b7a1e6b6da4b7fe8 new file mode 100644 index 0000000000000..e7beead53b399 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 b/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 new file mode 100644 index 0000000000000..7d44692e4f7c0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-10-2f9daada8878081cb8954880ad5a65c4 @@ -0,0 +1 @@ +12cd 12cd \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b b/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b new file mode 100644 index 0000000000000..90e0d872f8bfc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-11-76b7a339d5c62808b9f4f78816d4c55b @@ -0,0 +1 @@ +123d \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 b/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 new file mode 100644 index 0000000000000..0770d02cb2303 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-12-a5b3e5fadeec1e03371160879f060b05 @@ -0,0 +1 @@ +Ãbcd \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 b/sql/hive/src/test/resources/golden/udf_translate-2-42aba80bf1913dd7c64545831f476c58 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a b/sql/hive/src/test/resources/golden/udf_translate-3-20904c8be8fed5cbd2d66ead6248a60a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba b/sql/hive/src/test/resources/golden/udf_translate-4-5d4abaf86254bacaa545c769bd7e50ba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 b/sql/hive/src/test/resources/golden/udf_translate-5-f2637240d227f9732d3db76f2e9d3a59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b b/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b new file mode 100644 index 0000000000000..0a54ee40ecb13 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-6-55122cc5ea4f49e737fee58945f3f21b @@ -0,0 +1 @@ +12cd 12d \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 b/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 new file mode 100644 index 0000000000000..0a54ee40ecb13 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-7-856c9d0157c34ab85cc6c83d560bfd47 @@ -0,0 +1 @@ +12cd 12d \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 b/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 new file mode 100644 index 0000000000000..50c123df9d1d3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-8-f56e5c0a7fdd81d6ca0bb84f2cadcf11 @@ -0,0 +1 @@ +1bc \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 b/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 new file mode 100644 index 0000000000000..b98d7e1c34024 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_translate-9-ee69663d7662760973b72785595be2b1 @@ -0,0 +1 @@ +NULL NULL NULL NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f b/sql/hive/src/test/resources/golden/udf_trim-0-18aa2b7ff8d263124ea47c9e27fc672f new file mode 100644 index 0000000000000..f0620a190ae4e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 b/sql/hive/src/test/resources/golden/udf_trim-1-e23715e112959e6840b6feed2ecf38a7 new file mode 100644 index 0000000000000..49a13dd64b286 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d b/sql/hive/src/test/resources/golden/udf_ucase-0-8f8c18102eb02df524106be5ea49f23d new file mode 100644 index 0000000000000..057c231dccf0e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f b/sql/hive/src/test/resources/golden/udf_ucase-1-640713eb89dbb09dbb6e5b472fc0ec8f new file mode 100644 index 0000000000000..128d1e21fae83 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 b/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 new file mode 100644 index 0000000000000..44b2a42cc26c5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unhex-0-11eb3cc5216d5446f4165007203acc47 @@ -0,0 +1 @@ +unhex(str) - Converts hexadecimal argument to binary diff --git a/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 b/sql/hive/src/test/resources/golden/udf_unhex-1-a660886085b8651852b9b77934848ae4 new file mode 100644 index 0000000000000..97af3b812a429 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 b/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 new file mode 100644 index 0000000000000..b4a6f2b692227 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unhex-2-78ba44cd7dae6619772c7620cb39b68 @@ -0,0 +1 @@ +MySQL 1267 a -4 diff --git a/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 b/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 new file mode 100644 index 0000000000000..3a67adaf0a9a8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_unhex-3-8823f9cc76adaf32a9a6110df99ce4d6 @@ -0,0 +1 @@ +NULL NULL NULL diff --git a/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c b/sql/hive/src/test/resources/golden/udf_union-0-e3498ef95bc9d8c8ce55e75962b4a92c new file mode 100644 index 0000000000000..3f3bb2b7ce02e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 b/sql/hive/src/test/resources/golden/udf_union-1-f6f01250718087029165e23badc02cd6 new file mode 100644 index 0000000000000..748ed97055401 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 b/sql/hive/src/test/resources/golden/udf_union-2-6af20858e3209d7cf37f736805ec5182 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 b/sql/hive/src/test/resources/golden/udf_union-3-705d165fec6761744dd19b142c566d61 new file mode 100644 index 0000000000000..4b9ae5402aa53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead b/sql/hive/src/test/resources/golden/udf_upper-0-47dc226b0435f668df20fe0e84293ead new file mode 100644 index 0000000000000..2e239cca5b860 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d b/sql/hive/src/test/resources/golden/udf_upper-1-f81f11c3563dbc9ff80496c4b10bcd1d new file mode 100644 index 0000000000000..673307e284b2e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_var_pop-0-3187e740690ccc1988a19fea4202a6de new file mode 100644 index 0000000000000..9dbf241e28107 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_var_pop-1-fd25e5226312bf54d918858511814766 new file mode 100644 index 0000000000000..234a65bd8255d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 b/sql/hive/src/test/resources/golden/udf_var_samp-0-b918928871d1b7f944315558c230c229 new file mode 100644 index 0000000000000..673deeb031b44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b b/sql/hive/src/test/resources/golden/udf_var_samp-1-59032ed5856fd4aa17c3e8e6721eec2b new file mode 100644 index 0000000000000..673deeb031b44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 b/sql/hive/src/test/resources/golden/udf_var_samp-2-b918928871d1b7f944315558c230c229 new file mode 100644 index 0000000000000..673deeb031b44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b b/sql/hive/src/test/resources/golden/udf_var_samp-3-59032ed5856fd4aa17c3e8e6721eec2b new file mode 100644 index 0000000000000..673deeb031b44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 b/sql/hive/src/test/resources/golden/udf_variance-0-fd23be1baa8b5ffa0d4519560d3fca87 new file mode 100644 index 0000000000000..f58ae0769d5a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a b/sql/hive/src/test/resources/golden/udf_variance-1-c1856abae908b05bfd6183189b4fd06a new file mode 100644 index 0000000000000..e17528e607aba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_variance-2-3187e740690ccc1988a19fea4202a6de new file mode 100644 index 0000000000000..9dbf241e28107 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_variance-3-fd25e5226312bf54d918858511814766 new file mode 100644 index 0000000000000..234a65bd8255d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 b/sql/hive/src/test/resources/golden/udf_variance-4-fd23be1baa8b5ffa0d4519560d3fca87 new file mode 100644 index 0000000000000..f58ae0769d5a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a b/sql/hive/src/test/resources/golden/udf_variance-5-c1856abae908b05bfd6183189b4fd06a new file mode 100644 index 0000000000000..e17528e607aba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de b/sql/hive/src/test/resources/golden/udf_variance-6-3187e740690ccc1988a19fea4202a6de new file mode 100644 index 0000000000000..9dbf241e28107 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 b/sql/hive/src/test/resources/golden/udf_variance-7-fd25e5226312bf54d918858511814766 new file mode 100644 index 0000000000000..234a65bd8255d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 b/sql/hive/src/test/resources/golden/udf_weekofyear-0-d6b4490b549a358be375511e39627dc2 new file mode 100644 index 0000000000000..29e3b370b03b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd b/sql/hive/src/test/resources/golden/udf_weekofyear-1-b7bbdfabe6054a66701250fd70065ddd new file mode 100644 index 0000000000000..c7939a11937c5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 b/sql/hive/src/test/resources/golden/udf_weekofyear-2-57ec578b3e226b6971e0fc0694b513d6 new file mode 100644 index 0000000000000..20d1dc50ce1f7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 b/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 new file mode 100644 index 0000000000000..2e36162dbaa33 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-0-88b97c6722176393e9b3d089559d2d11 @@ -0,0 +1 @@ +There is no documentation for function 'when' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 b/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 new file mode 100644 index 0000000000000..2e36162dbaa33 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-1-7365d5fe39dc7a025c942dad8fb9f0d4 @@ -0,0 +1 @@ +There is no documentation for function 'when' \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 b/sql/hive/src/test/resources/golden/udf_when-2-ff1118e7d06a4725e1e98a6d70a59295 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f b/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f new file mode 100644 index 0000000000000..6929fdca28eb8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_when-3-e63043e8d6ecf300c1fcf8654176896f @@ -0,0 +1 @@ +2 9 14 NULL 24 NULL \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 b/sql/hive/src/test/resources/golden/udf_xpath-0-622670bd1cbf4bf0cf04a553006e3c8 new file mode 100644 index 0000000000000..18d9720653a5b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 b/sql/hive/src/test/resources/golden/udf_xpath-1-5a6f10392053a4eabe62f5cabb073a71 new file mode 100644 index 0000000000000..3b092dafd32c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d b/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d new file mode 100644 index 0000000000000..0637a088a01e8 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-2-6b0a9d3874868d88d54ae133c978753d @@ -0,0 +1 @@ +[] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea b/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea new file mode 100644 index 0000000000000..8e8aff2387621 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-3-5700d81a9c2a22dcca287bf8439277ea @@ -0,0 +1 @@ +["b1","b2","b3","c1","c2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a b/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a new file mode 100644 index 0000000000000..e518b1f37e124 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-4-2960b453dd1dcf9ab7d4156c717d9e0a @@ -0,0 +1 @@ +["b1","b2","b3"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 b/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 new file mode 100644 index 0000000000000..45615b772ba35 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-5-7395e1cd3b543316a753978f556975e0 @@ -0,0 +1 @@ +["c1","c2"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f b/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f new file mode 100644 index 0000000000000..327bf3a1e76cc --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath-6-108134b2ae4a8de5f6257d6c6391fb3f @@ -0,0 +1 @@ +["b1","c1"] \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-0-8b2f3f8132bfc8344e243cdaf45eb371 new file mode 100644 index 0000000000000..134e9e99b7423 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-1-9e935539c9a3f3a118145096e7f978c4 new file mode 100644 index 0000000000000..624889cc0ac67 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-2-66148c16fde3008348a24cec5d9ba02b @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 new file mode 100644 index 0000000000000..02e4a84d62c4b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-3-c6cf7ede46f0412fe7a37ac52061b060 @@ -0,0 +1 @@ +false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-4-45d2e8baee72a0d741eb0a976af3a965 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 new file mode 100644 index 0000000000000..02e4a84d62c4b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-5-dbc57416174a6ba26f29dfc13f91f302 @@ -0,0 +1 @@ +false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f b/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f new file mode 100644 index 0000000000000..02e4a84d62c4b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-6-45d3c4fa4f86577fc26e9de7fc552e8f @@ -0,0 +1 @@ +false \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 new file mode 100644 index 0000000000000..f32a5804e292d --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_boolean-7-824c07ed4ef40cd509fea55575e43303 @@ -0,0 +1 @@ +true \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 b/sql/hive/src/test/resources/golden/udf_xpath_double-0-39199612969071d58b24034a2d17ca67 new file mode 100644 index 0000000000000..b56f4d00a8a74 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 b/sql/hive/src/test/resources/golden/udf_xpath_double-1-2d5ceab0a14d6e837ed153e1a2235bb2 new file mode 100644 index 0000000000000..b6b6227174647 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 b/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 new file mode 100644 index 0000000000000..7104585f9a439 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-10-fe9ae5013ee4f11b357128ee5ffb56c0 @@ -0,0 +1 @@ +7.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 b/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 new file mode 100644 index 0000000000000..6e6366051638f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-11-a1960676c1b2dc76aa178ea288d801d2 @@ -0,0 +1 @@ +5.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 b/sql/hive/src/test/resources/golden/udf_xpath_double-2-a4d22cea9dffaf26b485b3eb08963789 new file mode 100644 index 0000000000000..5b6de7db2f9f3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 b/sql/hive/src/test/resources/golden/udf_xpath_double-3-ea050c5b9e74d4b87b037236ef9e4fc2 new file mode 100644 index 0000000000000..b8accbcae1e17 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c b/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-4-93f47057c68385cff3b6f5c42307590c @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 b/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-5-c811a2353f5baf585da8654acd13b0e5 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 b/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 new file mode 100644 index 0000000000000..0a8ac8c629540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-6-10fcb281fd8db12dd1ac41380b8030c6 @@ -0,0 +1 @@ +8.0E19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c b/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-7-4532934141d2adb1f4ff56e65e8cf14c @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d b/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d new file mode 100644 index 0000000000000..9f8e9b69a33f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-8-547e750f5e401511db56283e66d1231d @@ -0,0 +1 @@ +1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e b/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e new file mode 100644 index 0000000000000..3d3be3c32ed85 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_double-9-b45b8b60031ac43ed2ffcd883ba2f19e @@ -0,0 +1 @@ +15.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 b/sql/hive/src/test/resources/golden/udf_xpath_float-0-7483dafee0dc7334eecabba31977e791 new file mode 100644 index 0000000000000..ea4df0ae58e12 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a b/sql/hive/src/test/resources/golden/udf_xpath_float-1-f6ddff2daba006787aeb861ca6f9d37a new file mode 100644 index 0000000000000..6bc4d4a46de72 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 b/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-2-70b7180f7bcd1d09727ad73084788d16 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 b/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 new file mode 100644 index 0000000000000..b955cad2a39a0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-3-79b7e57efe31a0373c39f0ba79b5bd54 @@ -0,0 +1 @@ +NaN \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 new file mode 100644 index 0000000000000..0a8ac8c629540 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-4-6e1f61ebe10c5fde60148e3a31706352 @@ -0,0 +1 @@ +8.0E19 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f b/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f new file mode 100644 index 0000000000000..171538eb0b00f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-5-410760f9560157587fbba7a677e12b9f @@ -0,0 +1 @@ +0.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf b/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf new file mode 100644 index 0000000000000..9f8e9b69a33f4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-6-d83a5002f52e3acb7dbb725bb434eebf @@ -0,0 +1 @@ +1.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa b/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa new file mode 100644 index 0000000000000..3d3be3c32ed85 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-7-b57c58f9343a82846f54ef60b216dfaa @@ -0,0 +1 @@ +15.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 b/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 new file mode 100644 index 0000000000000..7104585f9a439 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-8-3a62c730d4e5d9be6fb01307577d6f48 @@ -0,0 +1 @@ +7.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 b/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 new file mode 100644 index 0000000000000..6e6366051638f --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_float-9-16793f703b552fcb24f7aea9bb8a2401 @@ -0,0 +1 @@ +5.0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 b/sql/hive/src/test/resources/golden/udf_xpath_int-0-a9ed102a70e7e9d505be1555f0c7eb12 new file mode 100644 index 0000000000000..67ab19492e405 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 b/sql/hive/src/test/resources/golden/udf_xpath_int-1-26ddf7e46a66065909e9e547f0459910 new file mode 100644 index 0000000000000..33349b0b22ad9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba b/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-2-f10e246ebfd5f82545a3df64f51f58ba @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba b/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-3-eaa4a790c726cfa76a247d08a6ee7aba @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 b/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 new file mode 100644 index 0000000000000..55a89f5564d21 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-4-b9ce6e38b52d6ffb9fc62f5e0a373890 @@ -0,0 +1 @@ +2147483647 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 b/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-5-7907dda43c9b4e9e0b5b01cf20413cd7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a b/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-6-9bfa4fdc7d551742ff23efad8507ba0a @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 b/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-7-e7026efa45ba8af0f569fcbe4f7119a4 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 b/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 new file mode 100644 index 0000000000000..c7930257dfef5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-8-d31e667d1c2ade0bec52cddf8f2b3833 @@ -0,0 +1 @@ +7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 b/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_int-9-e6c2548098879ee747a933c755c4c869 @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b b/sql/hive/src/test/resources/golden/udf_xpath_long-0-d274b272251e95ec2e8139bfa471bd0b new file mode 100644 index 0000000000000..748fd827e8545 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 b/sql/hive/src/test/resources/golden/udf_xpath_long-1-43fbf4d944cf7eaf57b4d6166b803fb6 new file mode 100644 index 0000000000000..65241b6ce9b64 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b b/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-2-d697d943b1f7e7762d804064d11b905b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d b/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-3-c0b20c651ae5a352322067f97bd6ae5d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 b/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 new file mode 100644 index 0000000000000..996d127e59365 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-4-821e1cdea837bee7a8c2745bc3b85b9 @@ -0,0 +1 @@ +9223372036854775807 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d b/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-5-22a71b88c5bcb3db6e299a88ab791d4d @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 b/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-6-f2460325cf46c830631d8bc32565c787 @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 b/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-7-b3f1d4b505151180b82fddb18cf795d0 @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 b/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 new file mode 100644 index 0000000000000..c7930257dfef5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-8-c21bebec7d1a4aec99fba6b0a9a03083 @@ -0,0 +1 @@ +7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 b/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_long-9-7bafedc7b884df49a9f6752360969bf1 @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 b/sql/hive/src/test/resources/golden/udf_xpath_short-0-7d5231aed9cbbf68cd740791f9e5be17 new file mode 100644 index 0000000000000..22ccb9ca9defa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 b/sql/hive/src/test/resources/golden/udf_xpath_short-1-5d45932563b78e0b435b8cfebfe3cc2 new file mode 100644 index 0000000000000..931ce2abb3d4d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 b/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-2-10e9d4899d2fd352b58010c778c1f7a8 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 b/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-3-5ac84f46610107b1846f4a9b26a69576 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c b/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c new file mode 100644 index 0000000000000..d7d17fcbef95c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-4-8a300079521fefbe0d2f943851c1c53c @@ -0,0 +1 @@ +-1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e b/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-5-51f5de44cf1d5289fa5892ffe16e473e @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 b/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 new file mode 100644 index 0000000000000..56a6051ca2b02 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-6-fc85e2c7b89fe11adb7b6b1fb696bd04 @@ -0,0 +1 @@ +1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d b/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d new file mode 100644 index 0000000000000..3f10ffe7a4c47 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-7-e24ee123f331429c22de0a06054d0d5d @@ -0,0 +1 @@ +15 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 b/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 new file mode 100644 index 0000000000000..c7930257dfef5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-8-f8256e7c0dc4ac31303c1e21c8fcba95 @@ -0,0 +1 @@ +7 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e b/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e new file mode 100644 index 0000000000000..7813681f5b41c --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_short-9-22088ba0fb00eaa28e3460ca018b343e @@ -0,0 +1 @@ +5 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 b/sql/hive/src/test/resources/golden/udf_xpath_string-0-e315d11e9feb29177b5cb1e221c7cfa4 new file mode 100644 index 0000000000000..5b807fcf43195 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 b/sql/hive/src/test/resources/golden/udf_xpath_string-1-110b583cde6cd23c486d8223c444cbe9 new file mode 100644 index 0000000000000..8f3ab457faf5c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f b/sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f new file mode 100644 index 0000000000000..2f10d1e91ed93 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_string-2-a147b4eaa40b03355f666ea660cbff1f @@ -0,0 +1 @@ +bbcc diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a b/sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a new file mode 100644 index 0000000000000..e0b3f1b09bd18 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_string-3-a62072b86a6044d5b97911d662899b5a @@ -0,0 +1 @@ +bb diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a b/sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a new file mode 100644 index 0000000000000..46c1d6125b7b4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_string-4-152b0424ec68120f4ef15269eac0528a @@ -0,0 +1 @@ +cc diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 b/sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 new file mode 100644 index 0000000000000..8b137891791fe --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_string-5-2ab527e587583138ca764f5518a88c14 @@ -0,0 +1 @@ + diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 b/sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 new file mode 100644 index 0000000000000..c9c6af7f78bc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_string-6-58204a4df0fbf861028a1a94fb20ec61 @@ -0,0 +1 @@ +b1 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f b/sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f new file mode 100644 index 0000000000000..c9c6af7f78bc4 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_string-7-b5b211e4abd14990c28007c6638cb44f @@ -0,0 +1 @@ +b1 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc b/sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc new file mode 100644 index 0000000000000..e6bfff5c1d0f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_string-8-8dcf59ffc63ea3b225fcdd33dc2be9fc @@ -0,0 +1 @@ +b2 diff --git a/sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 b/sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 new file mode 100644 index 0000000000000..e6bfff5c1d0f0 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_xpath_string-9-647aca756f43e791dd5497e1b4c6af89 @@ -0,0 +1 @@ +b2 diff --git a/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a b/sql/hive/src/test/resources/golden/udtf_stack-0-665a27212319e1b2d1ad6b6dbaa3ce9a new file mode 100644 index 0000000000000..599bf1880a83a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 b/sql/hive/src/test/resources/golden/udtf_stack-1-879ca1a8453ced55a8617b390670a4e1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 b/sql/hive/src/test/resources/golden/udtf_stack-2-e9b1b8a1b6172a00deeb15a07f928dc5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 b/sql/hive/src/test/resources/golden/unicode_notation-0-8ee564d593fc64e0ad8a52b5d659f286 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-1-3de206f543c9e1525c54547f076b99c3 new file mode 100644 index 0000000000000..e82f93fe65c5e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 b/sql/hive/src/test/resources/golden/unicode_notation-2-9a9c2395773e362f0db32d5603673291 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e b/sql/hive/src/test/resources/golden/unicode_notation-3-4ac0de021c370cc6b8e753e00f96682e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-4-3de206f543c9e1525c54547f076b99c3 new file mode 100644 index 0000000000000..e82f93fe65c5e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 b/sql/hive/src/test/resources/golden/unicode_notation-5-9a9c2395773e362f0db32d5603673291 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 b/sql/hive/src/test/resources/golden/unicode_notation-6-b52052b427ea699f7bc7bee7e32d1de9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 b/sql/hive/src/test/resources/golden/unicode_notation-7-3de206f543c9e1525c54547f076b99c3 new file mode 100644 index 0000000000000..2be72c6e4e8c7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 b/sql/hive/src/test/resources/golden/unicode_notation-8-9a9c2395773e362f0db32d5603673291 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union10-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af b/sql/hive/src/test/resources/golden/union10-1-c5b4e04f745a28463e64aeeec6d4b2af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union10-2-60ee9eae687170317ff91dafe6b799bf b/sql/hive/src/test/resources/golden/union10-2-60ee9eae687170317ff91dafe6b799bf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 b/sql/hive/src/test/resources/golden/union10-3-6b4ed91859c7ab8d3548d4c7eeb65182 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 b/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 new file mode 100644 index 0000000000000..1d5891034ddec --- /dev/null +++ b/sql/hive/src/test/resources/golden/union10-4-7f83822f19aa9b973198fe4c42c66856 @@ -0,0 +1,3 @@ +tst1 500 +tst2 500 +tst3 500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union11-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 b/sql/hive/src/test/resources/golden/union11-1-a6f37db310de2edce6696c25a4fe96e3 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f b/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f new file mode 100644 index 0000000000000..1ec0096b865a7 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union11-2-cd756f39e22e121bdbd51400662aa47f @@ -0,0 +1,3 @@ +tst1 1 +tst2 1 +tst3 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union12-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 b/sql/hive/src/test/resources/golden/union12-1-9d2793d1cfd2645ac7f373a0a127e599 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 b/sql/hive/src/test/resources/golden/union12-2-69ba763b72862bf0f8b2bdf1a712e3b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union13-0-6970d1e2405f3769a28a8804887ac657 b/sql/hive/src/test/resources/golden/union13-0-6970d1e2405f3769a28a8804887ac657 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 b/sql/hive/src/test/resources/golden/union13-1-534d0853c5fc094404f65ca4631c1c20 new file mode 100644 index 0000000000000..2209b11d74282 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union14-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union14-1-11603fb6225423979af6c062cfa9389b b/sql/hive/src/test/resources/golden/union14-1-11603fb6225423979af6c062cfa9389b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 b/sql/hive/src/test/resources/golden/union14-2-8e01b2f4a18ad41a622e0aadbe680398 new file mode 100644 index 0000000000000..07f67df79dcbd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union15-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 b/sql/hive/src/test/resources/golden/union15-1-caf8a4bf47a5f5dfd27388fe00686c56 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 b/sql/hive/src/test/resources/golden/union15-2-a89acfb4bbc044c483b94e28152a41e0 new file mode 100644 index 0000000000000..b00b55f6fa92d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 b/sql/hive/src/test/resources/golden/union16-0-7d7f87e388835405f8c7baba969e04f5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 b/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 new file mode 100644 index 0000000000000..6f680072350ab --- /dev/null +++ b/sql/hive/src/test/resources/golden/union16-1-9f76074598f9b55d8afbb5659737a382 @@ -0,0 +1 @@ +12500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a b/sql/hive/src/test/resources/golden/union17-0-383b0c63b3fe31509d23612cb6ae88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union17-1-9491c271b7e2c351ddcf126f73679892 b/sql/hive/src/test/resources/golden/union17-1-9491c271b7e2c351ddcf126f73679892 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union17-2-e6fb104913b9da8193167ee263993dd1 b/sql/hive/src/test/resources/golden/union17-2-e6fb104913b9da8193167ee263993dd1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 b/sql/hive/src/test/resources/golden/union17-3-eef8248caa65e21a7c2956aa488297a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 b/sql/hive/src/test/resources/golden/union17-4-a6b1560ec2eb0bd64c9dd2c8daae99c5 new file mode 100644 index 0000000000000..a5a9e42fff209 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 b/sql/hive/src/test/resources/golden/union17-5-404392d6faff5db5f36b4aa87ac8e8c9 new file mode 100644 index 0000000000000..be13b26eadc86 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a b/sql/hive/src/test/resources/golden/union18-0-383b0c63b3fe31509d23612cb6ae88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union18-1-9491c271b7e2c351ddcf126f73679892 b/sql/hive/src/test/resources/golden/union18-1-9491c271b7e2c351ddcf126f73679892 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 b/sql/hive/src/test/resources/golden/union18-2-584e61e1599fe449cc998a3332dcb7e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 b/sql/hive/src/test/resources/golden/union18-3-b86b4936bb620a8f5d929943f4aa75f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 b/sql/hive/src/test/resources/golden/union18-4-1799ebb147238db6032fd6fe2fd36878 new file mode 100644 index 0000000000000..3dee790d7c2ca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 b/sql/hive/src/test/resources/golden/union18-5-b12dcddfa4f02a14318f6564947c98a0 new file mode 100644 index 0000000000000..e438a64050723 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a b/sql/hive/src/test/resources/golden/union19-0-383b0c63b3fe31509d23612cb6ae88a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union19-1-9491c271b7e2c351ddcf126f73679892 b/sql/hive/src/test/resources/golden/union19-1-9491c271b7e2c351ddcf126f73679892 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union19-2-b6161b64e57a2502c79662866782cbdf b/sql/hive/src/test/resources/golden/union19-2-b6161b64e57a2502c79662866782cbdf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 b/sql/hive/src/test/resources/golden/union19-3-4e2ace50cdaad15bbe48cb793f72cbd2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 b/sql/hive/src/test/resources/golden/union19-4-1799ebb147238db6032fd6fe2fd36878 new file mode 100644 index 0000000000000..e4dfefc43e26a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 b/sql/hive/src/test/resources/golden/union19-5-b12dcddfa4f02a14318f6564947c98a0 new file mode 100644 index 0000000000000..e438a64050723 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union2-0-58813daf3d1af57b17518b606f8345b2 b/sql/hive/src/test/resources/golden/union2-0-58813daf3d1af57b17518b606f8345b2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 b/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 new file mode 100644 index 0000000000000..e37d32abba426 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union2-1-90d739774cb96e7d0d96513c1c9968b4 @@ -0,0 +1 @@ +1000 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d b/sql/hive/src/test/resources/golden/union20-0-a7f64187712936e741d5c33b2f5dfa6d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c b/sql/hive/src/test/resources/golden/union20-1-968e353589f1fddb914242beb25be94c new file mode 100644 index 0000000000000..385b8df6703a0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c b/sql/hive/src/test/resources/golden/union21-0-ecfd22e2a24ed9f113229c80a2aaee9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 b/sql/hive/src/test/resources/golden/union21-1-fb1497f4c21bf7d28162f27d50320d13 new file mode 100644 index 0000000000000..26a2aff52e9a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union22-0-4bd6583b3635f1db95765565518bda8d b/sql/hive/src/test/resources/golden/union22-0-4bd6583b3635f1db95765565518bda8d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 b/sql/hive/src/test/resources/golden/union22-1-9037e5e0cb2d64e4efe6280dc03396f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union22-10-90e9c4388340428a1f68868e9322c400 b/sql/hive/src/test/resources/golden/union22-10-90e9c4388340428a1f68868e9322c400 new file mode 100644 index 0000000000000..a9df5cf33c86d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 b/sql/hive/src/test/resources/golden/union22-2-a4c7c7ba0177c0a8fe415e2bd3e93002 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 b/sql/hive/src/test/resources/golden/union22-3-291783e57ccb7c6cad6b5e8d2dae89a0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union22-4-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union22-4-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union22-4-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union22-5-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union22-5-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union22-5-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 b/sql/hive/src/test/resources/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union22-6-44d382ce6848d3f0b900b0808747d8e9 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union22-7-25fc734982956a164adde6bb1d4d8751 b/sql/hive/src/test/resources/golden/union22-7-25fc734982956a164adde6bb1d4d8751 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union22-7-25fc734982956a164adde6bb1d4d8751 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 b/sql/hive/src/test/resources/golden/union22-8-c78e8af8e8e327c58a74f7cb34a1c912 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 b/sql/hive/src/test/resources/golden/union22-9-f4414e4636a16596d04fcc433d1119b6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 b/sql/hive/src/test/resources/golden/union23-0-c86f3e120d66f1b06a8b916a4a67e4b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd b/sql/hive/src/test/resources/golden/union23-1-7830963417e3535034962e2597970ddd new file mode 100644 index 0000000000000..bf0f76662bd2c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 b/sql/hive/src/test/resources/golden/union24-0-5659f2e36b79fa312aaf6e233dc575c9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c b/sql/hive/src/test/resources/golden/union24-1-92aaed1a225be0e1b3d4556680a77b8c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 b/sql/hive/src/test/resources/golden/union24-10-ae53b6e1d433f000da3fbe130222b89 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a b/sql/hive/src/test/resources/golden/union24-11-a41e75063b4cac0dbaf99b4aabc3201a new file mode 100644 index 0000000000000..e850208008112 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 b/sql/hive/src/test/resources/golden/union24-2-4afc74338258d50ae6ecdb0589bd2a38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f b/sql/hive/src/test/resources/golden/union24-3-d3370b90a3ace4b4d1cefd1ffc79906f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union24-4-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union24-4-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union24-4-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union24-5-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 b/sql/hive/src/test/resources/golden/union24-6-83d9274a535ebebe0170ce8f16062ef4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 b/sql/hive/src/test/resources/golden/union24-7-afb3649d2de5113691a37d0c77b28b31 new file mode 100644 index 0000000000000..2251af07b7c05 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb b/sql/hive/src/test/resources/golden/union24-8-b6d9a41e2b537869e977cfb969b51edb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 b/sql/hive/src/test/resources/golden/union24-9-db8ce0526e33b73f5589e3e34aca0198 new file mode 100644 index 0000000000000..17fd2beaf0661 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d b/sql/hive/src/test/resources/golden/union26-0-a0d1a7f8eb3fba9455ab517aab14552d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union26-1-4702087f160230463e6114d6534d47e4 b/sql/hive/src/test/resources/golden/union26-1-4702087f160230463e6114d6534d47e4 new file mode 100644 index 0000000000000..9ba1c19df23f3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 b/sql/hive/src/test/resources/golden/union26-2-5c88f10d25ae39c1e2df8f4beaa8d562 new file mode 100644 index 0000000000000..9ba1c19df23f3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union26-3-4702087f160230463e6114d6534d47e4 b/sql/hive/src/test/resources/golden/union26-3-4702087f160230463e6114d6534d47e4 new file mode 100644 index 0000000000000..9ba1c19df23f3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 b/sql/hive/src/test/resources/golden/union27-0-7f319eff3c0237b6c06cb704dcde9195 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f b/sql/hive/src/test/resources/golden/union27-1-fb2ddef0c7d9b05d7fb26442599d354f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 b/sql/hive/src/test/resources/golden/union27-2-10d1593b6663231ff7de938024ce6bb6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 b/sql/hive/src/test/resources/golden/union27-3-ab84df3813ff23be99f148449610e530 new file mode 100644 index 0000000000000..199095f1f9848 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 b/sql/hive/src/test/resources/golden/union28-0-6bccc08f175712c17aa66cebbf5b7e36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 b/sql/hive/src/test/resources/golden/union28-1-f1bd585c115dbabf655d8dc38cd4e026 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f b/sql/hive/src/test/resources/golden/union28-2-9985bbc70b8dd624fac261eafb10a78f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union28-3-b1d75ba0d33a452619e41f70e69616e9 new file mode 100644 index 0000000000000..293f324297419 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 b/sql/hive/src/test/resources/golden/union29-0-6bccc08f175712c17aa66cebbf5b7e36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union29-1-a40036f756a306f4226634b6ab67c72c b/sql/hive/src/test/resources/golden/union29-1-a40036f756a306f4226634b6ab67c72c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 b/sql/hive/src/test/resources/golden/union29-2-c7fab64fc276c13149e1e68c0deb6594 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union29-3-b1d75ba0d33a452619e41f70e69616e9 new file mode 100644 index 0000000000000..87a971baed428 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d b/sql/hive/src/test/resources/golden/union3-0-6a8a35102de1b0b88c6721a704eb174d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 b/sql/hive/src/test/resources/golden/union3-1-ae80732ae5db530cc4e355d5d4724457 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 b/sql/hive/src/test/resources/golden/union3-2-2a1dcd937f117f1955a169592b96d5f9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 b/sql/hive/src/test/resources/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 new file mode 100644 index 0000000000000..94ebaf9001613 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union3-3-8fc63f8edb2969a63cd4485f1867ba97 @@ -0,0 +1,4 @@ +1 +2 +3 +4 diff --git a/sql/hive/src/test/resources/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 b/sql/hive/src/test/resources/golden/union30-0-6bccc08f175712c17aa66cebbf5b7e36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 b/sql/hive/src/test/resources/golden/union30-1-1aff6fba829bae7d20266cf91383ef78 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union30-2-252f12f7532ca67132bfc62063c58430 b/sql/hive/src/test/resources/golden/union30-2-252f12f7532ca67132bfc62063c58430 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 b/sql/hive/src/test/resources/golden/union30-3-b1d75ba0d33a452619e41f70e69616e9 new file mode 100644 index 0000000000000..a6f502e6c3e3a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 b/sql/hive/src/test/resources/golden/union31-0-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-1-b12aa87aaf64b573ce0be7013117651b b/sql/hive/src/test/resources/golden/union31-1-b12aa87aaf64b573ce0be7013117651b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 b/sql/hive/src/test/resources/golden/union31-10-ca695e6973bc98b7f06c998b964ae339 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 b/sql/hive/src/test/resources/golden/union31-11-70dda95c1b749e4104cc2c767cd7fec0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-12-de2477a134ab37bb22d879028becf2e5 b/sql/hive/src/test/resources/golden/union31-12-de2477a134ab37bb22d879028becf2e5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 b/sql/hive/src/test/resources/golden/union31-13-22b1078764cb1c9b2fb95358c968e987 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 b/sql/hive/src/test/resources/golden/union31-14-c36a1d8de2713f722ec42bc4686d6125 new file mode 100644 index 0000000000000..dadf53962e7a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e b/sql/hive/src/test/resources/golden/union31-15-5df6435aed6e0a6a6853480a027b911e new file mode 100644 index 0000000000000..dadf53962e7a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 b/sql/hive/src/test/resources/golden/union31-16-ca16024e6f5399b1d035f5b9fd665163 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-17-b12aa87aaf64b573ce0be7013117651b b/sql/hive/src/test/resources/golden/union31-17-b12aa87aaf64b573ce0be7013117651b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 b/sql/hive/src/test/resources/golden/union31-18-1522ec86f2e6735cf809a52c1f6f7883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e b/sql/hive/src/test/resources/golden/union31-19-1e9320795b3e4acd7e6d89a8c9ba897e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 b/sql/hive/src/test/resources/golden/union31-2-1522ec86f2e6735cf809a52c1f6f7883 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-20-dceb6d9a03096d68d090557692ed6899 b/sql/hive/src/test/resources/golden/union31-20-dceb6d9a03096d68d090557692ed6899 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-21-bda2e62d0ecd30200b2359df420da574 b/sql/hive/src/test/resources/golden/union31-21-bda2e62d0ecd30200b2359df420da574 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a b/sql/hive/src/test/resources/golden/union31-22-fa1c85506afbfa956251cd1253f5d79a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df b/sql/hive/src/test/resources/golden/union31-23-c9d9d45a7d1a0c69efb8d8babc4a50df new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 b/sql/hive/src/test/resources/golden/union31-24-df38c8164af7cc164c728b8178da72c5 new file mode 100644 index 0000000000000..ede645acaf95b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 b/sql/hive/src/test/resources/golden/union31-25-1485e295a99908e1862eae397b814045 new file mode 100644 index 0000000000000..ede645acaf95b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc b/sql/hive/src/test/resources/golden/union31-3-35d1dd67e8c3d76c77baaeae0760ddfc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-4-77653483e5ce18437c9f85d3b297335d b/sql/hive/src/test/resources/golden/union31-4-77653483e5ce18437c9f85d3b297335d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 b/sql/hive/src/test/resources/golden/union31-5-4651987fca60c1c1a59b8b28713ea737 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 b/sql/hive/src/test/resources/golden/union31-6-69c2b75a25f884a72d89e43e0186f5a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 b/sql/hive/src/test/resources/golden/union31-7-7b021f95774705e4d7ff7049ee0e0b74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb b/sql/hive/src/test/resources/golden/union31-8-ba92b89786ffaecd74a740705e0fa0cb new file mode 100644 index 0000000000000..dadf53962e7a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 b/sql/hive/src/test/resources/golden/union31-9-56dfdb30edd8a687f9aa9cad29b42760 new file mode 100644 index 0000000000000..a71793e8c52a3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e b/sql/hive/src/test/resources/golden/union32-0-4c7f0fb27ce3a8f80b02dab6fc5cb45e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 b/sql/hive/src/test/resources/golden/union32-1-e6c80e7d6171ae5fc428506e57dc8753 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b b/sql/hive/src/test/resources/golden/union32-10-44a9b373ee7d43a4ef2bc4b8a708601b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 b/sql/hive/src/test/resources/golden/union32-11-40d51bc9958b27c36ef647f0598fdee5 new file mode 100644 index 0000000000000..9bf0de2f06c9a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 b/sql/hive/src/test/resources/golden/union32-2-8e39fef33c859ef83912d0fcda319218 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 b/sql/hive/src/test/resources/golden/union32-3-d31e252450077ac54f4cb18a9ad95a84 new file mode 100644 index 0000000000000..462dca3124c41 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e b/sql/hive/src/test/resources/golden/union32-4-79787e084ca15d479cee3a7e1ed2281e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 b/sql/hive/src/test/resources/golden/union32-5-51c997d0a1103b60764bbb8316a38746 new file mode 100644 index 0000000000000..2e662f39d1572 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b b/sql/hive/src/test/resources/golden/union32-6-96fa13d8790bbfa1d6109b7cbf890d1b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 b/sql/hive/src/test/resources/golden/union32-7-f936440d63f4e1027dda4de605660633 new file mode 100644 index 0000000000000..2e662f39d1572 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb b/sql/hive/src/test/resources/golden/union32-8-f42d83f502a7b8d3a36331e0f5621cfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db b/sql/hive/src/test/resources/golden/union32-9-74fb695786df4c024288ae23ac8c00db new file mode 100644 index 0000000000000..106f1874451ff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f b/sql/hive/src/test/resources/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union33-0-c67a488530dc7e20a9e7acf02c14380f @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union33-1-1df7c476abb48234c839878414720873 b/sql/hive/src/test/resources/golden/union33-1-1df7c476abb48234c839878414720873 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union33-2-3987150ad66482f5186826efcfb117f7 b/sql/hive/src/test/resources/golden/union33-2-3987150ad66482f5186826efcfb117f7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union33-3-bfe75625806b414a5098af487b91046a b/sql/hive/src/test/resources/golden/union33-3-bfe75625806b414a5098af487b91046a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 b/sql/hive/src/test/resources/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 new file mode 100644 index 0000000000000..a1e0432c9a7d5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union33-4-1d22c1e85a4bfa338aff713d45440877 @@ -0,0 +1 @@ +312 diff --git a/sql/hive/src/test/resources/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 b/sql/hive/src/test/resources/golden/union33-5-13c01085e0a2c46e745985fa11bedf34 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 b/sql/hive/src/test/resources/golden/union33-6-796e4fce2bfb63c09bc04ac6eca7c9d8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 b/sql/hive/src/test/resources/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 new file mode 100644 index 0000000000000..a1e0432c9a7d5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union33-7-1d22c1e85a4bfa338aff713d45440877 @@ -0,0 +1 @@ +312 diff --git a/sql/hive/src/test/resources/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 b/sql/hive/src/test/resources/golden/union34-0-a9a6e5b0059d8f66a79ee702f2bec0a4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union34-1-320c2fe802fab386957039aaff6399f1 b/sql/hive/src/test/resources/golden/union34-1-320c2fe802fab386957039aaff6399f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 b/sql/hive/src/test/resources/golden/union34-10-da2b79118c21ac45ce85001fa61b0043 new file mode 100644 index 0000000000000..d572335ff5185 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 b/sql/hive/src/test/resources/golden/union34-2-5707c9fd2cfd049426383e5e5dc80d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union34-3-c179e14d840249d6953cb552279822f1 b/sql/hive/src/test/resources/golden/union34-3-c179e14d840249d6953cb552279822f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 b/sql/hive/src/test/resources/golden/union34-4-101829a66cab2efd31dcb0d86e302956 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union34-5-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f b/sql/hive/src/test/resources/golden/union34-6-ff0312eeb487fc393a06880ef5bb286f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 b/sql/hive/src/test/resources/golden/union34-7-da2b79118c21ac45ce85001fa61b0043 new file mode 100644 index 0000000000000..d572335ff5185 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b b/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union34-8-b1e2ade89ae898650f0be4f796d8947b @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 b/sql/hive/src/test/resources/golden/union34-9-d82ffe361a119651bbf41a149b6eba91 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union4-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union4-1-88deb3987655dc41989f39c0827cec74 b/sql/hive/src/test/resources/golden/union4-1-88deb3987655dc41989f39c0827cec74 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 b/sql/hive/src/test/resources/golden/union4-2-41739c4bd273f0cfa94b347c36a71f38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b b/sql/hive/src/test/resources/golden/union4-3-2fb47e6ec0a780da5bda97752e573c0b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 b/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 new file mode 100644 index 0000000000000..948aca9180ba9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union4-4-7f83822f19aa9b973198fe4c42c66856 @@ -0,0 +1,2 @@ +tst1 500 +tst2 500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union5-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 b/sql/hive/src/test/resources/golden/union5-1-54c6369ded20ba12138e6c53a157dae8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 b/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 new file mode 100644 index 0000000000000..c0c7d9f5e8ef5 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union5-2-2c19c8d564b010eeb42deee63d66a292 @@ -0,0 +1,2 @@ +tst1 1 +tst2 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union6-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d b/sql/hive/src/test/resources/golden/union6-1-7d631b7dbfe39ccaf995e16186b3e94d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 b/sql/hive/src/test/resources/golden/union6-2-31cbd867e4b88605d230c542af64b7c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 b/sql/hive/src/test/resources/golden/union6-3-c39f37580f6b3cecbcf3192ee0cd2aa9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 b/sql/hive/src/test/resources/golden/union6-4-a03959cc5aaa8f6521a73e6dae04cd15 new file mode 100644 index 0000000000000..b5e91e032212a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 b/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union7-0-863233ccd616401efb4bf83c4b9e3a52 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 b/sql/hive/src/test/resources/golden/union7-1-f449f20f6bfc4f038c79c3b7e8f48c37 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 b/sql/hive/src/test/resources/golden/union7-2-55d6e503a281acf3289a7874c0fba3f5 new file mode 100644 index 0000000000000..07f67df79dcbd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e b/sql/hive/src/test/resources/golden/union8-0-d0a83a1cedcbd1d2158acf7eb938d18e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 b/sql/hive/src/test/resources/golden/union8-1-1b422e4c1c8b97775518f760b995c771 new file mode 100644 index 0000000000000..3617909902993 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 b/sql/hive/src/test/resources/golden/union9-0-fad6a053ab42c3cad89052e88d819e83 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 b/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 new file mode 100644 index 0000000000000..37021f4a27201 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union9-1-a77ee9f723b3b17a3a02164c5d0000c1 @@ -0,0 +1 @@ +1500 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 b/sql/hive/src/test/resources/golden/union_date-0-a0bade1c77338d4f72962389a1f5bea2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c b/sql/hive/src/test/resources/golden/union_date-1-21306adbd8be8ad75174ad9d3e42b73c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 b/sql/hive/src/test/resources/golden/union_date-2-b386e4435da29e7e240893639c6dd9c4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 b/sql/hive/src/test/resources/golden/union_date-3-f0f678c9032603ca0ae729a31c8c9d38 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c b/sql/hive/src/test/resources/golden/union_date-4-d812f7feef3b6857aeca9007f0af44c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d b/sql/hive/src/test/resources/golden/union_date-5-b54839e0200bec94aa751fec8c5dbd3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 b/sql/hive/src/test/resources/golden/union_lateralview-0-fbcc72d0cd912977292df8a9e55a4e98 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 b/sql/hive/src/test/resources/golden/union_lateralview-1-55dc0ec224133e4714291cc13a7a8ce0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 b/sql/hive/src/test/resources/golden/union_lateralview-2-2252a6cc9b8af3e9c6eb12d470072339 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 b/sql/hive/src/test/resources/golden/union_lateralview-3-97ea99689397f6e0c8b07d598dd9c042 new file mode 100644 index 0000000000000..71bb7c3ad9311 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a b/sql/hive/src/test/resources/golden/union_null-0-27e98c4939abf1ad4445b4e715b0262a new file mode 100644 index 0000000000000..468f774ef5146 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b b/sql/hive/src/test/resources/golden/union_ppr-0-eadce09bf5dfa4849ed4536e04c0c32b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed b/sql/hive/src/test/resources/golden/union_ppr-1-4d073ff9d6978f3c3be447920ab8daed new file mode 100644 index 0000000000000..ed8993a9cbd0c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_1-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_1-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_1-10-3ef350a0f7bbc1c54774e33dd54d9e46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_1-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..c203753d8e42d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_1-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_1-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_1-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_1-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_1-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 b/sql/hive/src/test/resources/golden/union_remove_1-6-a6c043a89a9c3456af8ee065cb17239 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_1-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_1-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_1-9-19865a08066d80cb069ae6312c465ee6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_10-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_10-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 b/sql/hive/src/test/resources/golden/union_remove_10-10-7eaf13bc61bd4b43f4da67c347768598 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 b/sql/hive/src/test/resources/golden/union_remove_10-11-b62595b91d2d9e03a010b49ab81725d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_10-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..94a711cdab117 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_10-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_10-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_10-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_10-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_10-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_10-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 b/sql/hive/src/test/resources/golden/union_remove_10-7-a7ba0436265932086d2a2e228356971 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_10-8-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_10-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_11-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_11-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 b/sql/hive/src/test/resources/golden/union_remove_11-10-9788525d32af3dccdefb459669f886c1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a b/sql/hive/src/test/resources/golden/union_remove_11-11-42ecec4117d372551310ad1d85a9c09a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_11-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..d9cfa9e952c3a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_11-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_11-14-e409e7032445097ace016b1876d95b3e new file mode 100644 index 0000000000000..2817d74afee13 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_11-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_11-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_11-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_11-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_11-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 b/sql/hive/src/test/resources/golden/union_remove_11-7-ba03d14418197f8ad6496d30c0c29a59 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_11-8-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_11-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_12-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_12-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_12-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c b/sql/hive/src/test/resources/golden/union_remove_12-11-a667f24e26435cd2a29fef0ee45bab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 b/sql/hive/src/test/resources/golden/union_remove_12-12-c696750d8b7b65b00bf9d402cf7abaa5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_12-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..57d5146698ee7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_12-14-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_12-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_12-3-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_12-4-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_12-5-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_12-6-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_12-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 b/sql/hive/src/test/resources/golden/union_remove_12-8-9dd030d38eece4630dec5951fc8a0622 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_12-9-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_13-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_13-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_13-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 b/sql/hive/src/test/resources/golden/union_remove_13-11-ed5d1c2bfc3dd1b2f5321bc8800e69e8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af b/sql/hive/src/test/resources/golden/union_remove_13-12-27c0fa25c4f67fc1e0e113aca6dd47af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_13-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..c914024c14c8e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_13-14-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_13-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_13-3-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_13-4-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_13-5-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_13-6-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_13-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb b/sql/hive/src/test/resources/golden/union_remove_13-8-234ecbc3f8c0e4686d3586f81cf191eb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_13-9-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_14-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_14-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_14-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c b/sql/hive/src/test/resources/golden/union_remove_14-11-a667f24e26435cd2a29fef0ee45bab3c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 b/sql/hive/src/test/resources/golden/union_remove_14-12-c696750d8b7b65b00bf9d402cf7abaa5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_14-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..5a55428afbe1a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_14-14-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_14-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_14-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_14-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 b/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_14-5-24ca942f094b14b92086305cc125e833 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_14-6-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_14-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d b/sql/hive/src/test/resources/golden/union_remove_14-8-ed33d620523b2634285698a83f433b6d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_14-9-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_15-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 b/sql/hive/src/test/resources/golden/union_remove_15-11-93b7341b523213ab6e58169459bc6818 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 b/sql/hive/src/test/resources/golden/union_remove_15-12-616cc477ed00e691dbc2b310d1c6dd12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_15-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..a2441e330ffc6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 new file mode 100644 index 0000000000000..a58273a1b054e --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-14-37f9f7bc2d7456046a9f967347337e47 @@ -0,0 +1,2 @@ +ds=1 +ds=2 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_15-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d b/sql/hive/src/test/resources/golden/union_remove_15-8-552c4eba867e7408fc8652ff0a19170d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_15-9-a63925fb2aa8c5df6854c248e674b0ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_16-10-a63925fb2aa8c5df6854c248e674b0ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_16-11-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 b/sql/hive/src/test/resources/golden/union_remove_16-12-93b7341b523213ab6e58169459bc6818 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 b/sql/hive/src/test/resources/golden/union_remove_16-13-616cc477ed00e691dbc2b310d1c6dd12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_16-14-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..bec8202dcbcad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 new file mode 100644 index 0000000000000..a58273a1b054e --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-15-37f9f7bc2d7456046a9f967347337e47 @@ -0,0 +1,2 @@ +ds=1 +ds=2 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-16-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-7-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_16-8-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a b/sql/hive/src/test/resources/golden/union_remove_16-9-ec47ba0fc527a4a04d452a009d59147a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_17-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 b/sql/hive/src/test/resources/golden/union_remove_17-11-a55dcc374cb69e225a4f1da56bbd7a68 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d b/sql/hive/src/test/resources/golden/union_remove_17-12-c91289e16ad403babfc91c093ac9b86d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_17-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..d0b37e7625a64 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 new file mode 100644 index 0000000000000..c21f4017362c1 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-14-37f9f7bc2d7456046a9f967347337e47 @@ -0,0 +1,2 @@ +ds=1 +ds=2 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 b/sql/hive/src/test/resources/golden/union_remove_17-16-626a252f75285872c1d72706f7f972c6 new file mode 100644 index 0000000000000..1ab5c482feca9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a b/sql/hive/src/test/resources/golden/union_remove_17-17-1252985379f11ae4b98d2a0e2f121b8a new file mode 100644 index 0000000000000..94ec062e2f4f3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_17-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b b/sql/hive/src/test/resources/golden/union_remove_17-8-67e57f56d1106a57329bf75c491e3c8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef b/sql/hive/src/test/resources/golden/union_remove_17-9-a63925fb2aa8c5df6854c248e674b0ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_18-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_18-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_18-10-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad b/sql/hive/src/test/resources/golden/union_remove_18-11-37ae8e8b4f39f3ff5516afd44715c2ad new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 b/sql/hive/src/test/resources/golden/union_remove_18-12-461a24ba6c00f60d75afba62da2ac4f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_18-13-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..971e095df7be8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 b/sql/hive/src/test/resources/golden/union_remove_18-14-37f9f7bc2d7456046a9f967347337e47 new file mode 100644 index 0000000000000..5690d94a65857 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_18-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_18-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_18-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_18-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe b/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_18-5-16367c381d4b189b3640c92511244bfe @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 b/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_18-6-a4fb8359a2179ec70777aad6366071b7 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_18-7-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 b/sql/hive/src/test/resources/golden/union_remove_18-8-b1b996b2e72ca14150af7c82dbc6e139 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 b/sql/hive/src/test/resources/golden/union_remove_18-9-ea467d0fee062a23c720cf47eacfef08 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_19-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_19-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b b/sql/hive/src/test/resources/golden/union_remove_19-10-8d1e5af37e7992708bf15ab7d887405b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_19-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..0d0d3a30af36c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_19-12-68e15b1729669c4cb2291dcabfea4387 new file mode 100644 index 0000000000000..719e4a7bd4dbb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 b/sql/hive/src/test/resources/golden/union_remove_19-13-b8b9df8f376df228e3c8ae65defe2801 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 b/sql/hive/src/test/resources/golden/union_remove_19-14-eed866a1ad3106a306322519f4bb52f2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 new file mode 100644 index 0000000000000..361f2ab78eae3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_19-15-68e15b1729669c4cb2291dcabfea4387 @@ -0,0 +1,2 @@ +7 1 +7 1 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e b/sql/hive/src/test/resources/golden/union_remove_19-16-471f8e794fd712dce2e40334b383e08e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 b/sql/hive/src/test/resources/golden/union_remove_19-17-ae0f7d5734ca67cbfebed70c4657e330 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_19-18-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_19-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_19-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_19-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_19-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 b/sql/hive/src/test/resources/golden/union_remove_19-6-a6c043a89a9c3456af8ee065cb17239 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_19-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_19-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 b/sql/hive/src/test/resources/golden/union_remove_19-9-8a08edd1aa63fd3b051da82246793259 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_2-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_2-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_2-10-2309570010c3e679b884c100de57d002 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_2-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..d3c821fa22182 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_2-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_2-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_2-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_2-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_2-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e b/sql/hive/src/test/resources/golden/union_remove_2-6-1acf02741a2ff987d3e00ae9722c26e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_2-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_2-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_2-9-4a9974900fb5bc2fc8e5d614f5026c0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_20-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_20-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 b/sql/hive/src/test/resources/golden/union_remove_20-10-89c57c91facbf54299e08955e3783ea6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_20-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..978049640929c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_20-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_20-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_20-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_20-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_20-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e b/sql/hive/src/test/resources/golden/union_remove_20-6-a73143117ffec1225f6d492e5aa577e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f b/sql/hive/src/test/resources/golden/union_remove_20-7-82f81adc097c247475fd29076e0cb85f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_20-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa b/sql/hive/src/test/resources/golden/union_remove_20-9-683949ae07de12da0b7e7ba7f4450daa new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_21-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_21-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 b/sql/hive/src/test/resources/golden/union_remove_21-10-e19469a1b70be25caaf670fe68f0a747 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_21-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..ce7e0438f9eff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_21-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_21-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_21-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_21-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_21-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f b/sql/hive/src/test/resources/golden/union_remove_21-6-aeba356b56f8659963d8b2dc07a84a6f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 b/sql/hive/src/test/resources/golden/union_remove_21-7-5716c408db679fb88352eaceb1703bd7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_21-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 b/sql/hive/src/test/resources/golden/union_remove_21-9-deadbce171926623b0586587fbbcd144 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_22-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_22-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 b/sql/hive/src/test/resources/golden/union_remove_22-10-403471d96c56b565272d2e4c4926d240 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_22-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..c261ce48bdfb9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 b/sql/hive/src/test/resources/golden/union_remove_22-12-68e15b1729669c4cb2291dcabfea4387 new file mode 100644 index 0000000000000..d36ac6b834792 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd b/sql/hive/src/test/resources/golden/union_remove_22-13-a28b876b5df29140ef2bf62b4d0de3fd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af b/sql/hive/src/test/resources/golden/union_remove_22-14-8eb6d4bed951caa7ba9fe6b3a83d76af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_22-15-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_22-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_22-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_22-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_22-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 b/sql/hive/src/test/resources/golden/union_remove_22-6-722acd65700dd132dc6b4bc8c56f4ce0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 b/sql/hive/src/test/resources/golden/union_remove_22-7-46da090f5a2c73b175207cf63ff46653 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_22-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb b/sql/hive/src/test/resources/golden/union_remove_22-9-b3edbbee0543ff268db4059afb9cb2cb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_23-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_23-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 b/sql/hive/src/test/resources/golden/union_remove_23-10-613ce50facecdc8d7bf8806a8ff17c13 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_23-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..50297abd4e9a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_23-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_23-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_23-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_23-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_23-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 b/sql/hive/src/test/resources/golden/union_remove_23-6-b4e2e8cf42d1f3d4f54615a7b3d4c9a7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_23-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_23-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 b/sql/hive/src/test/resources/golden/union_remove_23-9-2dca12ca51c74540e7cdbbb05e336ed5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_24-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_24-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 b/sql/hive/src/test/resources/golden/union_remove_24-10-6d89089b1eead05510dbccad5fcc4805 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_24-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..f3c452ebe9284 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_24-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_24-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_24-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_24-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_24-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 b/sql/hive/src/test/resources/golden/union_remove_24-6-72ae7a9640ae611f61ac954ac1a4b682 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e b/sql/hive/src/test/resources/golden/union_remove_24-7-44a7b0f14b4f5151c37498367ad7fe1e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_24-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f b/sql/hive/src/test/resources/golden/union_remove_24-9-4c0550cc9c28de25993c1f98de39168f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_3-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_3-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 b/sql/hive/src/test/resources/golden/union_remove_3-10-cfc49418c6621c665dfb671ed5656fc9 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_3-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..d46af80f0d1ae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_3-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_3-13-e409e7032445097ace016b1876d95b3e new file mode 100644 index 0000000000000..2817d74afee13 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_3-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_3-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_3-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_3-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 b/sql/hive/src/test/resources/golden/union_remove_3-6-c36130e41df18093eee24b45fc0846f0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_3-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_3-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 b/sql/hive/src/test/resources/golden/union_remove_3-9-7da122d7b42dc7ef504ccc442d6545f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_4-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_4-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_4-10-19865a08066d80cb069ae6312c465ee6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_4-11-3ef350a0f7bbc1c54774e33dd54d9e46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_4-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..67faa5b13d984 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_4-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_4-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_4-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_4-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_4-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_4-6-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead b/sql/hive/src/test/resources/golden/union_remove_4-7-90fcd4eb330919ad92aecb8a5bf30ead new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_4-8-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_4-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_5-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_5-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_5-10-4a9974900fb5bc2fc8e5d614f5026c0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_5-11-2309570010c3e679b884c100de57d002 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_5-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..19aeb0b602859 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_5-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_5-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_5-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_5-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_5-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_5-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 b/sql/hive/src/test/resources/golden/union_remove_5-7-4da6ca94af4caf4426e5676a32b70375 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_5-8-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_5-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_6-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_6-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 b/sql/hive/src/test/resources/golden/union_remove_6-10-52171e0094ea92aaea8a485aedc88e88 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 b/sql/hive/src/test/resources/golden/union_remove_6-11-60361645974d38829674ec6bb5502a62 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_6-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e b/sql/hive/src/test/resources/golden/union_remove_6-13-e409e7032445097ace016b1876d95b3e new file mode 100644 index 0000000000000..8e7ee8a2b47bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 b/sql/hive/src/test/resources/golden/union_remove_6-14-f74b28904e86047150396bc42680ca38 new file mode 100644 index 0000000000000..8e7ee8a2b47bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_6-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_6-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_6-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_6-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 b/sql/hive/src/test/resources/golden/union_remove_6-6-22f22c916f3bc3bf0a28259c7b232b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a b/sql/hive/src/test/resources/golden/union_remove_6-7-48f70528347f5201f387d28dae37a14a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf b/sql/hive/src/test/resources/golden/union_remove_6-8-19e054c1050e5dbf703e91a48253ecdf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_6-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_7-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_7-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 b/sql/hive/src/test/resources/golden/union_remove_7-10-3ef350a0f7bbc1c54774e33dd54d9e46 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_7-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..1c63b3851aad2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_7-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_7-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_7-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_7-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_7-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca b/sql/hive/src/test/resources/golden/union_remove_7-6-3744f0ebf5c002fdfcec67fbce03dfca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_7-7-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_7-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 b/sql/hive/src/test/resources/golden/union_remove_7-9-19865a08066d80cb069ae6312c465ee6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_8-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_8-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 b/sql/hive/src/test/resources/golden/union_remove_8-10-2309570010c3e679b884c100de57d002 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_8-11-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..f718ab2990cdd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_8-12-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_8-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 b/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_8-3-dc129f70e75cd575ce8c0de288884523 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab b/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_8-4-a572a07cd60fd4607ddd7613db8a64ab @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_8-5-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee b/sql/hive/src/test/resources/golden/union_remove_8-6-895f2432b4be6fcb11641c1d063570ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_8-7-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_8-8-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f b/sql/hive/src/test/resources/golden/union_remove_8-9-4a9974900fb5bc2fc8e5d614f5026c0f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 b/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_9-0-3c29684bfd2df7439ee0551eb42cfa0 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c b/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_9-1-16a6a293f1d2ce481b1d2482b1d5787c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 b/sql/hive/src/test/resources/golden/union_remove_9-10-12cf3335c756f8715a07c5a604f10f64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c b/sql/hive/src/test/resources/golden/union_remove_9-11-4e84cd589eceda668833f8f19ec28e7c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 b/sql/hive/src/test/resources/golden/union_remove_9-12-ea111d286c70e4a0c6a68a7420dc7b7 new file mode 100644 index 0000000000000..65a91a6027545 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a b/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_9-13-43d53504df013e6b35f81811138a167a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 b/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_9-2-cafed8ca348b243372b9114910be1557 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a b/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_9-3-b12e5c70d6d29757471b900b6160fa8a @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 b/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_9-4-593999fae618b6b38322bc9ae4e0c027 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_9-5-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 b/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_remove_9-6-a2a411ad6620aa1ab24550ade336e785 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 b/sql/hive/src/test/resources/golden/union_remove_9-7-f77ac921b27860fac94cac6b352f3eb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 b/sql/hive/src/test/resources/golden/union_remove_9-8-68f30e8e5c05bbedbda95d88ab6f3ee2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e b/sql/hive/src/test/resources/golden/union_remove_9-9-8ff0bb1bf3da91b51d37923f1876be0e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 b/sql/hive/src/test/resources/golden/union_script-0-ca195b08d98d7f21fe93208499bf0ff6 new file mode 100644 index 0000000000000..42d07ed10bad4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 b/sql/hive/src/test/resources/golden/union_script-1-982cc6d7b98f8fb1055a10ef021e2769 new file mode 100644 index 0000000000000..2f23db6a2ff4a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d b/sql/hive/src/test/resources/golden/union_view-0-e56367a21517656c18a5bcfeecb4327d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 b/sql/hive/src/test/resources/golden/union_view-1-c790d4344144460224b0f02be7e137a8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 b/sql/hive/src/test/resources/golden/union_view-2-7e33b0744f57bdc8ebcd9d45348aef14 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 b/sql/hive/src/test/resources/golden/union_view-3-4a746bd076e063017c4d6a2f9218a6e4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e b/sql/hive/src/test/resources/golden/union_view-4-d3d75f376f83b694b1dc62c46fa53f4e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 b/sql/hive/src/test/resources/golden/union_view-5-5d56e4e800a2a2ea70bd1f2ac31f2245 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae b/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_view-6-c9d7dcde469d3b9a66965a64dd15e4ae @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 b/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_view-7-3b03210f94ec40db9ab02620645014d1 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 b/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/union_view-8-35f48c7d6fa164bb84643657bc9280a8 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af b/sql/hive/src/test/resources/golden/varchar_2-0-fbced67562c536a7e9b6d3928bd9d0af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 b/sql/hive/src/test/resources/golden/varchar_2-1-bee4e3a9ded7276960e619e6185bf329 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_2-2-779a555944479991520be53f7f247626 b/sql/hive/src/test/resources/golden/varchar_2-2-779a555944479991520be53f7f247626 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da b/sql/hive/src/test/resources/golden/varchar_2-3-a8c072e5b13997e9c79484b4af9d78da new file mode 100644 index 0000000000000..08661a3d7f794 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f b/sql/hive/src/test/resources/golden/varchar_2-4-c7ac6a1b3e9416e42d2d1b56f017fe6f new file mode 100644 index 0000000000000..08661a3d7f794 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 b/sql/hive/src/test/resources/golden/varchar_2-5-96353c24b5f2b361f72e5c26b4135519 new file mode 100644 index 0000000000000..f0c744ca37d95 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f b/sql/hive/src/test/resources/golden/varchar_2-6-2b62789d07b4044bc32190261bf3490f new file mode 100644 index 0000000000000..f0c744ca37d95 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af b/sql/hive/src/test/resources/golden/varchar_2-7-fbced67562c536a7e9b6d3928bd9d0af new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e b/sql/hive/src/test/resources/golden/varchar_join1-0-f170a8c3d5db4cbf4e7c10aee1ef652e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 b/sql/hive/src/test/resources/golden/varchar_join1-1-8318072ee410cae36186c9880989a1a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce b/sql/hive/src/test/resources/golden/varchar_join1-10-1958143ee083437e87662cadb48c37ce new file mode 100644 index 0000000000000..35e7795b9c2a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c b/sql/hive/src/test/resources/golden/varchar_join1-11-a55f750032663f77066e4979dedea1c new file mode 100644 index 0000000000000..35e7795b9c2a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e b/sql/hive/src/test/resources/golden/varchar_join1-12-f170a8c3d5db4cbf4e7c10aee1ef652e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 b/sql/hive/src/test/resources/golden/varchar_join1-13-8318072ee410cae36186c9880989a1a5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 b/sql/hive/src/test/resources/golden/varchar_join1-14-8bbaae82c89553165b975c5b8ca3aee4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 b/sql/hive/src/test/resources/golden/varchar_join1-2-8bbaae82c89553165b975c5b8ca3aee4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 b/sql/hive/src/test/resources/golden/varchar_join1-3-735f68c77eae74623c8bdbebc995a3c7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 b/sql/hive/src/test/resources/golden/varchar_join1-4-d5105d36e4d077e0929597124526c532 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 b/sql/hive/src/test/resources/golden/varchar_join1-5-93624b74270b7ebd40bbb8c0a7e93e28 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef b/sql/hive/src/test/resources/golden/varchar_join1-6-6bb08c5baa913d9dc506aef65425ef new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 b/sql/hive/src/test/resources/golden/varchar_join1-7-341118dab140b17f0b9d2c7b101d1298 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 b/sql/hive/src/test/resources/golden/varchar_join1-8-afe7304d94450481c01ddbaf6cc3f596 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 b/sql/hive/src/test/resources/golden/varchar_join1-9-c152d3dc6495ef75e4872dea4b7e0f82 new file mode 100644 index 0000000000000..35e7795b9c2a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc b/sql/hive/src/test/resources/golden/varchar_nested_types-0-e41b0e9db7a9dbd0540e29df341933bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 b/sql/hive/src/test/resources/golden/varchar_nested_types-1-87a28b0e800f051525899324a064b878 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 b/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 new file mode 100644 index 0000000000000..05066db296756 --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-10-92f25849990eb777ac2711f9dd2e628 @@ -0,0 +1 @@ +c1 array None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 b/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 new file mode 100644 index 0000000000000..05cd8fccf276c --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-11-9780781a92fdd992f3cee080a8717238 @@ -0,0 +1 @@ +["val_0","val_0"] diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f b/sql/hive/src/test/resources/golden/varchar_nested_types-12-e1a4006971319a352280fc52eabf449f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 b/sql/hive/src/test/resources/golden/varchar_nested_types-13-d4574217a243a7d506398a819cd0eab4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 b/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 new file mode 100644 index 0000000000000..1ec6828af9fd2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-14-7c33a62195359bc89460ad65f6a5f763 @@ -0,0 +1 @@ +c1 map None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 b/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 new file mode 100644 index 0000000000000..8f12062de4e27 --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-15-d5e5cec4ce94e8928e69e3ffb8e0215 @@ -0,0 +1 @@ +{0:"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 b/sql/hive/src/test/resources/golden/varchar_nested_types-16-93811d146a429c44a2494b3aa4b2caa2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc b/sql/hive/src/test/resources/golden/varchar_nested_types-17-5724af3985c67a0cb69919c9bbce15dc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 b/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 new file mode 100644 index 0000000000000..030d9d6ec0429 --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-18-d1be2ee3765a80469837ba11eb8685e1 @@ -0,0 +1 @@ +c1 struct None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 b/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 new file mode 100644 index 0000000000000..f350c1112ac2c --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-19-a54f9a284228e7cdce8c34b8094f2377 @@ -0,0 +1 @@ +{"a":0,"b":"val_0","c":"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb b/sql/hive/src/test/resources/golden/varchar_nested_types-2-ae69b82461acc2aa366d8b1f8626d6fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 b/sql/hive/src/test/resources/golden/varchar_nested_types-20-51a9279006f61097e68a52201daf6710 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad b/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad new file mode 100644 index 0000000000000..030d9d6ec0429 --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-21-1d304e09cc2a8b2824bfc04bdbc976ad @@ -0,0 +1 @@ +c1 struct None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 b/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 new file mode 100644 index 0000000000000..f350c1112ac2c --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-22-c2bf5acbe5ebc16eaf2388fd36a61fe8 @@ -0,0 +1 @@ +{"a":0,"b":"val_0","c":"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 b/sql/hive/src/test/resources/golden/varchar_nested_types-23-c7ea918777b725f2790da4fa00a3efa6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 b/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 new file mode 100644 index 0000000000000..030d9d6ec0429 --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-24-5a69236334a3a1c4e771206cf547d730 @@ -0,0 +1 @@ +c1 struct None diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb b/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb new file mode 100644 index 0000000000000..f350c1112ac2c --- /dev/null +++ b/sql/hive/src/test/resources/golden/varchar_nested_types-25-61b030bb2220a533532d871ae0e08cdb @@ -0,0 +1 @@ +{"a":0,"b":"val_0","c":"val_0"} diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc b/sql/hive/src/test/resources/golden/varchar_nested_types-26-e41b0e9db7a9dbd0540e29df341933bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 b/sql/hive/src/test/resources/golden/varchar_nested_types-27-87a28b0e800f051525899324a064b878 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb b/sql/hive/src/test/resources/golden/varchar_nested_types-28-ae69b82461acc2aa366d8b1f8626d6fb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 b/sql/hive/src/test/resources/golden/varchar_nested_types-29-39d44d19c7963a9647fd3293eef670d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 b/sql/hive/src/test/resources/golden/varchar_nested_types-3-39d44d19c7963a9647fd3293eef670d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 b/sql/hive/src/test/resources/golden/varchar_nested_types-30-162806477075d97de16dfa6f2576b751 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 b/sql/hive/src/test/resources/golden/varchar_nested_types-31-5b28e1fdb28b365ef419008a4752ed53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 b/sql/hive/src/test/resources/golden/varchar_nested_types-4-162806477075d97de16dfa6f2576b751 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 b/sql/hive/src/test/resources/golden/varchar_nested_types-5-5b28e1fdb28b365ef419008a4752ed53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b b/sql/hive/src/test/resources/golden/varchar_nested_types-6-a47b99c355df4aad56dceb7f81a7fd5b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 b/sql/hive/src/test/resources/golden/varchar_nested_types-7-8c483a7a0e148ca13a292a625f8702f1 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 b/sql/hive/src/test/resources/golden/varchar_nested_types-8-94309e2b4e68ab7e25f7d9656f10b352 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 b/sql/hive/src/test/resources/golden/varchar_nested_types-9-f248796769bc7f57cf56a75034a45520 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 b/sql/hive/src/test/resources/golden/varchar_serde-0-750a23ebdd77f32b555d4caba7ac5445 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d b/sql/hive/src/test/resources/golden/varchar_serde-1-6af2fafa0ffdb36eec00f28ed8e1d76d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 b/sql/hive/src/test/resources/golden/varchar_serde-10-496280d9365ee601d9b68a91495d7160 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e b/sql/hive/src/test/resources/golden/varchar_serde-11-2f4d5f574f659df2bd0fb97f2d50a36e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 b/sql/hive/src/test/resources/golden/varchar_serde-12-80727f22f1343407ba9200c86ed84280 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e b/sql/hive/src/test/resources/golden/varchar_serde-13-4829e6e5822c73fb33ba3d619b4bd31e new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 b/sql/hive/src/test/resources/golden/varchar_serde-14-4794ee93811ce3a81f64c65aed0b8b13 new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 b/sql/hive/src/test/resources/golden/varchar_serde-15-6ae634e1ae1aa0730a33396bce5a6604 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 b/sql/hive/src/test/resources/golden/varchar_serde-16-516202183287d734d35d8c7788d22652 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a b/sql/hive/src/test/resources/golden/varchar_serde-17-8d4419222a728e2bbc6a464b5a0b5f7a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d b/sql/hive/src/test/resources/golden/varchar_serde-18-b2dfd28d6a5b1e6f4556ad19755b739d new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad b/sql/hive/src/test/resources/golden/varchar_serde-19-f258df2db09e9cc0e049e85e6ad950ad new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e b/sql/hive/src/test/resources/golden/varchar_serde-2-87ba3f40293b9c79fcdb3064d964232e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 b/sql/hive/src/test/resources/golden/varchar_serde-20-dcbc6c836bd9b95be56dd0c9c3ea83d2 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b b/sql/hive/src/test/resources/golden/varchar_serde-21-64536c77ae91bfb6cf7f93f178c6200b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 b/sql/hive/src/test/resources/golden/varchar_serde-22-e7fea2396e28d9db1dc9dea3aacc7b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 b/sql/hive/src/test/resources/golden/varchar_serde-23-d4dfc5f1e98b84b16b2b5b1d50917b70 new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 b/sql/hive/src/test/resources/golden/varchar_serde-24-770169b632b76cedcd6dfb87fdc46575 new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc b/sql/hive/src/test/resources/golden/varchar_serde-25-3470a259b04e126c655531491787e2fc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f b/sql/hive/src/test/resources/golden/varchar_serde-26-55808e190e0ab81dcdc1feb52543ad9f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 b/sql/hive/src/test/resources/golden/varchar_serde-27-8fe526fdd347c25529a383f27ad20566 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e b/sql/hive/src/test/resources/golden/varchar_serde-28-5e4de93349ba89a8344bb799ad60678e new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c b/sql/hive/src/test/resources/golden/varchar_serde-29-42b8e7b82cd061dfd388fbf13c8f6b0c new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 b/sql/hive/src/test/resources/golden/varchar_serde-3-d2a0276dbf0ab98a4aa5192b7c0f2709 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a b/sql/hive/src/test/resources/golden/varchar_serde-30-c7a29f8b2668b31bb34cafdc1e6f3c9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 b/sql/hive/src/test/resources/golden/varchar_serde-31-db1e88efcd55aaef567edaa89c1c1e12 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb b/sql/hive/src/test/resources/golden/varchar_serde-32-bffe130dfc0a3b6a072b5ebbaf092cfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 b/sql/hive/src/test/resources/golden/varchar_serde-33-620729dc7661c22f1acdc425a7cf0364 new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 b/sql/hive/src/test/resources/golden/varchar_serde-34-807ee73e1cd66704dd585f7e0de954d9 new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 b/sql/hive/src/test/resources/golden/varchar_serde-35-750a23ebdd77f32b555d4caba7ac5445 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d b/sql/hive/src/test/resources/golden/varchar_serde-36-6af2fafa0ffdb36eec00f28ed8e1d76d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e b/sql/hive/src/test/resources/golden/varchar_serde-37-87ba3f40293b9c79fcdb3064d964232e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 b/sql/hive/src/test/resources/golden/varchar_serde-38-d2a0276dbf0ab98a4aa5192b7c0f2709 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f b/sql/hive/src/test/resources/golden/varchar_serde-39-8b5f81c27c41807c757522e257a8003f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f b/sql/hive/src/test/resources/golden/varchar_serde-4-8b5f81c27c41807c757522e257a8003f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba b/sql/hive/src/test/resources/golden/varchar_serde-40-787193a1679a2153c037d3e4c8192bba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba b/sql/hive/src/test/resources/golden/varchar_serde-5-787193a1679a2153c037d3e4c8192bba new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 b/sql/hive/src/test/resources/golden/varchar_serde-6-122f15d410249b554e12eccdfa46cc43 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 b/sql/hive/src/test/resources/golden/varchar_serde-7-81e68559f5ba5fe6d3c590ae75fdd810 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 b/sql/hive/src/test/resources/golden/varchar_serde-8-3bfa13085b001c1a333cc72d5c9f4244 new file mode 100644 index 0000000000000..3f02a574d3dd5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 b/sql/hive/src/test/resources/golden/varchar_serde-9-ec43be632e5f74057aba54c4f562c601 new file mode 100644 index 0000000000000..1e2a6a6562290 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e b/sql/hive/src/test/resources/golden/varchar_union1-0-433f0ff2f1c8dc92fba226759326067e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec b/sql/hive/src/test/resources/golden/varchar_union1-1-2d1b9ffae67b8cff19b5c0c9ada38cec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 b/sql/hive/src/test/resources/golden/varchar_union1-10-6ec48d5fea3e4a35275956b9b4467715 new file mode 100644 index 0000000000000..173eb955a485a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 b/sql/hive/src/test/resources/golden/varchar_union1-11-78f6e219b974e1fdf3663e46f57892a9 new file mode 100644 index 0000000000000..173eb955a485a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e b/sql/hive/src/test/resources/golden/varchar_union1-12-433f0ff2f1c8dc92fba226759326067e new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec b/sql/hive/src/test/resources/golden/varchar_union1-13-2d1b9ffae67b8cff19b5c0c9ada38cec new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 b/sql/hive/src/test/resources/golden/varchar_union1-14-217ba1b02c693576b652d719970a0048 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 b/sql/hive/src/test/resources/golden/varchar_union1-2-217ba1b02c693576b652d719970a0048 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca b/sql/hive/src/test/resources/golden/varchar_union1-3-ce2fdb2b3ec26c9a6b351d10ac8283ca new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b b/sql/hive/src/test/resources/golden/varchar_union1-4-86e7275fc6c5ce64da5031aa7c79821b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 b/sql/hive/src/test/resources/golden/varchar_union1-5-bf77db56df2b2a077d668e47211be708 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 b/sql/hive/src/test/resources/golden/varchar_union1-6-67e66fa14dddc17757436539eca9ef64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 b/sql/hive/src/test/resources/golden/varchar_union1-7-48766d09c5ed1b6abe9ce0b8996adf36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 b/sql/hive/src/test/resources/golden/varchar_union1-8-3a6c2e0125fb0c18f5f783b5d3c409d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 b/sql/hive/src/test/resources/golden/varchar_union1-9-b9455ffec62df97cfec63204ce02a110 new file mode 100644 index 0000000000000..173eb955a485a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/view-0-5528e36b3b0f5b14313898cc45f9c23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 b/sql/hive/src/test/resources/golden/view-1-7650b86c86dd6b1a99c86ddc5a31bd63 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 b/sql/hive/src/test/resources/golden/view-10-7aae4448a05e8a8a3bace7522e952cd0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 b/sql/hive/src/test/resources/golden/view-11-dc95343d3e57846485dd543476391376 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 b/sql/hive/src/test/resources/golden/view-12-371764e1cae31ea0518c03060528d239 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 b/sql/hive/src/test/resources/golden/view-13-2abce88008f8a19164758ee821aaa8a6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee b/sql/hive/src/test/resources/golden/view-14-deb504f4f70fd7db975950c3c47959ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b b/sql/hive/src/test/resources/golden/view-15-6f2797b6f81943d3b53b8d247ae8512b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 b/sql/hive/src/test/resources/golden/view-2-9c529f486fa81a032bfe1253808fca8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a b/sql/hive/src/test/resources/golden/view-3-89c80c0e90409d5e304775c9f420915a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 b/sql/hive/src/test/resources/golden/view-4-4a64d1a623ca71e515796787dbd0f904 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b b/sql/hive/src/test/resources/golden/view-5-f6d1bce095ecbf1aa484891392fdb07b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 b/sql/hive/src/test/resources/golden/view-6-47b5043f03a84695b6784682b4402ac8 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 b/sql/hive/src/test/resources/golden/view-7-8b1bbdadfd1e11af1b56064196164e58 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 b/sql/hive/src/test/resources/golden/view-8-60d2f3ee552ae7021f9fa72f0dcf2867 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 b/sql/hive/src/test/resources/golden/view-9-66c68babac10ae0f645fe8334c5a42d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf b/sql/hive/src/test/resources/golden/view_cast-0-89bd46ad04f967f1f5ee17c6f201aacf new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb b/sql/hive/src/test/resources/golden/view_cast-1-85685a26971fb51ab6e28f9c5e8421bb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 b/sql/hive/src/test/resources/golden/view_cast-10-a7c865e5180df8d73dba90ede8be0d45 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de b/sql/hive/src/test/resources/golden/view_cast-2-af2050aa97f0cd930cb1b8ec791007de new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 b/sql/hive/src/test/resources/golden/view_cast-3-2a232c31d056e6abc76f8ebe53ccd97 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 b/sql/hive/src/test/resources/golden/view_cast-4-d9edb83f4cf847e141d97012314917d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a b/sql/hive/src/test/resources/golden/view_cast-5-6db508ccd85562a9ca7841fb0a08981a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 b/sql/hive/src/test/resources/golden/view_cast-6-aa5be3380ddc7104258567b406d93cc5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb b/sql/hive/src/test/resources/golden/view_cast-7-78ac3800b22682b31708b6a09b402bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 b/sql/hive/src/test/resources/golden/view_cast-8-2cc0c576f0a008abf5bdf3308d500869 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 b/sql/hive/src/test/resources/golden/view_cast-9-f306bf3ad1c2a99f6f1843db44d7dfb4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 b/sql/hive/src/test/resources/golden/view_inputs-0-9e67dfd1d595ab8b1935b789645f76c0 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe b/sql/hive/src/test/resources/golden/view_inputs-1-5af97e73bc3841793440105aae766bbe new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d b/sql/hive/src/test/resources/golden/virtual_column-0-9bacd1908e56d621913a74fe9a583d9d new file mode 100644 index 0000000000000..ea00577174e43 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 b/sql/hive/src/test/resources/golden/virtual_column-1-3ebad682d4ff6ca9c806db3471cf3945 new file mode 100644 index 0000000000000..df07a9da29f01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 b/sql/hive/src/test/resources/golden/virtual_column-10-2915b222a58bc994246591e536d388b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 b/sql/hive/src/test/resources/golden/virtual_column-2-1536b365fe0a94b30a62364996529966 new file mode 100644 index 0000000000000..2a0cdc268347a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 b/sql/hive/src/test/resources/golden/virtual_column-3-c66776673c986b59b27e704664935988 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 b/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 new file mode 100644 index 0000000000000..94754d8bd4c82 --- /dev/null +++ b/sql/hive/src/test/resources/golden/virtual_column-4-e47094c927b1091e31c185db0a4e69a6 @@ -0,0 +1 @@ +97 val_97 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 b/sql/hive/src/test/resources/golden/virtual_column-5-d137fa7c27bc98d5f1a33f666a07f6b7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 b/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 new file mode 100644 index 0000000000000..c227083464fb9 --- /dev/null +++ b/sql/hive/src/test/resources/golden/virtual_column-6-68d6973677af5c9f1f1f49360c3175e7 @@ -0,0 +1 @@ +0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 b/sql/hive/src/test/resources/golden/virtual_column-7-20d7d672a4289fbd1a5de485a8353ac6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 b/sql/hive/src/test/resources/golden/virtual_column-8-2b312255c5d1dcf9a617b1ebfd8a00f7 new file mode 100644 index 0000000000000..e6902f60ff486 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 b/sql/hive/src/test/resources/golden/virtual_column-9-c152da33c1517ecfc336f72b9c133d5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/add_partition_with_whitelist.q new file mode 100644 index 0000000000000..8f0a60b713ab8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/addpart1.q new file mode 100644 index 0000000000000..a7c9fe91f6cd9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_concatenate_indexed_table.q new file mode 100644 index 0000000000000..4881757a4613a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_non_native.q new file mode 100644 index 0000000000000..73ae853778832 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_2columns.q new file mode 100644 index 0000000000000..e10f77cf3f161 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidcolname.q new file mode 100644 index 0000000000000..66eba75d4084d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_coltype_invalidtype.q new file mode 100644 index 0000000000000..ad016c5f3a76e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_invalidspec.q new file mode 100644 index 0000000000000..5f9d5ef9ca09e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop.q new file mode 100644 index 0000000000000..92af30b6b5a15 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_nodrop_table.q new file mode 100644 index 0000000000000..135411fd320ef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_offline.q new file mode 100644 index 0000000000000..899145deaf8c5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_partition_with_whitelist.q new file mode 100644 index 0000000000000..6e33bc0782d29 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure.q new file mode 100644 index 0000000000000..26ba2878903fe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure2.q new file mode 100644 index 0000000000000..6e51c2f7629c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_rename_partition_failure3.q new file mode 100644 index 0000000000000..2d4ce0b9f6051 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_add_partition.q new file mode 100644 index 0000000000000..2427c3b2a45f8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_table_wrong_regex.q new file mode 100644 index 0000000000000..fad194d016ec9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_not_exist.q new file mode 100644 index 0000000000000..30fe4d9916ab3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_as_select_with_partition.q new file mode 100644 index 0000000000000..dca6770b1b17e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure.q new file mode 100644 index 0000000000000..705b985095fa3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure2.q new file mode 100644 index 0000000000000..26d2c4f3ad2fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure3.q new file mode 100644 index 0000000000000..49c17a8b573c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure4.q new file mode 100644 index 0000000000000..e2fad270b1d89 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure5.q new file mode 100644 index 0000000000000..e44766e113063 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure6.q new file mode 100644 index 0000000000000..dab7b145f7c4e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure7.q new file mode 100644 index 0000000000000..eff04c5b47dee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure8.q new file mode 100644 index 0000000000000..9dff784250616 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/alter_view_failure9.q new file mode 100644 index 0000000000000..0f40fad90d974 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/altern1.q new file mode 100644 index 0000000000000..60414c1f3a7a5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col.q new file mode 100644 index 0000000000000..866cec126f782 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col0.q new file mode 100644 index 0000000000000..46349c60bc79b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col1.q new file mode 100644 index 0000000000000..fdf20f850e8f8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ambiguous_col2.q new file mode 100644 index 0000000000000..de59bc579a0fe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q new file mode 100644 index 0000000000000..874f5bfc14125 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze.q @@ -0,0 +1 @@ +analyze table srcpart compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q new file mode 100644 index 0000000000000..057a1a0b482ea --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze1.q @@ -0,0 +1 @@ +analyze table srcpart partition (key) compute statistics; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/analyze_view.q new file mode 100644 index 0000000000000..af4970f52e8be --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive1.q new file mode 100644 index 0000000000000..a4b50f5e14107 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive2.q new file mode 100644 index 0000000000000..ff8dcb248568c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive3.q new file mode 100644 index 0000000000000..53057daa1b622 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive4.q new file mode 100644 index 0000000000000..56d6f1798debe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive5.q new file mode 100644 index 0000000000000..4f6dc8d72ceed --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_corrupt.q new file mode 100644 index 0000000000000..bea25391628a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert1.q new file mode 100644 index 0000000000000..deaff63d673a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert2.q new file mode 100644 index 0000000000000..d744f2487694c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert3.q new file mode 100644 index 0000000000000..c6cb142824c8d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_insert4.q new file mode 100644 index 0000000000000..c36f3ef9e8775 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi1.q new file mode 100644 index 0000000000000..8c702ed008bfa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi2.q new file mode 100644 index 0000000000000..d3cfb89c98749 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi3.q new file mode 100644 index 0000000000000..75f5dfad47b37 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi4.q new file mode 100644 index 0000000000000..abe0647ae6ee1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi5.q new file mode 100644 index 0000000000000..71635e054a1e9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi6.q new file mode 100644 index 0000000000000..5bb1474fdc383 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_multi7.q new file mode 100644 index 0000000000000..db7f392737e97 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec1.q new file mode 100644 index 0000000000000..d83b19d9fe31a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec2.q new file mode 100644 index 0000000000000..ed14bbf688d52 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec3.q new file mode 100644 index 0000000000000..f27ad6d63b08e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec4.q new file mode 100644 index 0000000000000..491c2ac4596fc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/archive_partspec5.q new file mode 100644 index 0000000000000..bb25ef2c7e0f1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_1.q new file mode 100644 index 0000000000000..c38dab5eb702f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_2.q new file mode 100644 index 0000000000000..341e44774d9c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_3.q new file mode 100644 index 0000000000000..6a56daa05fee8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_4.q new file mode 100644 index 0000000000000..f0cb6459a255b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_5.q new file mode 100644 index 0000000000000..b4efab5667f6c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_6.q new file mode 100644 index 0000000000000..977246948cad0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_fail_7.q new file mode 100644 index 0000000000000..492deed10bfeb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/authorization_part.q new file mode 100644 index 0000000000000..a654a2380c75c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/autolocal1.q new file mode 100644 index 0000000000000..bd1c9d6e15a76 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_exec_hooks.q new file mode 100644 index 0000000000000..709d8d9c85442 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_indextype.q new file mode 100644 index 0000000000000..8f5bf42664b96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bad_sample_clause.q new file mode 100644 index 0000000000000..fd6769827b829 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_mismatch1.q new file mode 100644 index 0000000000000..048a8fd5cfb8e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_1.q new file mode 100644 index 0000000000000..9478a2f1b989c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/bucket_mapjoin_wrong_table_metadata_2.q new file mode 100644 index 0000000000000..69afe0ae700ed --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cachingprintstream.q new file mode 100644 index 0000000000000..d57a4517f00fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/cluster_tasklog_retrieval.q new file mode 100644 index 0000000000000..bc980448a9e22 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbydistributeby.q new file mode 100644 index 0000000000000..4c6a9b38d785d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbyorderby.q new file mode 100644 index 0000000000000..d9ee9b9d262d0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clusterbysortby.q new file mode 100644 index 0000000000000..7b4e744ba66df --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern1.q new file mode 100644 index 0000000000000..0ff44779657ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern2.q new file mode 100644 index 0000000000000..9ed8944d2bb6c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern3.q new file mode 100644 index 0000000000000..23f73667edf5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/clustern4.q new file mode 100644 index 0000000000000..3a9b45ca60576 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_change_skewedcol_type1.q new file mode 100644 index 0000000000000..9a3e0b2efe693 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename1.q new file mode 100644 index 0000000000000..d99b821802df7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename2.q new file mode 100644 index 0000000000000..cccc8ad54e300 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename3.q new file mode 100644 index 0000000000000..91c9537a99ad5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename4.q new file mode 100644 index 0000000000000..dd89a5a10b227 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/column_rename5.q new file mode 100644 index 0000000000000..3827b83361fbb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_dp.q new file mode 100644 index 0000000000000..af923504c8c4f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_incorrect_num_keys.q new file mode 100644 index 0000000000000..d9725ddc7fdf6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_invalid_values.q new file mode 100644 index 0000000000000..eb73962ba133c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_partlvl_multiple_part_clause.q new file mode 100644 index 0000000000000..dbfaaecbdf614 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl.q new file mode 100644 index 0000000000000..ca8548958fcd6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_complex_type.q new file mode 100644 index 0000000000000..5bbd70d86b297 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/columnstats_tbllvl_incorrect_column.q new file mode 100644 index 0000000000000..ca8548958fcd6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_double_bigint.q new file mode 100644 index 0000000000000..8ee4b277cbf72 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/compare_string_bigint.q new file mode 100644 index 0000000000000..810f65d4d2b44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_insert_outputformat.q new file mode 100644 index 0000000000000..a052663055ef6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view1.q new file mode 100644 index 0000000000000..c332278b84f68 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view2.q new file mode 100644 index 0000000000000..b53dd07ce8ae6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view3.q new file mode 100644 index 0000000000000..d6fa7785dfa95 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view4.q new file mode 100644 index 0000000000000..12b6059b9e3ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view5.q new file mode 100644 index 0000000000000..4eb9c94896d8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view6.q new file mode 100644 index 0000000000000..a2f916fb26528 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view7.q new file mode 100644 index 0000000000000..765a96572a04f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_or_replace_view8.q new file mode 100644 index 0000000000000..f3a59b1d07be7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_col_name_value_no_mismatch.q new file mode 100644 index 0000000000000..1d6574e739602 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_dup_col_name.q new file mode 100644 index 0000000000000..726f6dd1dfcf6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_skewed_table_failure_invalid_col_name.q new file mode 100644 index 0000000000000..30dd4181653d5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure1.q new file mode 100644 index 0000000000000..e87c12b8a1fe6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure2.q new file mode 100644 index 0000000000000..0bddae066450e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure3.q new file mode 100644 index 0000000000000..9f9f5f64dfd94 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_failure4.q new file mode 100644 index 0000000000000..67745e011141a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_table_wrong_regex.q new file mode 100644 index 0000000000000..dc91c9c9ef05e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_udaf_failure.q new file mode 100644 index 0000000000000..e0bb408a64f2f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_genericudf.q new file mode 100644 index 0000000000000..07010c11c7d40 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_unknown_udf_udaf.q new file mode 100644 index 0000000000000..a243fff033c4d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure1.q new file mode 100644 index 0000000000000..c9060c6766491 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure2.q new file mode 100644 index 0000000000000..6fdcd4a9d377d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure3.q new file mode 100644 index 0000000000000..ad5fc499edf94 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure4.q new file mode 100644 index 0000000000000..eecde65e1137e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure5.q new file mode 100644 index 0000000000000..f720899168735 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure6.q new file mode 100644 index 0000000000000..57f52a8af149d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure7.q new file mode 100644 index 0000000000000..00d7f9fbf4eda --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure8.q new file mode 100644 index 0000000000000..08291826d978b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/create_view_failure9.q new file mode 100644 index 0000000000000..d7d44a49c393a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ctas.q new file mode 100644 index 0000000000000..507a7a76b1ee7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_already_exists.q new file mode 100644 index 0000000000000..3af7607739a54 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_create_invalid_name.q new file mode 100644 index 0000000000000..5d6749542b470 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_does_not_exist.q new file mode 100644 index 0000000000000..66a940e63dea6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty.q new file mode 100644 index 0000000000000..ae5a443f10627 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_drop_not_empty_restrict.q new file mode 100644 index 0000000000000..e1cb81c93f27b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/database_switch_does_not_exist.q new file mode 100644 index 0000000000000..5cd469769e0aa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal1.q new file mode 100644 index 0000000000000..b7fac0d3df346 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal2.q new file mode 100644 index 0000000000000..711dc9e0fd357 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/date_literal3.q new file mode 100644 index 0000000000000..9483509b6bb79 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ddltime.q new file mode 100644 index 0000000000000..3517a6046de14 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision.q new file mode 100644 index 0000000000000..f49649837e214 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/decimal_precision_1.q new file mode 100644 index 0000000000000..036ff1facc0a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/default_partition_name.q new file mode 100644 index 0000000000000..816b6cb80a964 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/deletejar.q new file mode 100644 index 0000000000000..7b0c92311a11a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q new file mode 100644 index 0000000000000..f7304b12e65f6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure1.q @@ -0,0 +1 @@ +DESC NonExistentTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure2.q new file mode 100644 index 0000000000000..f28b610466499 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/desc_failure3.q new file mode 100644 index 0000000000000..bee0ea5788b4e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q new file mode 100644 index 0000000000000..ea72f83e1d585 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath1.q @@ -0,0 +1 @@ +describe src_thrift.$elem$; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q new file mode 100644 index 0000000000000..f1fee1ac444de --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath2.q @@ -0,0 +1 @@ +describe src_thrift.$key$; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q new file mode 100644 index 0000000000000..4a11f6845f396 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath3.q @@ -0,0 +1 @@ +describe src_thrift.lint.abc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q new file mode 100644 index 0000000000000..0912bf1cd9ddd --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/describe_xpath4.q @@ -0,0 +1 @@ +describe src_thrift.mStringString.abc; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on1.q new file mode 100644 index 0000000000000..d0d748cf4ffd7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/disallow_incompatible_type_change_on2.q new file mode 100644 index 0000000000000..4460c3edd7e4b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_function_failure.q new file mode 100644 index 0000000000000..51dc5e9d8e32c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_index_failure.q new file mode 100644 index 0000000000000..6e907dfa99b24 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q new file mode 100644 index 0000000000000..ae047bbc1780f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_native_udf.q @@ -0,0 +1 @@ +DROP TEMPORARY FUNCTION max; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_failure.q new file mode 100644 index 0000000000000..c2074f69cbf36 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure.q new file mode 100644 index 0000000000000..df476ed7c4634 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_partition_filter_failure2.q new file mode 100644 index 0000000000000..4d238d73a9116 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure1.q new file mode 100644 index 0000000000000..d47c08b876fca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure2.q new file mode 100644 index 0000000000000..631e4ffba7a42 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_table_failure3.q new file mode 100644 index 0000000000000..534ce0b0324af --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure1.q new file mode 100644 index 0000000000000..79cb4e445b05c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/drop_view_failure2.q new file mode 100644 index 0000000000000..93bb16232d57d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform.q new file mode 100644 index 0000000000000..b2e8567f09e16 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_alias_in_transform_schema.q new file mode 100644 index 0000000000000..dabbc351bc386 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert1.q new file mode 100644 index 0000000000000..fcbc7d5444a4d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert2.q new file mode 100644 index 0000000000000..4f79a0352f21c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/duplicate_insert3.q new file mode 100644 index 0000000000000..7b271a56d1844 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part1.q new file mode 100644 index 0000000000000..9f0b6c7a0cc88 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part2.q new file mode 100644 index 0000000000000..00a92783c0548 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part3.q new file mode 100644 index 0000000000000..7a8c58a6b255b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part4.q new file mode 100644 index 0000000000000..9aff7aa6310d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_empty.q.disabled new file mode 100644 index 0000000000000..a8fce595005d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max.q new file mode 100644 index 0000000000000..6a7a6255b959b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dyn_part_max_per_node.q new file mode 100644 index 0000000000000..a411ec520b6d0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/dynamic_partitions_with_whitelist.q new file mode 100644 index 0000000000000..0be2e71c94dff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_incomplete_partition.q new file mode 100644 index 0000000000000..e4f0daca92bd1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists.q new file mode 100644 index 0000000000000..4d1e0a62a431f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists2.q new file mode 100644 index 0000000000000..23777db3ea93d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_exists3.q new file mode 100644 index 0000000000000..350bf248acc9c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_partition_missing.q new file mode 100644 index 0000000000000..81944b3330853 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing.q new file mode 100644 index 0000000000000..7b926a3a8a51c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_table_missing2.q new file mode 100644 index 0000000000000..48fcd74a6f22d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exchange_partition_neg_test.q new file mode 100644 index 0000000000000..23e86e96ca4bc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_00_unsupported_schema.q new file mode 100644 index 0000000000000..d86ecd5785d02 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_01_nonpart_over_loaded.q new file mode 100644 index 0000000000000..5f3223152f766 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_02_all_part_over_overlap.q new file mode 100644 index 0000000000000..d7204dc478d25 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_03_nonpart_noncompat_colschema.q new file mode 100644 index 0000000000000..6cd7eda455ee6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_04_nonpart_noncompat_colnumber.q new file mode 100644 index 0000000000000..7f3f577c433b3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_05_nonpart_noncompat_coltype.q new file mode 100644 index 0000000000000..d3ec9fff82bf9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_06_nonpart_noncompat_storage.q new file mode 100644 index 0000000000000..1cc691fc2912d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_07_nonpart_noncompat_ifof.q new file mode 100644 index 0000000000000..27830ad5f93dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_08_nonpart_noncompat_serde.q new file mode 100644 index 0000000000000..d85048a97a876 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_09_nonpart_noncompat_serdeparam.q new file mode 100644 index 0000000000000..84b3786a161c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_10_nonpart_noncompat_bucketing.q new file mode 100644 index 0000000000000..eaf9c579d51d4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_11_nonpart_noncompat_sorting.q new file mode 100644 index 0000000000000..092fd779541c3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_12_nonnative_export.q new file mode 100644 index 0000000000000..289bcf001fded --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_13_nonnative_import.q new file mode 100644 index 0000000000000..05de3d77b07b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_14_nonpart_part.q new file mode 100644 index 0000000000000..dc194ca814ee4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_15_part_nonpart.q new file mode 100644 index 0000000000000..e233707cc4db2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_16_part_noncompat_schema.q new file mode 100644 index 0000000000000..a10788e3e3b6b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_17_part_spec_underspec.q new file mode 100644 index 0000000000000..cc4a56ca34b88 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_18_part_spec_missing.q new file mode 100644 index 0000000000000..140e3bb3b1bbc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_19_external_over_existing.q new file mode 100644 index 0000000000000..048befe4d3d43 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_20_managed_location_over_existing.q new file mode 100644 index 0000000000000..89cbb9ecd8086 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_21_part_managed_external.q new file mode 100644 index 0000000000000..0cbfc85258d2c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_22_export_authfail.q new file mode 100644 index 0000000000000..d9ab0cf0e4e50 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_23_import_exist_authfail.q new file mode 100644 index 0000000000000..2dbd534074fa8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_24_import_part_authfail.q new file mode 100644 index 0000000000000..ccbcee3698dae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/exim_25_import_nonexist_authfail.q new file mode 100644 index 0000000000000..50bfe005c4278 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external1.q new file mode 100644 index 0000000000000..d56c955050bc5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/external2.q new file mode 100644 index 0000000000000..0df85a09afdd5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fetchtask_ioexception.q new file mode 100644 index 0000000000000..9f44f225e955a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_bad_class.q new file mode 100644 index 0000000000000..33dd4fa614f05 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_input.q new file mode 100644 index 0000000000000..c514562b24160 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fileformat_void_output.q new file mode 100644 index 0000000000000..a9cef1eada16a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name1.q new file mode 100644 index 0000000000000..f50369b138573 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/fs_default_name2.q new file mode 100644 index 0000000000000..485c3db06823f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q new file mode 100644 index 0000000000000..bd633b9760ab6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/genericFileFormat.q @@ -0,0 +1 @@ +create table testFail (a int) stored as foo; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_map_skew_multi_distinct.q new file mode 100644 index 0000000000000..cecd9c6bd807c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby2_multi_distinct.q new file mode 100644 index 0000000000000..e3b0066112c5c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_map_skew_multi_distinct.q new file mode 100644 index 0000000000000..168aeb1261b33 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby3_multi_distinct.q new file mode 100644 index 0000000000000..1a28477918c8e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube1.q new file mode 100644 index 0000000000000..a0bc177ad6351 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_cube2.q new file mode 100644 index 0000000000000..f8ecb6a2d4347 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_id1.q new file mode 100644 index 0000000000000..ac5b6f7b03056 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets1.q new file mode 100644 index 0000000000000..ec6b16bfb28c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets2.q new file mode 100644 index 0000000000000..c988e04e74fa2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets3.q new file mode 100644 index 0000000000000..3e73552422956 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets4.q new file mode 100644 index 0000000000000..cf6352c47d7eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets5.q new file mode 100644 index 0000000000000..7df3318a644c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets6.q new file mode 100644 index 0000000000000..2783047698e78 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_grouping_sets7.q new file mode 100644 index 0000000000000..6c9d5133ad7ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_invalid_position.q new file mode 100644 index 0000000000000..173a752e351a8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_key.q new file mode 100644 index 0000000000000..20970152c33cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup1.q new file mode 100644 index 0000000000000..6366744276077 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/groupby_rollup2.q new file mode 100644 index 0000000000000..aa19b523e9d92 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/having1.q new file mode 100644 index 0000000000000..71f4fd13a0a0e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type.q new file mode 100644 index 0000000000000..1cdaffd1f31a3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/illegal_partition_type2.q new file mode 100644 index 0000000000000..2438288209896 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_bitmap_no_map_aggr.q new file mode 100644 index 0000000000000..a17cd1fec5366 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_entry_limit.q new file mode 100644 index 0000000000000..7d003e3e4b440 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/index_compact_size_limit.q new file mode 100644 index 0000000000000..d79674539a1b2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q new file mode 100644 index 0000000000000..92a6791acb652 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input1.q @@ -0,0 +1 @@ +SELECT a.* FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q new file mode 100644 index 0000000000000..0fe907d9d8ae7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input2.q @@ -0,0 +1 @@ +SELECT a.key FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input4.q new file mode 100644 index 0000000000000..60aea3208c4ed --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input41.q new file mode 100644 index 0000000000000..872ab1014874d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/input_part0_neg.q new file mode 100644 index 0000000000000..4656693d4838c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into1.q new file mode 100644 index 0000000000000..8c197670211bf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into2.q new file mode 100644 index 0000000000000..73a3b6ff13705 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into3.q new file mode 100644 index 0000000000000..4d048b337ec45 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_into4.q new file mode 100644 index 0000000000000..b8944e742b4da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insert_view_failure.q new file mode 100644 index 0000000000000..1f5e13906259a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertexternal1.q new file mode 100644 index 0000000000000..01ebae1022324 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/insertover_dynapart_ifnotexists.q new file mode 100644 index 0000000000000..cbf65c4ac69fe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_arithmetic_type.q new file mode 100644 index 0000000000000..ad37cff79b587 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q new file mode 100644 index 0000000000000..d5b58e0765536 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_avg_syntax.q @@ -0,0 +1 @@ +SELECT avg(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_1.q new file mode 100644 index 0000000000000..73e4729aa0fc1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_2.q new file mode 100644 index 0000000000000..50ec48152548c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_3.q new file mode 100644 index 0000000000000..16f56ec5d3403 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_4.q new file mode 100644 index 0000000000000..bd222f14b469a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_5.q new file mode 100644 index 0000000000000..594fd2bb6f625 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_from_binary_6.q new file mode 100644 index 0000000000000..40ff801460ef8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_1.q new file mode 100644 index 0000000000000..00cd98ed13b7d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_2.q new file mode 100644 index 0000000000000..f31344f835bb2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_3.q new file mode 100644 index 0000000000000..af23d29f4e984 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_4.q new file mode 100644 index 0000000000000..91abe1e6b8a27 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_5.q new file mode 100644 index 0000000000000..afd99be9765a3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_cast_to_binary_6.q new file mode 100644 index 0000000000000..c2143c5c9e955 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_columns.q new file mode 100644 index 0000000000000..14b3409cb4cc4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config1.q new file mode 100644 index 0000000000000..c49ac8a69086e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_config2.q new file mode 100644 index 0000000000000..fa023c8c4b5f2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl1.q new file mode 100644 index 0000000000000..2e1ea6b005618 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q new file mode 100644 index 0000000000000..408919ee2d633 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_create_tbl2.q @@ -0,0 +1 @@ +create tabl tmp_zshao_22 (id int, name strin; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_mapjoin1.q new file mode 100644 index 0000000000000..56d9211d28ebb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q new file mode 100644 index 0000000000000..20033734090f3 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_max_syntax.q @@ -0,0 +1 @@ +SELECT max(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q new file mode 100644 index 0000000000000..584283a08a9e7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_min_syntax.q @@ -0,0 +1 @@ +SELECT min(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column.q new file mode 100644 index 0000000000000..106ba42213197 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_subquery.q new file mode 100644 index 0000000000000..bc70dbca20772 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_column_with_tablename.q new file mode 100644 index 0000000000000..b821e6129a7bf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q new file mode 100644 index 0000000000000..01617f9363b58 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_select_expression.q @@ -0,0 +1 @@ +select foo from a a where foo > .foo; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q new file mode 100644 index 0000000000000..13104198a6db0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_std_syntax.q @@ -0,0 +1 @@ +SELECT std(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q new file mode 100644 index 0000000000000..c6a12526559e0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_stddev_samp_syntax.q @@ -0,0 +1 @@ +SELECT stddev_samp(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q new file mode 100644 index 0000000000000..2d591baa24eb7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_sum_syntax.q @@ -0,0 +1 @@ +SELECT sum(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter1.q new file mode 100644 index 0000000000000..bb19cff8a93ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_alter2.q new file mode 100644 index 0000000000000..aa01b358727b1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q new file mode 100644 index 0000000000000..978f4244a6ba0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_create2.q @@ -0,0 +1 @@ +CREATE TABLE datetime_test (d DATETIME); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q new file mode 100644 index 0000000000000..dfc4864acf43f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_t_transform.q @@ -0,0 +1 @@ +SELECT TRANSFORM(*) USING 'cat' AS (key DATETIME) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q new file mode 100644 index 0000000000000..09394e71ada96 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_tbl_name.q @@ -0,0 +1 @@ +create table invalid-name(a int, b string); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q new file mode 100644 index 0000000000000..ce2a8c4769111 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_var_samp_syntax.q @@ -0,0 +1 @@ +SELECT var_samp(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_1.q new file mode 100644 index 0000000000000..43de018c9f14c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_2.q new file mode 100644 index 0000000000000..3c199d31e7ffc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_varchar_length_3.q new file mode 100644 index 0000000000000..fed04764a9440 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q new file mode 100644 index 0000000000000..5b478299317aa --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalid_variance_syntax.q @@ -0,0 +1 @@ +SELECT variance(*) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/invalidate_view1.q new file mode 100644 index 0000000000000..dd39c5eb4a4fe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join2.q new file mode 100644 index 0000000000000..98a5f1e6629c7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join28.q new file mode 100644 index 0000000000000..32ff105c2e450 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join29.q new file mode 100644 index 0000000000000..53a1652d25b20 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join32.q new file mode 100644 index 0000000000000..54a4dcd9afe2a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join35.q new file mode 100644 index 0000000000000..fc8f77ca1232c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/join_nonexistent_part.q new file mode 100644 index 0000000000000..b4a4757d22147 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/joinneg.q new file mode 100644 index 0000000000000..a4967fd5dfb4a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_alias.q new file mode 100644 index 0000000000000..50d535e6e1ecf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lateral_view_join.q new file mode 100644 index 0000000000000..818754ecbf05b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/line_terminator.q new file mode 100644 index 0000000000000..ad3542c40ace4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_exist_part_authfail.q new file mode 100644 index 0000000000000..f86cd92d9d4c7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_non_native.q new file mode 100644 index 0000000000000..387aaed9a1e50 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_nonpart_authfail.q new file mode 100644 index 0000000000000..d807c698777f1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_authfail.q new file mode 100644 index 0000000000000..c409d5a94a9ce --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_part_nospec.q new file mode 100644 index 0000000000000..356c16a664386 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_stored_as_dirs.q new file mode 100644 index 0000000000000..eed5651cbf20b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_view_failure.q new file mode 100644 index 0000000000000..927f02e82bf91 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat.q new file mode 100644 index 0000000000000..16feeca22649a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_rc_seq.q new file mode 100644 index 0000000000000..7e589fbfde620 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_fileformat_txt_seq.q new file mode 100644 index 0000000000000..ff5ed4e2e3107 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/load_wrong_noof_part.q new file mode 100644 index 0000000000000..ffb64ed643b14 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/local_mapred_error_cache.q new file mode 100644 index 0000000000000..8f4b37a9d49f6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg1.q new file mode 100644 index 0000000000000..e1b58fca80af3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg2.q new file mode 100644 index 0000000000000..a4604cd470658 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg3.q new file mode 100644 index 0000000000000..f2252f7bdf4da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg4.q new file mode 100644 index 0000000000000..b47644cca362c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/lockneg5.q new file mode 100644 index 0000000000000..19c1ce28c2422 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/macro_unused_parameter.q new file mode 100644 index 0000000000000..523710ddf3a5b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace.q new file mode 100644 index 0000000000000..76c7ae94d4b63 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_hadoop20.q new file mode 100644 index 0000000000000..9d0548cc10f55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff.q new file mode 100644 index 0000000000000..c93aedb3137bf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/mapreduce_stack_trace_turnoff_hadoop20.q new file mode 100644 index 0000000000000..e319944958c2a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_1.q new file mode 100644 index 0000000000000..0a48c01546ec5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_2.q new file mode 100644 index 0000000000000..a4fab1c8b804d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/merge_negative_3.q new file mode 100644 index 0000000000000..6bc645e4c2378 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/minimr_broken_pipe.q new file mode 100644 index 0000000000000..8dda9cdf4a37d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nested_complex_neg.q new file mode 100644 index 0000000000000..ac6c4ee549d85 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q new file mode 100644 index 0000000000000..0c24b1626a532 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/no_matching_udf.q @@ -0,0 +1 @@ +SELECT percentile(3.5, 0.99) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q new file mode 100644 index 0000000000000..431e04efd9342 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nonkey_groupby.q @@ -0,0 +1 @@ +EXPLAIN SELECT key, count(1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_insert.q new file mode 100644 index 0000000000000..4841f9e11c84e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/nopart_load.q new file mode 100644 index 0000000000000..6e5ad6eb41a8b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias3.q new file mode 100644 index 0000000000000..6cc3e87288a89 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/notable_alias4.q new file mode 100644 index 0000000000000..e7ad6b79d3ed9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_invalid_position.q new file mode 100644 index 0000000000000..4dbf2a6d56a2e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderby_position_unsupported.q new file mode 100644 index 0000000000000..a490c2306ec4a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/orderbysortby.q new file mode 100644 index 0000000000000..5dff69fdbb78d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/part_col_complex_type.q new file mode 100644 index 0000000000000..4b9eb847db542 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part.q new file mode 100644 index 0000000000000..541599915afc0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part1.q new file mode 100644 index 0000000000000..99256da285c1a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part2.q new file mode 100644 index 0000000000000..72b55ea25d8e9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_part_no_drop.q new file mode 100644 index 0000000000000..b4e508ff98180 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl1.q new file mode 100644 index 0000000000000..236129902c07c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl2.q new file mode 100644 index 0000000000000..05964c35e9e08 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl3.q new file mode 100644 index 0000000000000..bbaa2670875b6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl4.q new file mode 100644 index 0000000000000..c7880de6d8aec --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl5.q new file mode 100644 index 0000000000000..cd848fd4a1b9f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl6.q new file mode 100644 index 0000000000000..26248cc6b4877 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl7.q new file mode 100644 index 0000000000000..afff8404edc01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl8.q new file mode 100644 index 0000000000000..809c287fc502a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/protectmode_tbl_no_drop.q new file mode 100644 index 0000000000000..a4ef2acbfd406 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AggrFuncsWithNoGBYNoPartDef.q new file mode 100644 index 0000000000000..542367ace22e0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_AmbiguousWindowDefn.q new file mode 100644 index 0000000000000..95b35113e3cf0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DistributeByOrderBy.q new file mode 100644 index 0000000000000..caebebf8eaa4c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_DuplicateWindowAlias.q new file mode 100644 index 0000000000000..3a0304188d2ac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithNoGBYNoWindowing.q new file mode 100644 index 0000000000000..f351a1448b150 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_HavingLeadWithPTF.q new file mode 100644 index 0000000000000..d0d3d3fae23f4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_InvalidValueBoundary.q new file mode 100644 index 0000000000000..40a39cb68b5e3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_JoinWithAmbigousAlias.q new file mode 100644 index 0000000000000..80441e4f571f8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_PartitionBySortBy.q new file mode 100644 index 0000000000000..1c98b8743cd72 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_negative_WhereWithRankCond.q new file mode 100644 index 0000000000000..8f4a21bd6c966 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries.q new file mode 100644 index 0000000000000..ddab4367bb66b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/ptf_window_boundaries2.q new file mode 100644 index 0000000000000..16cb52ca8414e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/recursive_view.q new file mode 100644 index 0000000000000..590523e9b6259 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q new file mode 100644 index 0000000000000..8333ddc948419 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_1.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT `+++` FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q new file mode 100644 index 0000000000000..d1aa1f1a9542d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_2.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT `.a.` FROM srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q new file mode 100644 index 0000000000000..53971916e6c96 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/regex_col_groupby.q @@ -0,0 +1,2 @@ +EXPLAIN +SELECT `..`, count(1) FROM srcpart GROUP BY `..`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sa_fail_hook3.q new file mode 100644 index 0000000000000..e54201c09e6f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sample.q new file mode 100644 index 0000000000000..0086352f8c47c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe1.q new file mode 100644 index 0000000000000..6b1c09decfb35 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe2.q new file mode 100644 index 0000000000000..1c3093c0e7023 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_broken_pipe3.q new file mode 100644 index 0000000000000..60f93d209802a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/script_error.q new file mode 100644 index 0000000000000..e46aed03b147f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_charliteral.q new file mode 100644 index 0000000000000..1e4c70e663f05 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/select_udtf_alias.q new file mode 100644 index 0000000000000..8ace4414fc14e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin1.q new file mode 100644 index 0000000000000..06e6cad34b4d8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin2.q new file mode 100644 index 0000000000000..46faae641640d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin3.q new file mode 100644 index 0000000000000..35b455a7292d1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/semijoin4.q new file mode 100644 index 0000000000000..4e52ebfb3cdeb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex.q new file mode 100644 index 0000000000000..13b3f165b9684 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex2.q new file mode 100644 index 0000000000000..a3955744221a4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/serde_regex3.q new file mode 100644 index 0000000000000..5a0295c971c26 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation0.q new file mode 100644 index 0000000000000..4cb48664b602f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_hiveconf_validation1.q new file mode 100644 index 0000000000000..330aafd19858c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/set_table_property.q new file mode 100644 index 0000000000000..d582aaeb386c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q new file mode 100644 index 0000000000000..25705dc3d5275 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns1.q @@ -0,0 +1,2 @@ +SHOW COLUMNS from shcol_test; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q new file mode 100644 index 0000000000000..c55b449a0b5ff --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns2.q @@ -0,0 +1,2 @@ +SHOW COLUMNS from shcol_test foo; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_columns3.q new file mode 100644 index 0000000000000..508a786609d8b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q new file mode 100644 index 0000000000000..83e5093aa1f21 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_does_not_exist.q @@ -0,0 +1,2 @@ +SHOW CREATE TABLE tmp_nonexist; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_create_table_index.q new file mode 100644 index 0000000000000..0dd0ef9a255bf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q new file mode 100644 index 0000000000000..71f68c894f2a0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_partitions1.q @@ -0,0 +1 @@ +SHOW PARTITIONS NonExistentTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q new file mode 100644 index 0000000000000..254a1d3a5ac3c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tableproperties1.q @@ -0,0 +1 @@ +SHOW TBLPROPERTIES NonExistentTable; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q new file mode 100644 index 0000000000000..1bc94d6392c67 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad1.q @@ -0,0 +1 @@ +SHOW TABLES JOIN; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q new file mode 100644 index 0000000000000..5e828b647ac38 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad2.q @@ -0,0 +1 @@ +SHOW TABLES FROM default LIKE a b; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db1.q new file mode 100644 index 0000000000000..d0141f6c291cb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tables_bad_db2.q new file mode 100644 index 0000000000000..ee0deba87a948 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus.q new file mode 100644 index 0000000000000..283b5836e27fa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/show_tablestatus_not_existing_part.q new file mode 100644 index 0000000000000..242e165285547 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_bucketmapjoin.q new file mode 100644 index 0000000000000..880323c604b66 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/smb_mapjoin_14.q new file mode 100644 index 0000000000000..54bfba03d82d7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/sortmerge_mapjoin_mismatch_1.q new file mode 100644 index 0000000000000..7d11f450edfde --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_out_of_range.q new file mode 100644 index 0000000000000..66af1fd7da682 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format.q new file mode 100644 index 0000000000000..f71cc4487910a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/split_sample_wrong_format2.q new file mode 100644 index 0000000000000..1a13c0ff4cb28 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_1.q new file mode 100644 index 0000000000000..401cc37f67dd0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_aggregator_error_2.q new file mode 100644 index 0000000000000..c7e63591adde0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_noscan_non_native.q new file mode 100644 index 0000000000000..bde66278360c5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_autogether.q new file mode 100644 index 0000000000000..47a8148e0869c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_external.q new file mode 100644 index 0000000000000..c206b8b5d765e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partialscan_non_native.q new file mode 100644 index 0000000000000..8e02ced85e708 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_partscan_norcfile.q new file mode 100644 index 0000000000000..56d93d08aa697 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_1.q new file mode 100644 index 0000000000000..7fa0f55f2a450 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/stats_publisher_error_2.q new file mode 100644 index 0000000000000..f82d4b54b697b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_join.q new file mode 100644 index 0000000000000..d618ee28fdb2e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_orderby.q new file mode 100644 index 0000000000000..781cdbb05088b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/strict_pruning.q new file mode 100644 index 0000000000000..270ab2f593ac2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/subq_insert.q new file mode 100644 index 0000000000000..0bc9e24e4828c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q new file mode 100644 index 0000000000000..9efbba0082b6f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch1.q @@ -0,0 +1 @@ +ALTER TABLE srcpart TOUCH PARTITION (ds='2008-04-08', hr='13'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q new file mode 100644 index 0000000000000..923a171e0482e --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/touch2.q @@ -0,0 +1 @@ +ALTER TABLE src TOUCH PARTITION (ds='2008-04-08', hr='12'); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_bucketed_column.q new file mode 100644 index 0000000000000..e53665695a39d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_indexed_table.q new file mode 100644 index 0000000000000..13f32c8968a16 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_list_bucketing.q new file mode 100644 index 0000000000000..0ece6007f7b61 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_column_seqfile.q new file mode 100644 index 0000000000000..903540dae8985 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_nonexistant_column.q new file mode 100644 index 0000000000000..5509552811b09 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column.q new file mode 100644 index 0000000000000..134743ac13a5d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_partition_column2.q new file mode 100644 index 0000000000000..47635208a7810 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure1.q new file mode 100644 index 0000000000000..f6cfa44bbb126 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure2.q new file mode 100644 index 0000000000000..1137d893eb0e1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure3.q new file mode 100644 index 0000000000000..c5cf58775b309 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/truncate_table_failure4.q new file mode 100644 index 0000000000000..a7f1e92d55980 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q new file mode 100644 index 0000000000000..f37ce72ae4197 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udaf_invalid_place.q @@ -0,0 +1 @@ +select distinct key, sum(key) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong1.q new file mode 100644 index 0000000000000..c2a132d4db05e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_array_contains_wrong2.q new file mode 100644 index 0000000000000..36f85d34a6e0b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true.q new file mode 100644 index 0000000000000..73b3f9654f1c3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_assert_true2.q new file mode 100644 index 0000000000000..4b62220764bb9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong.q new file mode 100644 index 0000000000000..2fb5ff74cc3ce --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong2.q new file mode 100644 index 0000000000000..5772dc1a95c9d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_case_type_wrong3.q new file mode 100644 index 0000000000000..5aaf0188eb9c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_coalesce.q new file mode 100644 index 0000000000000..7405e387caf70 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong1.q new file mode 100644 index 0000000000000..8c2017bc636cf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong2.q new file mode 100644 index 0000000000000..c49e7868bbb5e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_concat_ws_wrong3.q new file mode 100644 index 0000000000000..72b86271f5ea6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q new file mode 100644 index 0000000000000..fbe4902d644cc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_args_len.q @@ -0,0 +1 @@ +SELECT elt(3) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_elt_wrong_type.q new file mode 100644 index 0000000000000..bb1fdbf789e3e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q new file mode 100644 index 0000000000000..9703c82d8a4d9 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_args_len.q @@ -0,0 +1 @@ +SELECT field(3) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_field_wrong_type.q new file mode 100644 index 0000000000000..61b2cd06496e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong1.q new file mode 100644 index 0000000000000..18c985c606849 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong2.q new file mode 100644 index 0000000000000..7959c20b28e56 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong3.q new file mode 100644 index 0000000000000..7d90ef86da7b7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong4.q new file mode 100644 index 0000000000000..e545f4aa1420c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong5.q new file mode 100644 index 0000000000000..a6f71778f143b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong6.q new file mode 100644 index 0000000000000..e5b11b9b71ee7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_format_number_wrong7.q new file mode 100644 index 0000000000000..aa4a3a44751c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_function_does_not_implement_udf.q new file mode 100644 index 0000000000000..21ca6e7d36255 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q new file mode 100644 index 0000000000000..74458d0c3db2c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_not_bool.q @@ -0,0 +1 @@ +SELECT IF('STRING', 1, 1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q new file mode 100644 index 0000000000000..ad19364c33076 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_if_wrong_args_len.q @@ -0,0 +1 @@ +SELECT IF(TRUE, 1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_in.q new file mode 100644 index 0000000000000..ce9ce54fac680 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q new file mode 100644 index 0000000000000..ac8253fb1e941 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_args_len.q @@ -0,0 +1 @@ +SELECT instr('abcd') FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_instr_wrong_type.q new file mode 100644 index 0000000000000..9ac3ed6614897 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_args_len.q new file mode 100644 index 0000000000000..ca7caad54d646 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_locate_wrong_type.q new file mode 100644 index 0000000000000..4bbf79a310b01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_num.q new file mode 100644 index 0000000000000..ebb6c2ab418eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_keys_arg_type.q new file mode 100644 index 0000000000000..0757d1494f3cd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_num.q new file mode 100644 index 0000000000000..c97476a1263ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_map_values_arg_type.q new file mode 100644 index 0000000000000..cc060ea0f0ec7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q new file mode 100644 index 0000000000000..7282e07596032 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_max.q @@ -0,0 +1,2 @@ +SELECT max(map("key", key, "value", value)) +FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q new file mode 100644 index 0000000000000..b9528fa6dafef --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_min.q @@ -0,0 +1,2 @@ +SELECT min(map("key", key, "value", value)) +FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong1.q new file mode 100644 index 0000000000000..88ca4fefc3055 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong2.q new file mode 100644 index 0000000000000..01ed2ffcf017f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong3.q new file mode 100644 index 0000000000000..71f118b8dc0dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_printf_wrong4.q new file mode 100644 index 0000000000000..71f118b8dc0dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_reflect_neg.q new file mode 100644 index 0000000000000..67efb64505d9e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_args_len.q new file mode 100644 index 0000000000000..c628ff8aa1972 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q new file mode 100644 index 0000000000000..16695f6adc3f4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_size_wrong_type.q @@ -0,0 +1 @@ +SELECT SIZE('wrong type: string') FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong1.q new file mode 100644 index 0000000000000..9954f4ab4d3c5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong2.q new file mode 100644 index 0000000000000..32c2645519497 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_sort_array_wrong3.q new file mode 100644 index 0000000000000..034de06b8e391 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error.q new file mode 100644 index 0000000000000..846f87c2e51b1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_test_error_reduce.q new file mode 100644 index 0000000000000..b1a06f2a07af3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong.q new file mode 100644 index 0000000000000..d4d2d2e48517f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong2.q new file mode 100644 index 0000000000000..79fa65f63da56 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udf_when_type_wrong3.q new file mode 100644 index 0000000000000..8bb5fdd7ea377 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udfnull.q new file mode 100644 index 0000000000000..3c4204f780004 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported1.q new file mode 100644 index 0000000000000..942ae5d8315f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported2.q new file mode 100644 index 0000000000000..00d359a75ce0c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported3.q new file mode 100644 index 0000000000000..51df8fa862e1a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_explode_not_supported4.q new file mode 100644 index 0000000000000..ae8dff7bad8d8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q new file mode 100644 index 0000000000000..ab84a801e9ed6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_invalid_place.q @@ -0,0 +1 @@ +select distinct key, explode(key) from src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported1.q new file mode 100644 index 0000000000000..04e98d52c548a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/udtf_not_supported3.q new file mode 100644 index 0000000000000..f4fe0dde3e62c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union.q new file mode 100644 index 0000000000000..e3c5c830897ea --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union2.q new file mode 100644 index 0000000000000..38db488eaf68e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union22.q new file mode 100644 index 0000000000000..72f3314bdac96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/union3.q new file mode 100644 index 0000000000000..ce657478c150d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin.q new file mode 100644 index 0000000000000..d6a19c397d80d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin2.q new file mode 100644 index 0000000000000..6e9a08251407d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/uniquejoin3.q new file mode 100644 index 0000000000000..89a8f1b2aaa84 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_table_property.q new file mode 100644 index 0000000000000..7a24e652b46f2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/unset_view_property.q new file mode 100644 index 0000000000000..11131006e998a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_leadlag_in_udaf.q new file mode 100644 index 0000000000000..b54b7a532176f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/windowing_ll_no_over.q new file mode 100644 index 0000000000000..3ca1104b0158d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientnegative/wrong_column_type.q new file mode 100644 index 0000000000000..490f0c3b4d110 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_part_exist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_part_exist.q new file mode 100644 index 0000000000000..d176661ecb8ea --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_part_multiple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_part_multiple.q new file mode 100644 index 0000000000000..9248a7f3b94b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_no_whitelist.q new file mode 100644 index 0000000000000..f51c53c2ff627 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/add_partition_with_whitelist.q new file mode 100644 index 0000000000000..009c7610ef917 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alias_casted_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alias_casted_column.q new file mode 100644 index 0000000000000..f1fc91a5f2d0c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/allcolref_in_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/allcolref_in_udf.q new file mode 100644 index 0000000000000..020975cc69b94 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter1.q new file mode 100644 index 0000000000000..5fd19455f93cd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter2.q new file mode 100644 index 0000000000000..ddf57873a548c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter3.q new file mode 100644 index 0000000000000..3cde00748b98e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter4.q new file mode 100644 index 0000000000000..542ff019b2595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter5.q new file mode 100644 index 0000000000000..0d14f228d1c30 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_concatenate_indexed_table.q new file mode 100644 index 0000000000000..807ef539c8643 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_index.q new file mode 100644 index 0000000000000..2aa13da46961f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge.q new file mode 100644 index 0000000000000..c3502739d54a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_2.q new file mode 100644 index 0000000000000..65ddfed13dac4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_merge_stats.q new file mode 100644 index 0000000000000..23bae55ab2add --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table.q new file mode 100644 index 0000000000000..b6d1eb8f2d4b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_numbuckets_partitioned_table2.q new file mode 100644 index 0000000000000..c6a4ad24fcdde --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_clusterby_sortby.q new file mode 100644 index 0000000000000..7481c651bfb7a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_coltype.q new file mode 100644 index 0000000000000..5479afbbd5ab0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_format_loc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_format_loc.q new file mode 100644 index 0000000000000..bef6865833960 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_protect_mode.q new file mode 100644 index 0000000000000..7bcb9f071c415 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_partition_with_whitelist.q new file mode 100644 index 0000000000000..301362a881456 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition.q new file mode 100644 index 0000000000000..d498cd52a5ffb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_rename_partition_authorization.q new file mode 100644 index 0000000000000..cdefc2dcbe7b4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_skewed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_skewed_table.q new file mode 100644 index 0000000000000..216bbb58cdf3f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_table_not_sorted.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_table_not_sorted.q new file mode 100644 index 0000000000000..7aff3288cf646 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_table_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_table_serde.q new file mode 100644 index 0000000000000..a80693e7c5153 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_table_serde2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_table_serde2.q new file mode 100644 index 0000000000000..6ca3be3139e45 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar1.q new file mode 100644 index 0000000000000..781cbda735158 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_varchar2.q new file mode 100644 index 0000000000000..5a481e7f8ebfe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_view_as_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_view_as_select.q new file mode 100644 index 0000000000000..dcab3cab7e9ac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_view_rename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/alter_view_rename.q new file mode 100644 index 0000000000000..68cf9d6e0a974 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ambiguous_col.q new file mode 100644 index 0000000000000..e7053c1c8eb55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive.q new file mode 100644 index 0000000000000..a928a81023ff6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_corrupt.q new file mode 100644 index 0000000000000..b83eab5d38137 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_excludeHadoop20.q new file mode 100644 index 0000000000000..ddc06a99bb5bc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_multi.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/archive_multi.q new file mode 100644 index 0000000000000..2c1a6d83a35c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_1.q new file mode 100644 index 0000000000000..25c99187f3c00 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_2.q new file mode 100644 index 0000000000000..4fc79b9ed294a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_3.q new file mode 100644 index 0000000000000..ba76b0002fde8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_4.q new file mode 100644 index 0000000000000..152c8e5cf305a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_5.q new file mode 100644 index 0000000000000..8869edce2f4ea --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_6.q new file mode 100644 index 0000000000000..258ada4988362 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_7.q new file mode 100644 index 0000000000000..ae49000442a1f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/authorization_8.q new file mode 100644 index 0000000000000..67fcf3162dd56 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join0.q new file mode 100644 index 0000000000000..84731dbce4a03 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join1.q new file mode 100644 index 0000000000000..2062a28b46e21 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join10.q new file mode 100644 index 0000000000000..2dbdcf75f3c24 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join11.q new file mode 100644 index 0000000000000..d8045b1f86592 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join12.q new file mode 100644 index 0000000000000..b31e4f7f4d97a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join13.q new file mode 100644 index 0000000000000..c116eba6b7597 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14.q new file mode 100644 index 0000000000000..b282fb9a1aba0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join14_hadoop20.q new file mode 100644 index 0000000000000..235b7c1b3fcd2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join15.q new file mode 100644 index 0000000000000..66b745f62ead9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join16.q new file mode 100644 index 0000000000000..0ade940892f42 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join17.q new file mode 100644 index 0000000000000..4bb804e3e7f38 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join18.q new file mode 100644 index 0000000000000..7a3d6c9a4448b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join18_multi_distinct.q new file mode 100644 index 0000000000000..7465bc9b23bd2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join19.q new file mode 100644 index 0000000000000..32935f1d89cf5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join2.q new file mode 100644 index 0000000000000..f598ccb4da234 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join20.q new file mode 100644 index 0000000000000..e61958414294c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join21.q new file mode 100644 index 0000000000000..e77f922aaada0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join22.q new file mode 100644 index 0000000000000..b5eec90d09de1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join23.q new file mode 100644 index 0000000000000..191d269ba840c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join24.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join24.q new file mode 100644 index 0000000000000..e8b26481024ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join25.q new file mode 100644 index 0000000000000..eaf7489a17025 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join26.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join26.q new file mode 100644 index 0000000000000..16fbfe6cfbe53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join27.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join27.q new file mode 100644 index 0000000000000..2dc5845ed3b82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join28.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join28.q new file mode 100644 index 0000000000000..61b9d84291eda --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join29.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join29.q new file mode 100644 index 0000000000000..c9f9191258231 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join3.q new file mode 100644 index 0000000000000..954765444c4ea --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join30.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join30.q new file mode 100644 index 0000000000000..9e31f0ffe2aeb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join31.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join31.q new file mode 100644 index 0000000000000..cfa5eba17569c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join32.q new file mode 100644 index 0000000000000..289bfbc6da917 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join4.q new file mode 100644 index 0000000000000..1c2ad5f60b3e1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join5.q new file mode 100644 index 0000000000000..945629242329f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join6.q new file mode 100644 index 0000000000000..a0f8e55651db7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join7.q new file mode 100644 index 0000000000000..ca105601d08bc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join8.q new file mode 100644 index 0000000000000..553fc25ba7df4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join9.q new file mode 100644 index 0000000000000..fb3eafb441d70 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_filters.q new file mode 100644 index 0000000000000..458504cdc3d04 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_nulls.q new file mode 100644 index 0000000000000..766348d7c001e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_join_reordering_values.q new file mode 100644 index 0000000000000..46a4a0d5339a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_smb_mapjoin_14.q new file mode 100644 index 0000000000000..5911c49afea7b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_1.q new file mode 100644 index 0000000000000..e76b560f41899 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_10.q new file mode 100644 index 0000000000000..83e1cec536100 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_11.q new file mode 100644 index 0000000000000..f9fa1e4d4e1db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_12.q new file mode 100644 index 0000000000000..db53a656a850a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_13.q new file mode 100644 index 0000000000000..28bbc2d8f1a3e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_14.q new file mode 100644 index 0000000000000..4cda4df62ec35 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_15.q new file mode 100644 index 0000000000000..c7bcae64b38d5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_2.q new file mode 100644 index 0000000000000..75339778af802 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_3.q new file mode 100644 index 0000000000000..39a695fdf039d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_4.q new file mode 100644 index 0000000000000..6072272c478a3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_5.q new file mode 100644 index 0000000000000..a28ce3de5fc63 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_6.q new file mode 100644 index 0000000000000..309987b8f7711 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_7.q new file mode 100644 index 0000000000000..d62f6377218c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_8.q new file mode 100644 index 0000000000000..6302a1be1477c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/auto_sortmerge_join_9.q new file mode 100644 index 0000000000000..ef63dfed5267a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/autogen_colalias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/autogen_colalias.q new file mode 100644 index 0000000000000..e247a733ae60a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_change_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_change_schema.q new file mode 100644 index 0000000000000..08e72c408efde --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_compression_enabled.q new file mode 100644 index 0000000000000..8367206231974 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_evolved_schemas.q new file mode 100644 index 0000000000000..3fe8ff12b4506 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_joins.q new file mode 100644 index 0000000000000..25b77c06270d0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_nullable_fields.q new file mode 100644 index 0000000000000..584c6f740ba91 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_partitioned.q new file mode 100644 index 0000000000000..8e4d40f2bd59b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_sanity_test.q new file mode 100644 index 0000000000000..e3f8b07b30c59 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_schema_error_message.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_schema_error_message.q new file mode 100644 index 0000000000000..cf1fda1d53409 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_schema_literal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/avro_schema_literal.q new file mode 100644 index 0000000000000..d77f31077a40f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table1.q new file mode 100644 index 0000000000000..6587f2e929f99 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table2.q new file mode 100644 index 0000000000000..d3c3e0807f08c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table3.q new file mode 100644 index 0000000000000..8137722965f6c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table_udfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table_udfs.q new file mode 100644 index 0000000000000..7a3d3f7e4180f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ba_table_union.q new file mode 100644 index 0000000000000..9804659d36625 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_constant.q new file mode 100644 index 0000000000000..e0a8b95401d0b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_output_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_output_format.q new file mode 100644 index 0000000000000..c5d9abd062c4f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_bincolserde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_bincolserde.q new file mode 100644 index 0000000000000..47a560c494aae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binary_table_colserde.q new file mode 100644 index 0000000000000..eadf07d14d756 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/binarysortable_1.q new file mode 100644 index 0000000000000..a98a2305cfcaf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket1.q new file mode 100644 index 0000000000000..ca9b9025e270f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket2.q new file mode 100644 index 0000000000000..d4bddf72ed74d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket3.q new file mode 100644 index 0000000000000..d891b90b7b379 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket4.q new file mode 100644 index 0000000000000..2b3f80543a224 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket5.q new file mode 100644 index 0000000000000..877f8a50a0e35 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_groupby.q new file mode 100644 index 0000000000000..a531ba70a7481 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_1.q new file mode 100644 index 0000000000000..33dd5d5cd2e0a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_map_join_2.q new file mode 100644 index 0000000000000..d1097e70a9880 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers.q new file mode 100644 index 0000000000000..37ae6cc7adeae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucket_num_reducers2.q new file mode 100644 index 0000000000000..8c64d60c5ac42 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_1.q new file mode 100644 index 0000000000000..5b013995d5ba4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_2.q new file mode 100644 index 0000000000000..f952f2ee6d445 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_3.q new file mode 100644 index 0000000000000..461fbb196e758 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_4.q new file mode 100644 index 0000000000000..366da4473b36d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_5.q new file mode 100644 index 0000000000000..411fed392a45e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_6.q new file mode 100644 index 0000000000000..204d1e9010d5d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_7.q new file mode 100644 index 0000000000000..b0bca460f303f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketcontext_8.q new file mode 100644 index 0000000000000..9533c5512f3dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat.q new file mode 100644 index 0000000000000..4c7f0c9595488 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketizedhiveinputformat_auto.q new file mode 100644 index 0000000000000..9cdfe8e6e9f6e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin1.q new file mode 100644 index 0000000000000..2bd8e1e2e5d98 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin10.q new file mode 100644 index 0000000000000..cf4222bdd781e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin11.q new file mode 100644 index 0000000000000..e10ab522a296d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin12.q new file mode 100644 index 0000000000000..8139000f4a733 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin13.q new file mode 100644 index 0000000000000..f01c43e5694f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin2.q new file mode 100644 index 0000000000000..fdbadfcb04c9c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin3.q new file mode 100644 index 0000000000000..8fda80231522e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin4.q new file mode 100644 index 0000000000000..c1a8f2ab7bfa0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin5.q new file mode 100644 index 0000000000000..2df49b64f3227 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin6.q new file mode 100644 index 0000000000000..2998d7ccf1961 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin7.q new file mode 100644 index 0000000000000..3a96c642b3e4c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin8.q new file mode 100644 index 0000000000000..5e8daa5f8f125 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin9.q new file mode 100644 index 0000000000000..86344a53f5a95 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative.q new file mode 100644 index 0000000000000..d7634333e2fee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative2.q new file mode 100644 index 0000000000000..901f0567590ec --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketmapjoin_negative3.q new file mode 100644 index 0000000000000..d66e1238bb79b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_1.q new file mode 100644 index 0000000000000..8cc308ffb1156 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_2.q new file mode 100644 index 0000000000000..9ecd2c48ba764 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_3.q new file mode 100644 index 0000000000000..91e97de62c82f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_4.q new file mode 100644 index 0000000000000..623b22bfe494c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_5.q new file mode 100644 index 0000000000000..205a45064b4e3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_6.q new file mode 100644 index 0000000000000..a4e84f85e1664 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_7.q new file mode 100644 index 0000000000000..f597884bb7645 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/bucketsortoptimize_insert_8.q new file mode 100644 index 0000000000000..95a9a6481a4a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/case_sensitivity.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/case_sensitivity.q new file mode 100644 index 0000000000000..a4b73f0a87e95 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast1.q new file mode 100644 index 0000000000000..7f5f8a58deecf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cast_to_int.q new file mode 100644 index 0000000000000..729ffdc8687b5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cluster.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cluster.q new file mode 100644 index 0000000000000..24819707181a8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/column_access_stats.q new file mode 100644 index 0000000000000..3c8a309991d75 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnarserde_create_shortcut.q new file mode 100644 index 0000000000000..4c3b44ae14af6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_partlvl.q new file mode 100644 index 0000000000000..98627776d3163 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/columnstats_tbllvl.q new file mode 100644 index 0000000000000..72d88a67b5212 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine1.q new file mode 100644 index 0000000000000..86abf0996057b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2.q new file mode 100644 index 0000000000000..cfd9856f0868a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_hadoop20.q new file mode 100644 index 0000000000000..8f9a59d497536 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine2_win.q new file mode 100644 index 0000000000000..f6090bb99b29a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/combine3.q new file mode 100644 index 0000000000000..c9afc91bb4561 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_binary.q new file mode 100644 index 0000000000000..c19813602782f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_boolean.q new file mode 100644 index 0000000000000..dc76f7c7d57e1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_double.q new file mode 100644 index 0000000000000..6c6dc47e67701 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_empty_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_empty_table.q new file mode 100644 index 0000000000000..d4ed93fa63987 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_long.q new file mode 100644 index 0000000000000..7d0a1584a28cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/compute_stats_string.q new file mode 100644 index 0000000000000..f146f6b95b544 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/concatenate_inherit_table_location.q new file mode 100644 index 0000000000000..1ae194338ce8c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/constant_prop.q new file mode 100644 index 0000000000000..ced72d60445c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/convert_enum_to_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/convert_enum_to_string.q new file mode 100644 index 0000000000000..3dc4a6ad42523 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer1.q new file mode 100644 index 0000000000000..b3fd3f760eaf0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer10.q new file mode 100644 index 0000000000000..4369125a77a14 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer11.q new file mode 100644 index 0000000000000..412baa77f549f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer12.q new file mode 100644 index 0000000000000..409cdb4c5b38c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer13.q new file mode 100644 index 0000000000000..e979465d28af9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer14.q new file mode 100644 index 0000000000000..1322b793780a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer15.q new file mode 100644 index 0000000000000..dd6143c402c20 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer2.q new file mode 100644 index 0000000000000..af97f16e3629a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer3.q new file mode 100644 index 0000000000000..89b7c15690ab1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer4.q new file mode 100644 index 0000000000000..70fcdfc0b4288 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer5.q new file mode 100644 index 0000000000000..ac836c0bfdc91 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer6.q new file mode 100644 index 0000000000000..89c060919c205 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer7.q new file mode 100644 index 0000000000000..3ba83f489ccc3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer8.q new file mode 100644 index 0000000000000..4e79d80f11bbb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/correlationoptimizer9.q new file mode 100644 index 0000000000000..6169a30c05146 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/count.q new file mode 100644 index 0000000000000..0d66a5ec345e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cp_mj_rc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cp_mj_rc.q new file mode 100644 index 0000000000000..62d38e3caae58 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_1.q new file mode 100644 index 0000000000000..f348e5902263a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_alter_list_bucketing_table1.q new file mode 100644 index 0000000000000..b51b238a2d0e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_big_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_big_view.q new file mode 100644 index 0000000000000..49caacc3e2019 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_default_prop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_default_prop.q new file mode 100644 index 0000000000000..26d7332ecb88c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_escape.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_escape.q new file mode 100644 index 0000000000000..49f7c5fe601aa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_genericudaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_genericudaf.q new file mode 100644 index 0000000000000..b2bc0423e8b2a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_genericudf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_genericudf.q new file mode 100644 index 0000000000000..1d43f30e8e150 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_insert_outputformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_insert_outputformat.q new file mode 100644 index 0000000000000..f18d68866a8c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like.q new file mode 100644 index 0000000000000..cb4d6578af476 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like2.q new file mode 100644 index 0000000000000..146467b2cd2c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like_tbl_props.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like_tbl_props.q new file mode 100644 index 0000000000000..8f951d1dda446 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_like_view.q new file mode 100644 index 0000000000000..f0bf58596256e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_merge_compressed.q new file mode 100644 index 0000000000000..4418b3430affc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_nested_type.q new file mode 100644 index 0000000000000..2debd0d71d224 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_or_replace_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_or_replace_view.q new file mode 100644 index 0000000000000..a8f59b7f05354 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_skewed_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_skewed_table1.q new file mode 100644 index 0000000000000..5d3a48acb4b3a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_struct_table.q new file mode 100644 index 0000000000000..dd5aa63e4578a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_udaf.q new file mode 100644 index 0000000000000..a6bd7a0846da9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_union_table.q new file mode 100644 index 0000000000000..bb0e5b989d54d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view.q new file mode 100644 index 0000000000000..e08274c9e994f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_partitioned.q new file mode 100644 index 0000000000000..5e30f91f4eb78 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/create_view_translate.q new file mode 100644 index 0000000000000..2199750f42f7e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/cross_join.q new file mode 100644 index 0000000000000..1f888dd47b9e9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ct_case_insensitive.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ct_case_insensitive.q new file mode 100644 index 0000000000000..00e9722087414 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas.q new file mode 100644 index 0000000000000..e595904b412fa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_colname.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_colname.q new file mode 100644 index 0000000000000..80b8b4f06487d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_date.q new file mode 100644 index 0000000000000..03af69fd3bddf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_hadoop20.q new file mode 100644 index 0000000000000..4961b971dbdec --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_uses_database_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_uses_database_location.q new file mode 100644 index 0000000000000..86c940a25b291 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_varchar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ctas_varchar.q new file mode 100644 index 0000000000000..f646b36c8a6f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/custom_input_output_format.q new file mode 100644 index 0000000000000..ff5e86dc5a6b4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database.q new file mode 100644 index 0000000000000..9140a42b650e8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_drop.q new file mode 100644 index 0000000000000..4e17c7ad5ac10 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_location.q new file mode 100644 index 0000000000000..e131b5ff02e67 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_properties.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/database_properties.q new file mode 100644 index 0000000000000..7f3b239d1836d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_1.q new file mode 100644 index 0000000000000..a2322fc1e45a0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_2.q new file mode 100644 index 0000000000000..0821e012ba26b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_3.q new file mode 100644 index 0000000000000..be251484459b6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_4.q new file mode 100644 index 0000000000000..4801a79b8795b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_comparison.q new file mode 100644 index 0000000000000..bdcb6c1b6e447 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_join1.q new file mode 100644 index 0000000000000..a5844b76e9ef8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_serde.q new file mode 100644 index 0000000000000..ffc06d270d53a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/date_udf.q new file mode 100644 index 0000000000000..9696320a85fcd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ddltime.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ddltime.q new file mode 100644 index 0000000000000..3eead6f29dfa6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_1.q new file mode 100644 index 0000000000000..6c689e188a908 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_2.q new file mode 100644 index 0000000000000..4890618a0dc32 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_3.q new file mode 100644 index 0000000000000..28211e3f14cc1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_4.q new file mode 100644 index 0000000000000..e8a89c131cd85 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_join.q new file mode 100644 index 0000000000000..589fc6597dfa4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_precision.q new file mode 100644 index 0000000000000..403c2be3fbc10 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_serde.q new file mode 100644 index 0000000000000..3556807705ae5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/decimal_udf.q new file mode 100644 index 0000000000000..b5ff088d1613a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/default_partition_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/default_partition_name.q new file mode 100644 index 0000000000000..6a82318787979 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/delimiter.q new file mode 100644 index 0000000000000..112ac57c3be5b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/desc_non_existent_tbl.q new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_comment_indent.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_comment_indent.q new file mode 100644 index 0000000000000..310b694874f8d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_comment_nonascii.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_comment_nonascii.q new file mode 100644 index 0000000000000..9027d1ee46511 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_database_json.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_database_json.q new file mode 100644 index 0000000000000..2f4cedcdc4ff7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned.q new file mode 100644 index 0000000000000..241ff1113286e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_formatted_view_partitioned_json.q new file mode 100644 index 0000000000000..150ff70641198 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_pretty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_pretty.q new file mode 100644 index 0000000000000..89db075fd6d1b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_syntax.q new file mode 100644 index 0000000000000..c9255b5d3ceac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_table.q new file mode 100644 index 0000000000000..f72cae9d8003f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_table_json.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_table_json.q new file mode 100644 index 0000000000000..f19e98d7985ce --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_xpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/describe_xpath.q new file mode 100644 index 0000000000000..469cbf43c1141 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/diff_part_input_formats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/diff_part_input_formats.q new file mode 100644 index 0000000000000..c5741d005010d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_file_format_check.q new file mode 100644 index 0000000000000..6ea4156b3493e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disable_merge_for_bucketing.q new file mode 100644 index 0000000000000..471d296b8d2d0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/disallow_incompatible_type_change_off.q new file mode 100644 index 0000000000000..2a1e7276cacfe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/driverhook.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/driverhook.q new file mode 100644 index 0000000000000..191c15436b614 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_database_removes_partition_dirs.q new file mode 100644 index 0000000000000..a5f5189491fed --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_function.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_function.q new file mode 100644 index 0000000000000..18c428ce20a55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_index.q new file mode 100644 index 0000000000000..df23f8d33f0c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_index_removes_partition_dirs.q new file mode 100644 index 0000000000000..ed471db2cffb7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_multi_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_multi_partitions.q new file mode 100644 index 0000000000000..14e235660929f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter.q new file mode 100644 index 0000000000000..04fbcae0a6586 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter2.q new file mode 100644 index 0000000000000..798aa6d51a185 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_filter3.q new file mode 100644 index 0000000000000..2d41da4627bdd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_partitions_ignore_protection.q new file mode 100644 index 0000000000000..e825df9e5b5b4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_table.q new file mode 100644 index 0000000000000..6d189fc43c6b7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_table2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_table2.q new file mode 100644 index 0000000000000..a3e8c5c3bd5eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_table_removes_partition_dirs.q new file mode 100644 index 0000000000000..102b3ef5f5e3a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_udf.q new file mode 100644 index 0000000000000..8aaa66e3e694d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/drop_view.q new file mode 100644 index 0000000000000..4ff097541ee99 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/dynamic_partition_skip_default.q new file mode 100644 index 0000000000000..397a2200a8972 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/enforce_order.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/enforce_order.q new file mode 100644 index 0000000000000..6a303c3ad2139 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape1.q new file mode 100644 index 0000000000000..a7f4cf79c980f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape2.q new file mode 100644 index 0000000000000..473cbf8c94e4e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_clusterby1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_clusterby1.q new file mode 100644 index 0000000000000..13ea15153909a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_distributeby1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_distributeby1.q new file mode 100644 index 0000000000000..40ed2de635367 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_orderby1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_orderby1.q new file mode 100644 index 0000000000000..39a1c4c787e6a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_sortby1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/escape_sortby1.q new file mode 100644 index 0000000000000..6b487c8a47d21 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition.q new file mode 100644 index 0000000000000..6e8bf8ebc1fa2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition2.q new file mode 100644 index 0000000000000..27b335a3d7844 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exchange_partition3.q new file mode 100644 index 0000000000000..7b9060d420a1e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_00_nonpart_empty.q new file mode 100644 index 0000000000000..7fa96b629a0fa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_01_nonpart.q new file mode 100644 index 0000000000000..9920e778d18a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_00_part_empty.q new file mode 100644 index 0000000000000..4017c83aa3dd4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_02_part.q new file mode 100644 index 0000000000000..21138f0263418 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_03_nonpart_over_compat.q new file mode 100644 index 0000000000000..5f6bdee83cc90 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_all_part.q new file mode 100644 index 0000000000000..69c6faa30a07c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_04_evolved_parts.q new file mode 100644 index 0000000000000..cdc02fa25c169 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_05_some_part.q new file mode 100644 index 0000000000000..50a59463b1870 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_06_one_part.q new file mode 100644 index 0000000000000..5136090929fc5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_07_all_part_over_nonoverlap.q new file mode 100644 index 0000000000000..5b9d4ddc03c59 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_08_nonpart_rename.q new file mode 100644 index 0000000000000..173f1569c501a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_09_part_spec_nonoverlap.q new file mode 100644 index 0000000000000..178b76674d862 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_10_external_managed.q new file mode 100644 index 0000000000000..413f2aa4762f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_11_managed_external.q new file mode 100644 index 0000000000000..f3b2896a322a5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_12_external_location.q new file mode 100644 index 0000000000000..37d063432e16a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_13_managed_location.q new file mode 100644 index 0000000000000..fb5058b840bae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_14_managed_location_over_existing.q new file mode 100644 index 0000000000000..031b6bda6cece --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_15_external_part.q new file mode 100644 index 0000000000000..ff088c70d7ffb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_16_part_external.q new file mode 100644 index 0000000000000..6f4ee7a01c256 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_17_part_managed.q new file mode 100644 index 0000000000000..56ec152948aed --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_18_part_external.q new file mode 100644 index 0000000000000..7aa1297dc7d02 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_00_part_external_location.q new file mode 100644 index 0000000000000..cb9f8efc087da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_19_part_external_location.q new file mode 100644 index 0000000000000..bdbd19df70a21 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_20_part_managed_location.q new file mode 100644 index 0000000000000..eb44961a9b7ca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_21_export_authsuccess.q new file mode 100644 index 0000000000000..822ed70a38c0c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_22_import_exist_authsuccess.q new file mode 100644 index 0000000000000..440d08d2dc7b5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_23_import_part_authsuccess.q new file mode 100644 index 0000000000000..30fc343dd8f9c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/exim_24_import_nonexist_authsuccess.q new file mode 100644 index 0000000000000..2dc5af6ce4774 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_dependency.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_dependency.q new file mode 100644 index 0000000000000..eecf2a6bbef04 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_dependency2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_dependency2.q new file mode 100644 index 0000000000000..88d4464a91487 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_logical.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explain_logical.q new file mode 100644 index 0000000000000..4b0fa5b8932d7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explode_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/explode_null.q new file mode 100644 index 0000000000000..76e453525eeba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fetch_aggregation.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fetch_aggregation.q new file mode 100644 index 0000000000000..618fea158aa65 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fileformat_mix.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fileformat_mix.q new file mode 100644 index 0000000000000..1e0c164733fbe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fileformat_sequencefile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fileformat_sequencefile.q new file mode 100644 index 0000000000000..895c70f8af4c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fileformat_text.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/fileformat_text.q new file mode 100644 index 0000000000000..8233aded970c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask.q new file mode 100644 index 0000000000000..fe24da744112a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/filter_join_breaktask2.q new file mode 100644 index 0000000000000..f8d855b25948a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/global_limit.q new file mode 100644 index 0000000000000..b76cf34120701 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1.q new file mode 100755 index 0000000000000..1275eab281f42 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby10.q new file mode 100644 index 0000000000000..db38d43fe4354 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby11.q new file mode 100644 index 0000000000000..0bf92ac4483d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_limit.q new file mode 100644 index 0000000000000..1b6891e33a37e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map.q new file mode 100644 index 0000000000000..82cff36422e62 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_nomap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_nomap.q new file mode 100644 index 0000000000000..eb09a9c70e0c1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_map_skew.q new file mode 100644 index 0000000000000..874995888b447 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby1_noskew.q new file mode 100644 index 0000000000000..1b10f1e024819 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2.q new file mode 100755 index 0000000000000..80b50c39bb7cf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_limit.q new file mode 100644 index 0000000000000..4bc263c77f1b4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map.q new file mode 100644 index 0000000000000..c3cf598fb1c90 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_multi_distinct.q new file mode 100644 index 0000000000000..25e6789b63ef2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_map_skew.q new file mode 100644 index 0000000000000..39a2a178e3a5e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew.q new file mode 100644 index 0000000000000..c3c82d51749fe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby2_noskew_multi_distinct.q new file mode 100644 index 0000000000000..b80c271afe5dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3.q new file mode 100755 index 0000000000000..8f24584a58e50 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map.q new file mode 100644 index 0000000000000..7ecc71dfab64a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_multi_distinct.q new file mode 100644 index 0000000000000..50243beca9efa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_map_skew.q new file mode 100644 index 0000000000000..07d10c2d741d8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew.q new file mode 100644 index 0000000000000..d33f12c5744e9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby3_noskew_multi_distinct.q new file mode 100644 index 0000000000000..86d8986f1df7d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4.q new file mode 100755 index 0000000000000..ae5cae5b73e74 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map.q new file mode 100644 index 0000000000000..8ecce23eb8321 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_map_skew.q new file mode 100644 index 0000000000000..eb2001c6b21b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby4_noskew.q new file mode 100644 index 0000000000000..99c2d2d5a1784 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5.q new file mode 100755 index 0000000000000..0909c90bc4137 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map.q new file mode 100644 index 0000000000000..4fd6445d7927c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_map_skew.q new file mode 100644 index 0000000000000..eccd45dd5b422 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby5_noskew.q new file mode 100644 index 0000000000000..be60785d87ea5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6.q new file mode 100755 index 0000000000000..3a3cc58e0787f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map.q new file mode 100644 index 0000000000000..fbf761c3aea55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_map_skew.q new file mode 100644 index 0000000000000..ac79a286055d5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby6_noskew.q new file mode 100644 index 0000000000000..2c99d362ffff7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7.q new file mode 100644 index 0000000000000..1235e3c0a0ba5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map.q new file mode 100644 index 0000000000000..b1457d9349aee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_multi_single_reducer.q new file mode 100644 index 0000000000000..481b3cd084f16 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_map_skew.q new file mode 100644 index 0000000000000..a34ac8f64b117 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew.q new file mode 100644 index 0000000000000..94a3dcf7ec3f2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby7_noskew_multi_single_reducer.q new file mode 100644 index 0000000000000..802aea244da9e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8.q new file mode 100644 index 0000000000000..e7b08b274d82c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map.q new file mode 100644 index 0000000000000..62b6ff5ddebb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_map_skew.q new file mode 100644 index 0000000000000..846fd01017172 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby8_noskew.q new file mode 100644 index 0000000000000..20c8bef34223a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby9.q new file mode 100644 index 0000000000000..afd40c9ddbaa1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_bigdata.q new file mode 100644 index 0000000000000..7e97f75cecc92 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_complex_types.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_complex_types.q new file mode 100644 index 0000000000000..bb1e6d269710a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_complex_types_multi_single_reducer.q new file mode 100644 index 0000000000000..22c5a389a957a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_cube1.q new file mode 100644 index 0000000000000..46e1f00d0f224 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_distinct_samekey.q new file mode 100644 index 0000000000000..8219a5efb10de --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id1.q new file mode 100644 index 0000000000000..bced21f9e494a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_id2.q new file mode 100644 index 0000000000000..ffc627c82eaff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets1.q new file mode 100644 index 0000000000000..4fba7338f1d0a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets2.q new file mode 100644 index 0000000000000..9f2286cc9be31 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets3.q new file mode 100644 index 0000000000000..9a00d0a7aa77f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets4.q new file mode 100644 index 0000000000000..25f1fcd793a74 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_grouping_sets5.q new file mode 100644 index 0000000000000..fb0c5913fa07f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr.q new file mode 100644 index 0000000000000..f0a8b72b0c83e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_map_ppr_multi_distinct.q new file mode 100644 index 0000000000000..b863344485d9f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_insert_common_distinct.q new file mode 100644 index 0000000000000..aa1c48939a766 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_single_reducer.q new file mode 100644 index 0000000000000..145aab87f1877 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_single_reducer2.q new file mode 100644 index 0000000000000..b58f9d3ade441 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_multi_single_reducer3.q new file mode 100644 index 0000000000000..1253ddfed52a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_mutli_insert_common_distinct.q new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_neg_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_neg_float.q new file mode 100644 index 0000000000000..0e41131861b7d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_position.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_position.q new file mode 100644 index 0000000000000..4316d4a63ab82 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_ppd.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_ppd.q new file mode 100644 index 0000000000000..4cda623b80af8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_ppr.q new file mode 100644 index 0000000000000..1f588eec2477c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_ppr_multi_distinct.q new file mode 100644 index 0000000000000..20c73bd306648 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_rollup1.q new file mode 100644 index 0000000000000..f79b0c472ebb1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_1.q new file mode 100644 index 0000000000000..911a11ae8990f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_10.q new file mode 100644 index 0000000000000..b3ddd42dceed6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_11.q new file mode 100644 index 0000000000000..19063f667898e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_2.q new file mode 100644 index 0000000000000..31b4ec5c74dc2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_3.q new file mode 100644 index 0000000000000..103c57a123576 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_4.q new file mode 100644 index 0000000000000..e43da3c93225f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_5.q new file mode 100644 index 0000000000000..bef5e5d2d547a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_6.q new file mode 100644 index 0000000000000..cf076e8125612 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_7.q new file mode 100644 index 0000000000000..c2d42154e516c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_8.q new file mode 100644 index 0000000000000..121804e60a9da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_9.q new file mode 100644 index 0000000000000..1c3d1cdcc265d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_skew_1.q new file mode 100644 index 0000000000000..068c26a9c2b4b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/groupby_sort_test_1.q new file mode 100644 index 0000000000000..8efa05e2544d4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/having.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/having.q new file mode 100644 index 0000000000000..5b1aa69d9d82b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_context_cs.q new file mode 100644 index 0000000000000..94ba14802f015 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_order.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/hook_order.q new file mode 100644 index 0000000000000..b6753b308b5f1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/implicit_cast1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/implicit_cast1.q new file mode 100644 index 0000000000000..37fb8568c94f4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auth.q new file mode 100644 index 0000000000000..33a1fc581ed13 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto.q new file mode 100644 index 0000000000000..cb8a1d6293f8d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_empty.q new file mode 100644 index 0000000000000..cb32162d40a06 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_file_format.q new file mode 100644 index 0000000000000..790e6c223f746 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables.q new file mode 100644 index 0000000000000..2bf8481f1d6e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_mult_tables_compact.q new file mode 100644 index 0000000000000..808a04cc360a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_multiple.q new file mode 100644 index 0000000000000..06e97fa76bc97 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_partitioned.q new file mode 100644 index 0000000000000..5013d29e732c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_self_join.q new file mode 100644 index 0000000000000..0984a4a21ba17 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_unused.q new file mode 100644 index 0000000000000..d8f3eda1813c8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_update.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_auto_update.q new file mode 100644 index 0000000000000..976de5c67a0d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap.q new file mode 100644 index 0000000000000..f9deb2883960c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap1.q new file mode 100644 index 0000000000000..55633d9225fa7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap2.q new file mode 100644 index 0000000000000..bd15a21fa4200 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap3.q new file mode 100644 index 0000000000000..0d2c811459dcb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto.q new file mode 100644 index 0000000000000..672ce29f1bb47 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_auto_partitioned.q new file mode 100644 index 0000000000000..90d7987594bac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_compression.q new file mode 100644 index 0000000000000..2f5e5d4fb231b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_bitmap_rc.q new file mode 100644 index 0000000000000..054df51c32180 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact.q new file mode 100644 index 0000000000000..a936f1127f690 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_1.q new file mode 100644 index 0000000000000..837033be403a9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_2.q new file mode 100644 index 0000000000000..56119ac11f32f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_3.q new file mode 100644 index 0000000000000..f3fcb4af3d5e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compact_binary_search.q new file mode 100644 index 0000000000000..d0d9a32adcba1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_compression.q new file mode 100644 index 0000000000000..84ed3cc932e8c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_creation.q new file mode 100644 index 0000000000000..062821e870dce --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_serde.q new file mode 100644 index 0000000000000..a6fe16ba3b809 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale.q new file mode 100644 index 0000000000000..82e15b97b6ac4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/index_stale_partitioned.q new file mode 100644 index 0000000000000..e7cfeff31f6dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort.q new file mode 100644 index 0000000000000..07466fd871ae2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_bucketed_table.q new file mode 100644 index 0000000000000..d69f49f66747c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_convert_join.q new file mode 100644 index 0000000000000..6b656ffde745e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_dyn_part.q new file mode 100644 index 0000000000000..119994e91b056 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_grouping_operators.q new file mode 100644 index 0000000000000..e3b8a17c83daa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_list_bucket.q new file mode 100644 index 0000000000000..7d3c0dc7d5859 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_map_operators.q new file mode 100644 index 0000000000000..16e8715a7f3d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_merge.q new file mode 100644 index 0000000000000..41c1a13980cfe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_multi_insert.q new file mode 100644 index 0000000000000..e3992b8da07b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_num_buckets.q new file mode 100644 index 0000000000000..2255bdb34913d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_bucket_sort_reducers_power_two.q new file mode 100644 index 0000000000000..090f04f4b94c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/infer_const_type.q new file mode 100644 index 0000000000000..a039dc5a2a040 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/init_file.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/init_file.q new file mode 100644 index 0000000000000..f69a88d0366e5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/innerjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/innerjoin.q new file mode 100644 index 0000000000000..b384268dbbcff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inoutdriver.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inoutdriver.q new file mode 100644 index 0000000000000..34b0e0b28ea52 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input.q new file mode 100644 index 0000000000000..567d3b07f7ecc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input0.q new file mode 100644 index 0000000000000..63857c779173b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input0.q @@ -0,0 +1,5 @@ +EXPLAIN +SELECT * FROM src; + +SELECT * FROM src; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input1.q new file mode 100644 index 0000000000000..3f2cd96de9cbe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input10.q new file mode 100644 index 0000000000000..aef5c9bfe4b0a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input11.q new file mode 100644 index 0000000000000..5aa6ee785d682 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input11_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input11_limit.q new file mode 100644 index 0000000000000..f68398e3159b2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12.q new file mode 100644 index 0000000000000..d4bc409cce7d0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input12_hadoop20.q new file mode 100644 index 0000000000000..318cd378db137 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input13.q new file mode 100644 index 0000000000000..40fbc84a981f1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input14.q new file mode 100644 index 0000000000000..750c0b82aa3f1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input14_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input14_limit.q new file mode 100644 index 0000000000000..7316752a6df65 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input15.q new file mode 100644 index 0000000000000..2b76497fe5a7a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16.q new file mode 100644 index 0000000000000..82e6d81426efc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input16_cc.q new file mode 100644 index 0000000000000..5dab4103d8dda --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input17.q new file mode 100644 index 0000000000000..14bfb640067bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input18.q new file mode 100644 index 0000000000000..4761dd6cac34a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input19.q new file mode 100644 index 0000000000000..fec44e97669dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input1_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input1_limit.q new file mode 100644 index 0000000000000..71a9b8ff46a25 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input2.q new file mode 100644 index 0000000000000..00693851ef1a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input20.q new file mode 100644 index 0000000000000..0566ab18c2537 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input21.q new file mode 100644 index 0000000000000..d7c814e58061c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input22.q new file mode 100644 index 0000000000000..853947be57a1f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input23.q new file mode 100644 index 0000000000000..634699a00c4a0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input24.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input24.q new file mode 100644 index 0000000000000..95b2377f51b9d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input25.q new file mode 100644 index 0000000000000..e48368ff2a494 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input26.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input26.q new file mode 100644 index 0000000000000..642a7db60ebdf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input28.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input28.q new file mode 100644 index 0000000000000..12e857df8a325 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input2_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input2_limit.q new file mode 100644 index 0000000000000..81045100d380c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3.q new file mode 100644 index 0000000000000..2efa7a4d3d6bc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input30.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input30.q new file mode 100644 index 0000000000000..c84c131082ce0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input31.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input31.q new file mode 100644 index 0000000000000..ece940903a521 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input32.q new file mode 100644 index 0000000000000..cdc04b8c05b85 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input33.q new file mode 100644 index 0000000000000..7ab17515af5c7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input34.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input34.q new file mode 100644 index 0000000000000..c600bf57e49d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input35.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input35.q new file mode 100644 index 0000000000000..5e37ea370a2ef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input36.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input36.q new file mode 100644 index 0000000000000..fd1968f683550 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input37.q new file mode 100644 index 0000000000000..6fd136afec444 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input38.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input38.q new file mode 100644 index 0000000000000..1ccc5f26a603a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39.q new file mode 100644 index 0000000000000..04201dd108b35 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input39_hadoop20.q new file mode 100644 index 0000000000000..29e9fae1da9e3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input3_limit.q new file mode 100644 index 0000000000000..3584820acaf4a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4.q new file mode 100644 index 0000000000000..08d6d97603045 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input40.q new file mode 100644 index 0000000000000..4166cb5f94939 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input41.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input41.q new file mode 100644 index 0000000000000..5bc396e311f69 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input42.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input42.q new file mode 100644 index 0000000000000..8682eb7592fa6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input43.q new file mode 100644 index 0000000000000..5512dc328065d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input44.q new file mode 100644 index 0000000000000..4557edc178b7f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input45.q new file mode 100644 index 0000000000000..633a8c6edb5cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input46.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input46.q new file mode 100644 index 0000000000000..06291d0d66cc5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input49.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input49.q new file mode 100644 index 0000000000000..53fa51a6d3e4e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_cb_delim.q new file mode 100644 index 0000000000000..8c57dd3f25aa6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input4_limit.q new file mode 100644 index 0000000000000..e6e3c467fc763 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input5.q new file mode 100644 index 0000000000000..2af9163da82c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input6.q new file mode 100644 index 0000000000000..4f2f17e49be6e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input7.q new file mode 100644 index 0000000000000..f2e1ed0d3d79d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input8.q new file mode 100644 index 0000000000000..66252a9c5e100 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input9.q new file mode 100644 index 0000000000000..ded20a81c6511 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_columnarserde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_columnarserde.q new file mode 100644 index 0000000000000..9394382b2b347 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q new file mode 100644 index 0000000000000..4f5824df5c9d5 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dfs.q @@ -0,0 +1,2 @@ +dfs -cat ../data/files/kv1.txt; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dynamicserde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_dynamicserde.q new file mode 100644 index 0000000000000..14377429fda8d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_lazyserde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_lazyserde.q new file mode 100644 index 0000000000000..65e678bee8872 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_limit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_limit.q new file mode 100644 index 0000000000000..6d4363e590059 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part0.q new file mode 100644 index 0000000000000..e5526e55414ef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part1.q new file mode 100644 index 0000000000000..b9a210c93daa0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part10.q new file mode 100644 index 0000000000000..b0698b991041b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part10_win.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part10_win.q new file mode 100644 index 0000000000000..0f179ec25b1c1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part2.q new file mode 100644 index 0000000000000..c9aaf25363e41 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part3.q new file mode 100644 index 0000000000000..ea1d2b2408934 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part4.q new file mode 100644 index 0000000000000..71f27befee696 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part5.q new file mode 100644 index 0000000000000..ea9f70bdcb4fe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part6.q new file mode 100644 index 0000000000000..a60c7ac3a0284 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part7.q new file mode 100644 index 0000000000000..10f9336adcea6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part8.q new file mode 100644 index 0000000000000..6f1c9ec88f174 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_part9.q new file mode 100644 index 0000000000000..30008f0f5a964 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testsequencefile.q new file mode 100755 index 0000000000000..d9926888cef9c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath.q new file mode 100755 index 0000000000000..ed22404500811 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath2.q new file mode 100644 index 0000000000000..131105b763d0e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath3.q new file mode 100644 index 0000000000000..1b57ead8225f5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/input_testxpath4.q new file mode 100644 index 0000000000000..e8e8d2ef52b39 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl1.q new file mode 100644 index 0000000000000..b41f0b887cd29 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl2.q new file mode 100644 index 0000000000000..e34779112454e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl3.q new file mode 100644 index 0000000000000..946cf54e40445 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl4.q new file mode 100644 index 0000000000000..0e07d8da6b9a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl5.q new file mode 100644 index 0000000000000..9a7ca5da1277f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl6.q new file mode 100644 index 0000000000000..d33ab8d9a78f4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl7.q new file mode 100644 index 0000000000000..8a73935feec51 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/inputddl8.q new file mode 100644 index 0000000000000..66717a158bd8a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1.q new file mode 100644 index 0000000000000..5ac234db2069c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert1_overwrite_partitions.q new file mode 100644 index 0000000000000..6ad70b5673f30 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert2_overwrite_partitions.q new file mode 100644 index 0000000000000..598d30eaebba2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_compressed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_compressed.q new file mode 100644 index 0000000000000..8f1201cd6fb91 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into1.q new file mode 100644 index 0000000000000..edc65a442d098 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into2.q new file mode 100644 index 0000000000000..0cce9585a89e7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into3.q new file mode 100644 index 0000000000000..e58b212e190bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into4.q new file mode 100644 index 0000000000000..2afe41d668bda --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into5.q new file mode 100644 index 0000000000000..07e80ea4e30fe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_into6.q new file mode 100644 index 0000000000000..2b7988ec6d27a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insert_overwrite_local_directory_1.q new file mode 100644 index 0000000000000..25c127f67f980 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insertexternal1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/insertexternal1.q new file mode 100644 index 0000000000000..9004a0cce87bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join0.q new file mode 100644 index 0000000000000..feac6d85714cb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join1.q new file mode 100644 index 0000000000000..886cba7d30d33 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join10.q new file mode 100644 index 0000000000000..f424618034760 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join11.q new file mode 100644 index 0000000000000..a40f0cc99ed2c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join12.q new file mode 100644 index 0000000000000..7053a06f8e2dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join13.q new file mode 100644 index 0000000000000..914fb44860cb4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14.q new file mode 100644 index 0000000000000..83346b4c3466b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join14_hadoop20.q new file mode 100644 index 0000000000000..a12ef1afb055f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join15.q new file mode 100644 index 0000000000000..b86716c9be261 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join16.q new file mode 100644 index 0000000000000..4c9c9619f3e9a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join17.q new file mode 100644 index 0000000000000..5fbb620e218f3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join18.q new file mode 100644 index 0000000000000..0ec121874cb5c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join18_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join18_multi_distinct.q new file mode 100644 index 0000000000000..eb4e46af421a3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join19.q new file mode 100644 index 0000000000000..5a6b741f198d7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join2.q new file mode 100644 index 0000000000000..7bb547bcb9473 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join20.q new file mode 100644 index 0000000000000..b534a14031207 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join21.q new file mode 100644 index 0000000000000..50eb73ab85f2f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join22.q new file mode 100644 index 0000000000000..4a67d74e83abc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join23.q new file mode 100644 index 0000000000000..2db69d394171d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join24.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join24.q new file mode 100644 index 0000000000000..65c9256d6f264 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join25.q new file mode 100644 index 0000000000000..c0e4cbb93a386 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join26.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join26.q new file mode 100644 index 0000000000000..cecf8da533d8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join27.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join27.q new file mode 100644 index 0000000000000..2c6e95765752f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join28.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join28.q new file mode 100644 index 0000000000000..1d652b3a51843 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join29.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join29.q new file mode 100644 index 0000000000000..7fa4815486c8c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join3.q new file mode 100644 index 0000000000000..f38fe215e8767 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join30.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join30.q new file mode 100644 index 0000000000000..036a16d3b1eec --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join31.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join31.q new file mode 100644 index 0000000000000..0ba143aa76b41 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join32.q new file mode 100644 index 0000000000000..82fde0e965878 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join32_lessSize.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join32_lessSize.q new file mode 100644 index 0000000000000..58a26c5bb3fa5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join33.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join33.q new file mode 100644 index 0000000000000..82fde0e965878 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join34.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join34.q new file mode 100644 index 0000000000000..85adcfd232955 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join35.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join35.q new file mode 100644 index 0000000000000..d66616c3cd5ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join36.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join36.q new file mode 100644 index 0000000000000..9912610da9667 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join37.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join37.q new file mode 100644 index 0000000000000..a971f4ae07ebb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join38.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join38.q new file mode 100644 index 0000000000000..7fbe377585917 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join39.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join39.q new file mode 100644 index 0000000000000..d0edac50ef544 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join4.q new file mode 100644 index 0000000000000..100076e6cf1b5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join40.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join40.q new file mode 100644 index 0000000000000..e93f08289ac27 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join41.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join41.q new file mode 100644 index 0000000000000..b8d9397fd6569 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join5.q new file mode 100644 index 0000000000000..405a0a57448a5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join6.q new file mode 100644 index 0000000000000..f55bc08580eb4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join7.q new file mode 100644 index 0000000000000..34f502fa23cc8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join8.q new file mode 100644 index 0000000000000..a7fdf4cd14ad9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join9.q new file mode 100644 index 0000000000000..80add736c2a9e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_1to1.q new file mode 100644 index 0000000000000..b403814adb4c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_array.q new file mode 100644 index 0000000000000..e4d95a51fa927 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_casesensitive.q new file mode 100644 index 0000000000000..a9b69c5b1c448 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_empty.q new file mode 100644 index 0000000000000..1982487c84b9d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters.q new file mode 100644 index 0000000000000..d54aa950a1ec2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters_overlap.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_filters_overlap.q new file mode 100644 index 0000000000000..4394396790d9f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_hive_626.q new file mode 100644 index 0000000000000..31b0c8c91c10b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_literals.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_literals.q new file mode 100644 index 0000000000000..250bf839a4cdd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_map_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_map_ppr.q new file mode 100644 index 0000000000000..d8943c4204adb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nulls.q new file mode 100644 index 0000000000000..4ff60713d6b44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_nullsafe.q new file mode 100644 index 0000000000000..05b57bce202d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_rc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_rc.q new file mode 100644 index 0000000000000..8a7c2d3eebeb8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder.q new file mode 100644 index 0000000000000..b92a79ba07ab2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder2.q new file mode 100644 index 0000000000000..238c0adad3128 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder3.q new file mode 100644 index 0000000000000..1bda28fbc3d5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_reorder4.q new file mode 100644 index 0000000000000..126f356ef785a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_star.q new file mode 100644 index 0000000000000..8314161975761 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_thrift.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_thrift.q new file mode 100644 index 0000000000000..1b4d491207b8d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_vc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_vc.q new file mode 100644 index 0000000000000..63b3da7190f5c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/join_view.q new file mode 100644 index 0000000000000..16b6816f0c861 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/keyword_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/keyword_1.q new file mode 100644 index 0000000000000..2e996af28e39a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view.q new file mode 100644 index 0000000000000..f0c094fae3775 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_cp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_cp.q new file mode 100644 index 0000000000000..5e0b6fa20ebe9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_noalias.q new file mode 100644 index 0000000000000..11e54a5b04c07 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_outer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_outer.q new file mode 100644 index 0000000000000..d65e79df7b3d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lateral_view_ppd.q new file mode 100644 index 0000000000000..7be86a6f10ea2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag.q new file mode 100644 index 0000000000000..f49766771067b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leadlag_queries.q new file mode 100644 index 0000000000000..6ef3bdb107a12 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin.q new file mode 100644 index 0000000000000..abe3d3317157c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/leftsemijoin_mr.q new file mode 100644 index 0000000000000..5813ca3c619cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown.q new file mode 100644 index 0000000000000..e4d0aa06bde87 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/limit_pushdown_negative.q new file mode 100644 index 0000000000000..a86ddf14047ef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lineage1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lineage1.q new file mode 100644 index 0000000000000..4743b7f413ab9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_1.q new file mode 100644 index 0000000000000..1062e5503fd43 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_10.q new file mode 100644 index 0000000000000..2fbafc20169b5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_11.q new file mode 100644 index 0000000000000..8260459725efe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_12.q new file mode 100644 index 0000000000000..3606e3b8b77c3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_13.q new file mode 100644 index 0000000000000..4903604bd0063 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_2.q new file mode 100644 index 0000000000000..3a39f42b2a77d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_3.q new file mode 100644 index 0000000000000..a43613cad5e8a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_4.q new file mode 100644 index 0000000000000..918c817e49eff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_5.q new file mode 100644 index 0000000000000..5ea2e62551a66 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_6.q new file mode 100644 index 0000000000000..d50e159277eb8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_7.q new file mode 100644 index 0000000000000..f71e80fa2a099 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_8.q new file mode 100644 index 0000000000000..9e81b8d585e6f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_dml_9.q new file mode 100644 index 0000000000000..1b69a9a0d39ef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_1.q new file mode 100644 index 0000000000000..48b1ff3ac81a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_2.q new file mode 100644 index 0000000000000..a364168a6e3be --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_multiskew_3.q new file mode 100644 index 0000000000000..2d1193deff0cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_1.q new file mode 100644 index 0000000000000..7769fb7af75d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_2.q new file mode 100644 index 0000000000000..6a6945d4fdf9e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/list_bucket_query_oneskew_3.q new file mode 100644 index 0000000000000..892b4c5d3d7cc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_decimal.q new file mode 100644 index 0000000000000..a6ad4b8485a26 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_double.q new file mode 100644 index 0000000000000..08836127b9958 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_ints.q new file mode 100644 index 0000000000000..9da622e246793 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/literal_string.q new file mode 100644 index 0000000000000..21f0890ada168 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_binary_data.q new file mode 100644 index 0000000000000..7da363183ccf6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part1.q new file mode 100644 index 0000000000000..52b4937d4a9dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part10.q new file mode 100644 index 0000000000000..9517664675d69 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part11.q new file mode 100644 index 0000000000000..4cfbfe8eac003 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part12.q new file mode 100644 index 0000000000000..dd8bd530af43c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part13.q new file mode 100644 index 0000000000000..0b8d44f7cf597 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part14.q new file mode 100644 index 0000000000000..ba3ce39cf7920 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part14_win.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part14_win.q new file mode 100644 index 0000000000000..d32b539ae96ab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part15.q new file mode 100644 index 0000000000000..3c3310882a617 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part2.q new file mode 100644 index 0000000000000..03aa5df03501f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part3.q new file mode 100644 index 0000000000000..e4c8c17f63048 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part4.q new file mode 100644 index 0000000000000..3f3a0c8d51b41 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part5.q new file mode 100644 index 0000000000000..5780f5d43911b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part6.q new file mode 100644 index 0000000000000..b5e85aebf4ade --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part7.q new file mode 100644 index 0000000000000..b9e1da032660b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part8.q new file mode 100644 index 0000000000000..8073500c0bf07 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_dyn_part9.q new file mode 100644 index 0000000000000..01fa596cdf04a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_exist_part_authsuccess.q new file mode 100644 index 0000000000000..6d2a8b82d33c1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_file_with_space_in_the_name.q new file mode 100644 index 0000000000000..3b8951a1a782f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs.q new file mode 100644 index 0000000000000..c1ac29c172f60 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_fs2.q new file mode 100644 index 0000000000000..7255324d1653d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_hdfs_file_with_space_in_the_name.q new file mode 100644 index 0000000000000..cce297cca46db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_nonpart_authsuccess.q new file mode 100644 index 0000000000000..40d8210e57d2a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_overwrite.q new file mode 100644 index 0000000000000..73853f15a94bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/load_part_authsuccess.q new file mode 100644 index 0000000000000..ff54324a5a50c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart1.q new file mode 100644 index 0000000000000..0813bb23c3746 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/loadpart_err.q new file mode 100644 index 0000000000000..6e4df215479bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock1.q new file mode 100644 index 0000000000000..326cf4910fb49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock2.q new file mode 100644 index 0000000000000..c8bff56adabe9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock3.q new file mode 100644 index 0000000000000..7f2178f5408d6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/lock4.q new file mode 100644 index 0000000000000..e07bfe867d03f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/louter_join_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/louter_join_ppr.q new file mode 100644 index 0000000000000..c4e25eb470196 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/macro.q new file mode 100644 index 0000000000000..fd0f7f2b0cdd4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin1.q new file mode 100644 index 0000000000000..49de907948f6d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_distinct.q new file mode 100644 index 0000000000000..d021c631d1ea5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_filter_on_outerjoin.q new file mode 100644 index 0000000000000..a43d298a4ed57 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_hook.q new file mode 100644 index 0000000000000..d6811d493263f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_mapjoin.q new file mode 100644 index 0000000000000..3f87db28ed2e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery.q new file mode 100644 index 0000000000000..81cf940bd01c3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_subquery2.q new file mode 100644 index 0000000000000..9980946057fe1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_test_outer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapjoin_test_outer.q new file mode 100644 index 0000000000000..8d31590186514 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce1.q new file mode 100644 index 0000000000000..83328f1f83cfc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce2.q new file mode 100644 index 0000000000000..ef6f8e5766ffa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce3.q new file mode 100644 index 0000000000000..c31ede77d8718 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce4.q new file mode 100644 index 0000000000000..0c15f1fb55174 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce5.q new file mode 100644 index 0000000000000..d7c53d2889c0e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce6.q new file mode 100644 index 0000000000000..c37181d351068 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce7.q new file mode 100644 index 0000000000000..0da41d2ecde53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mapreduce8.q new file mode 100644 index 0000000000000..35d9ed90e77e5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge1.q new file mode 100644 index 0000000000000..834f2ce374608 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge2.q new file mode 100644 index 0000000000000..8b77bd2fe19ba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge3.q new file mode 100644 index 0000000000000..aacd0cd68fc50 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge4.q new file mode 100644 index 0000000000000..744783bd62f8d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition.q new file mode 100644 index 0000000000000..1379426b601f0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition2.q new file mode 100644 index 0000000000000..b51c70ed03a46 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition3.q new file mode 100644 index 0000000000000..b3bcf01ea043d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition4.q new file mode 100644 index 0000000000000..ef769a042d7ce --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/merge_dynamic_partition5.q new file mode 100644 index 0000000000000..a196fa05288b7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mergejoins.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mergejoins.q new file mode 100644 index 0000000000000..2857c8612aeb0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mergejoins_mixed.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mergejoins_mixed.q new file mode 100644 index 0000000000000..a7df4e3d2dc65 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadata_export_drop.q new file mode 100644 index 0000000000000..41be152e7871d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadataonly1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/metadataonly1.q new file mode 100644 index 0000000000000..eb595298497f1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/mi.q new file mode 100644 index 0000000000000..067c143c0e684 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/misc_json.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/misc_json.q new file mode 100644 index 0000000000000..541e36933a6d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin1.q new file mode 100644 index 0000000000000..9a0a792a91897 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multiMapJoin2.q new file mode 100644 index 0000000000000..ce6cf6d8d6c0d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert.q new file mode 100644 index 0000000000000..718211b968ccd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_gby.q new file mode 100644 index 0000000000000..3aac82ec5a95c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_gby2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_gby2.q new file mode 100644 index 0000000000000..46e2b19576856 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_gby3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_gby3.q new file mode 100644 index 0000000000000..040adca5c2e26 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_lateral_view.q new file mode 100644 index 0000000000000..eb1b9ed3b437a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_insert_move_tasks_share_dependencies.q new file mode 100644 index 0000000000000..e30992b2cde6b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_join_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_join_union.q new file mode 100644 index 0000000000000..39650fd4a1ef1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_sahooks.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multi_sahooks.q new file mode 100644 index 0000000000000..a6cd1ef940018 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multigroupby_singlemr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/multigroupby_singlemr.q new file mode 100644 index 0000000000000..a4e86ade476a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nested_complex.q new file mode 100644 index 0000000000000..b94fbb7b8be43 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nestedvirtual.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nestedvirtual.q new file mode 100644 index 0000000000000..7b80941b8c396 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/newline.q new file mode 100644 index 0000000000000..722ecf6d972ca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/no_hooks.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/no_hooks.q new file mode 100644 index 0000000000000..bbd1feaa37533 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/noalias_subq1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/noalias_subq1.q new file mode 100644 index 0000000000000..dca8a42badb89 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nomore_ambiguous_table_col.q new file mode 100644 index 0000000000000..6795475699583 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonblock_op_deduplicate.q new file mode 100644 index 0000000000000..a9cb7d0217540 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonmr_fetch.q new file mode 100644 index 0000000000000..e961e93a18790 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_input37.q new file mode 100644 index 0000000000000..9cb89da373d29 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nonreserved_keywords_insert_into1.q new file mode 100644 index 0000000000000..144cfeee6b14b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias1.q new file mode 100644 index 0000000000000..ed7e31883af33 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/notable_alias2.q new file mode 100644 index 0000000000000..b9c963c97a044 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_cast.q new file mode 100644 index 0000000000000..48c39b81fdd8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/null_column.q new file mode 100644 index 0000000000000..fa4a8639446e5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup.q new file mode 100644 index 0000000000000..5ec67e560513e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup2.q new file mode 100644 index 0000000000000..fecd6d4bd8cc8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup3.q new file mode 100644 index 0000000000000..a5bc9ff1f36ff --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup4.q new file mode 100644 index 0000000000000..917d1a78ee696 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup4_multi_distinct.q new file mode 100644 index 0000000000000..fcee62929e466 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullgroup5.q new file mode 100644 index 0000000000000..12773b6159a5d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullinput.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullinput.q new file mode 100644 index 0000000000000..4a58ed514ab17 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullinput2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullinput2.q new file mode 100644 index 0000000000000..f5c0af8fa5efa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/nullscript.q new file mode 100644 index 0000000000000..95c9e1df37679 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/num_op_type_conv.q new file mode 100644 index 0000000000000..7f858d3e6f136 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ops_comparison.q new file mode 100644 index 0000000000000..b685ae6c4e53f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optional_outer.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optional_outer.q new file mode 100644 index 0000000000000..10971ab51e328 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/optrstat_groupby.q new file mode 100644 index 0000000000000..5993041405ed8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_create.q new file mode 100644 index 0000000000000..6aca5486445c8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_createas1.q new file mode 100644 index 0000000000000..872692567b37d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_dictionary_threshold.q new file mode 100644 index 0000000000000..f916012b8365e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_diff_part_cols.q new file mode 100644 index 0000000000000..cbfd7b359f878 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_files.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_files.q new file mode 100644 index 0000000000000..d3cbc5a372955 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_empty_strings.q new file mode 100644 index 0000000000000..0ef57d18ccaf7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_ends_with_nulls.q new file mode 100644 index 0000000000000..6685da7a82245 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/orc_predicate_pushdown.q new file mode 100644 index 0000000000000..f5f25f00c951a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order.q new file mode 100644 index 0000000000000..d389892eda48c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/order2.q new file mode 100644 index 0000000000000..6c16fe30fa7ec --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/outer_join_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/outer_join_ppr.q new file mode 100644 index 0000000000000..38007cab5f679 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/overridden_confs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/overridden_confs.q new file mode 100644 index 0000000000000..9dcaed6acb12f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel.q new file mode 100644 index 0000000000000..03edeaadeef51 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parallel_orderby.q new file mode 100644 index 0000000000000..5e09395901c08 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parenthesis_star_by.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/parenthesis_star_by.q new file mode 100644 index 0000000000000..9e036c1a91d3b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partInit.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partInit.q new file mode 100644 index 0000000000000..c492b87e0e96e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/part_inherit_tbl_props.q new file mode 100644 index 0000000000000..3ee1b4ac80e32 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/part_inherit_tbl_props_empty.q new file mode 100644 index 0000000000000..f3f0335c883d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/part_inherit_tbl_props_with_star.q new file mode 100644 index 0000000000000..8dac6111ec059 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partcols1.q new file mode 100644 index 0000000000000..b7f8c64d4261f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date.q new file mode 100644 index 0000000000000..8738afdfa099a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_date2.q new file mode 100644 index 0000000000000..9b84b59608503 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_decode_name.q new file mode 100644 index 0000000000000..ba193cd51a26d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_schema1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_schema1.q new file mode 100644 index 0000000000000..550ea47beb0db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_serde_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_serde_format.q new file mode 100644 index 0000000000000..9f7620c148236 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_special_char.q new file mode 100644 index 0000000000000..81344334dfe01 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_type_check.q new file mode 100644 index 0000000000000..7f1accadac6ea --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_varchar1.q new file mode 100644 index 0000000000000..d700b1cbf8566 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_vs_table_metadata.q new file mode 100644 index 0000000000000..9f493e561f1f8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat.q new file mode 100644 index 0000000000000..f0d58cf61d714 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat10.q new file mode 100644 index 0000000000000..f15f72c08eb34 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat11.q new file mode 100644 index 0000000000000..1a4291fe64f28 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat12.q new file mode 100644 index 0000000000000..bc51cb5587b16 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat13.q new file mode 100644 index 0000000000000..2e4ae6954bcb5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat14.q new file mode 100644 index 0000000000000..f4d4d73745bf5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat15.q new file mode 100644 index 0000000000000..6fce1e0c77a4f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat16.q new file mode 100644 index 0000000000000..37bb1a76b258e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat17.q new file mode 100644 index 0000000000000..e9b574c1ca380 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat2.q new file mode 100644 index 0000000000000..5501a653a30a8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat3.q new file mode 100644 index 0000000000000..b85fec538fdd9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat4.q new file mode 100644 index 0000000000000..33decec6eda6f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat5.q new file mode 100644 index 0000000000000..fc3bb8458b3e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat6.q new file mode 100644 index 0000000000000..dac5ef213e7c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat7.q new file mode 100644 index 0000000000000..c4a19ada5ff2f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat8.q new file mode 100644 index 0000000000000..46ea10f4b7a2f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partition_wise_fileformat9.q new file mode 100644 index 0000000000000..5205585853140 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partitions_json.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/partitions_json.q new file mode 100644 index 0000000000000..825e3f600464f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/pcr.q new file mode 100644 index 0000000000000..09a39ae4e4476 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/plan_json.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/plan_json.q new file mode 100644 index 0000000000000..0243c995412b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd1.q new file mode 100644 index 0000000000000..a5a4db0544ed3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd2.q new file mode 100644 index 0000000000000..536eb989dff95 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_clusterby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_clusterby.q new file mode 100644 index 0000000000000..0c8b245d81320 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_constant_expr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_constant_expr.q new file mode 100644 index 0000000000000..969c7e4f0cf28 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_constant_where.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_constant_where.q new file mode 100644 index 0000000000000..0c102354fed7a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_gby.q new file mode 100644 index 0000000000000..65c0b01198491 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_gby2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_gby2.q new file mode 100644 index 0000000000000..ae288eada2610 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_gby_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_gby_join.q new file mode 100644 index 0000000000000..82f358b0d6ca1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join.q new file mode 100644 index 0000000000000..09323e965b846 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join2.q new file mode 100644 index 0000000000000..8bd4cae3f96c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join3.q new file mode 100644 index 0000000000000..7abf41098618e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join_filter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_join_filter.q new file mode 100644 index 0000000000000..9d42ef20860d1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_multi_insert.q new file mode 100644 index 0000000000000..a802df1b98819 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join1.q new file mode 100644 index 0000000000000..752176c4b7e4b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join2.q new file mode 100644 index 0000000000000..e22892b45f2d0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join3.q new file mode 100644 index 0000000000000..d756b90e84316 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join4.q new file mode 100644 index 0000000000000..e465eda68e258 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_outer_join5.q new file mode 100644 index 0000000000000..382b9494cf7b9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_random.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_random.q new file mode 100644 index 0000000000000..de0c067f353b9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_repeated_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_repeated_alias.q new file mode 100644 index 0000000000000..732671afbe3da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_transform.q new file mode 100644 index 0000000000000..65a498d021f77 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udf_case.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udf_case.q new file mode 100644 index 0000000000000..b7a6b15bfd1c2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udf_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_udf_col.q new file mode 100644 index 0000000000000..993da9752cb7f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union.q new file mode 100644 index 0000000000000..2cd876e078e93 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_union_view.q new file mode 100644 index 0000000000000..d635e2d171469 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_vc.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppd_vc.q new file mode 100644 index 0000000000000..b5efdb31bd78c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_allchildsarenull.q new file mode 100644 index 0000000000000..3cda01b8b343b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown.q new file mode 100644 index 0000000000000..860dd631ce10a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown2.q new file mode 100644 index 0000000000000..67c0da0dfc591 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ppr_pushdown3.q new file mode 100644 index 0000000000000..35c11be2d62e4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/print_header.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/print_header.q new file mode 100644 index 0000000000000..3ca0340e2d267 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/progress_1.q new file mode 100644 index 0000000000000..ad908a02ad181 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/protectmode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/protectmode.q new file mode 100644 index 0000000000000..27055fbbb8235 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/protectmode2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/protectmode2.q new file mode 100644 index 0000000000000..978b03264bb6e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf.q new file mode 100644 index 0000000000000..eea5415d682a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_decimal.q new file mode 100644 index 0000000000000..03f435e4539e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_general_queries.q new file mode 100644 index 0000000000000..885c3b3d43e17 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_matchpath.q new file mode 100644 index 0000000000000..72eeb104d5298 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_rcfile.q new file mode 100644 index 0000000000000..535a233a9e3c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_register_tblfn.q new file mode 100644 index 0000000000000..a2140cd049f2d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ptf_seqfile.q new file mode 100644 index 0000000000000..4aa8ce11bec9c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/push_or.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/push_or.q new file mode 100644 index 0000000000000..c9b6cea23cd85 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/ql_rewrite_gbtoidx.q new file mode 100644 index 0000000000000..f198baa6e4c5b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/query_properties.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/query_properties.q new file mode 100644 index 0000000000000..027ba7f7a833c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/query_result_fileformat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/query_result_fileformat.q new file mode 100644 index 0000000000000..3b87e4d170c1f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/query_with_semi.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/query_with_semi.q new file mode 100644 index 0000000000000..e0f90d5edb3e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote1.q new file mode 100644 index 0000000000000..f968132fa5913 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/quote2.q new file mode 100644 index 0000000000000..65b9f8776d192 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rand_partitionpruner1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rand_partitionpruner1.q new file mode 100644 index 0000000000000..3eeb5e7b88f6e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rand_partitionpruner2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rand_partitionpruner2.q new file mode 100644 index 0000000000000..e2f280efe754f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rand_partitionpruner3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rand_partitionpruner3.q new file mode 100644 index 0000000000000..9f4aaac267864 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_bigdata.q new file mode 100644 index 0000000000000..3e83e6693b276 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_columnar.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_columnar.q new file mode 100644 index 0000000000000..0178e535cdb90 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_createas1.q new file mode 100644 index 0000000000000..f36203724c15f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_default_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_default_format.q new file mode 100644 index 0000000000000..e61292e7f1edb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_lazydecompress.q new file mode 100644 index 0000000000000..7f55d10bd6458 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge1.q new file mode 100644 index 0000000000000..1f6f1bd251c25 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge2.q new file mode 100644 index 0000000000000..215d5ebc4a25b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge3.q new file mode 100644 index 0000000000000..39fbd2564664b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_merge4.q new file mode 100644 index 0000000000000..fe6df28566cf0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_null_value.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_null_value.q new file mode 100644 index 0000000000000..2e4c929fb3f52 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_toleratecorruptions.q new file mode 100644 index 0000000000000..077a52d3934ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rcfile_union.q new file mode 100644 index 0000000000000..fc8c7500804f2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/recursive_dir.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/recursive_dir.q new file mode 100644 index 0000000000000..709c6eb454ee6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate.q new file mode 100644 index 0000000000000..2e26adc9ed957 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_gby.q new file mode 100644 index 0000000000000..e28ed9fc536bf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate_exclude_join.q new file mode 100644 index 0000000000000..ff09754dd118e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reduce_deduplicate_extended.q new file mode 100644 index 0000000000000..48af2c2affc4a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reducesink_dedup.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reducesink_dedup.q new file mode 100644 index 0000000000000..d5aa2513c8b3b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regex_col.q new file mode 100644 index 0000000000000..9cfcee5a599c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regexp_extract.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/regexp_extract.q new file mode 100644 index 0000000000000..2074eea4396f3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/remote_script.q new file mode 100644 index 0000000000000..926601c647bc3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_column.q new file mode 100644 index 0000000000000..a3f3f30466ced --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_external_partition_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_external_partition_location.q new file mode 100644 index 0000000000000..faebd55b05f49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_partition_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_partition_location.q new file mode 100644 index 0000000000000..ee4ff8171f60b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_table_location.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/rename_table_location.q new file mode 100644 index 0000000000000..ad0ce3a933450 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/repair.q new file mode 100644 index 0000000000000..8d04d3e991c59 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reset_conf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/reset_conf.q new file mode 100644 index 0000000000000..8ddde2364536d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/router_join_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/router_join_ppr.q new file mode 100644 index 0000000000000..9ec5b7b8f5647 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample1.q new file mode 100644 index 0000000000000..6e86cef2675bd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample10.q new file mode 100644 index 0000000000000..1c6695c5410e8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample2.q new file mode 100644 index 0000000000000..9ca5d26c00895 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample3.q new file mode 100644 index 0000000000000..9dec7172d96af --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample4.q new file mode 100644 index 0000000000000..85a9d92a2d144 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample5.q new file mode 100644 index 0000000000000..836ad2f15e43e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample6.q new file mode 100644 index 0000000000000..ea3511f2435d6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample7.q new file mode 100644 index 0000000000000..18344d79c6a09 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample8.q new file mode 100644 index 0000000000000..90fb1fb1ff724 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample9.q new file mode 100644 index 0000000000000..03139ca1fd758 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook.q new file mode 100644 index 0000000000000..12f2bcd46ec8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sample_islocalmode_hook_hadoop20.q new file mode 100644 index 0000000000000..484e1fa617d8a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority.q new file mode 100644 index 0000000000000..c0e18c5688b3c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/schemeAuthority2.q new file mode 100644 index 0000000000000..ecd4d13d0e23b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/script_env_var1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/script_env_var1.q new file mode 100644 index 0000000000000..82268f4dfb595 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/script_env_var2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/script_env_var2.q new file mode 100644 index 0000000000000..7957fe518d55d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/script_pipe.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/script_pipe.q new file mode 100644 index 0000000000000..ae2fd2ed8f2dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/scriptfile1.q new file mode 100644 index 0000000000000..4f65016f3081c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_as_omitted.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_as_omitted.q new file mode 100644 index 0000000000000..f0fe03f180c08 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_transform_hint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_transform_hint.q new file mode 100644 index 0000000000000..33959ed5b46a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_unquote_and.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_unquote_and.q new file mode 100644 index 0000000000000..275c774d1d4eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_unquote_not.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_unquote_not.q new file mode 100644 index 0000000000000..93d992e69844f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_unquote_or.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/select_unquote_or.q new file mode 100644 index 0000000000000..5dd0776792f5b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/semicolon.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/semicolon.q new file mode 100644 index 0000000000000..fef2687308d02 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/semicolon.q @@ -0,0 +1,4 @@ +-- comment +-- comment; +-- comment +SELECT COUNT(1) FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/semijoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/semijoin.q new file mode 100644 index 0000000000000..142a51ab52e8f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_regex.q new file mode 100644 index 0000000000000..2a287bd877759 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_reported_schema.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_reported_schema.q new file mode 100644 index 0000000000000..01ec864bc925f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_user_properties.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/serde_user_properties.q new file mode 100644 index 0000000000000..4d523e1bf7167 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_processor_namespaces.q new file mode 100644 index 0000000000000..7e3d1f4d8aa41 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_variable_sub.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/set_variable_sub.q new file mode 100644 index 0000000000000..7bf1e60a5d3f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_columns.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_columns.q new file mode 100644 index 0000000000000..58a3a2c29e2cf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_alter.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_alter.q new file mode 100644 index 0000000000000..1158a69ba338b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_db_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_db_table.q new file mode 100644 index 0000000000000..1937acb547cdc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_delimited.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_delimited.q new file mode 100644 index 0000000000000..77229641f121e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_partitioned.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_partitioned.q new file mode 100644 index 0000000000000..3b691544e5313 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_serde.q new file mode 100644 index 0000000000000..a3eb5a8f7d246 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_create_table_view.q new file mode 100644 index 0000000000000..a378456494ab4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_describe_func_quotes.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_describe_func_quotes.q new file mode 100644 index 0000000000000..de1451d633834 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_functions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_functions.q new file mode 100644 index 0000000000000..c09f50a10c0d2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_edge_cases.q new file mode 100644 index 0000000000000..5fcdf97e2db47 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_indexes_syntax.q new file mode 100644 index 0000000000000..ab588937e179b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_partitions.q new file mode 100644 index 0000000000000..7fa7b828bd72e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tables.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tables.q new file mode 100644 index 0000000000000..f33325dbb9e41 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tablestatus.q new file mode 100644 index 0000000000000..9184d6da897c0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tblproperties.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/show_tblproperties.q new file mode 100644 index 0000000000000..c755108de22f1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/showparts.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/showparts.q new file mode 100644 index 0000000000000..dbee3efc24f7f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/showparts.q @@ -0,0 +1,4 @@ +EXPLAIN +SHOW PARTITIONS srcpart; + +SHOW PARTITIONS srcpart; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin.q new file mode 100644 index 0000000000000..ad917beeef9ce --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_1.q new file mode 100644 index 0000000000000..03eab4cd6d54e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoin_union_remove_2.q new file mode 100644 index 0000000000000..9cb919531f7c5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt1.q new file mode 100644 index 0000000000000..af446bb65c08e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt10.q new file mode 100644 index 0000000000000..199f3201afb77 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt11.q new file mode 100644 index 0000000000000..ef61fb22f17a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt12.q new file mode 100644 index 0000000000000..b5d9d9bc46874 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt13.q new file mode 100644 index 0000000000000..0634c4f4ff6db --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt14.q new file mode 100644 index 0000000000000..0f031dd4fc0ae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt15.q new file mode 100644 index 0000000000000..d5474a455e9dd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt16.q new file mode 100644 index 0000000000000..46b4f6d6e600d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt17.q new file mode 100644 index 0000000000000..0592ca8c3e498 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt18.q new file mode 100644 index 0000000000000..433fea336dfd4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt19.q new file mode 100644 index 0000000000000..0b11ebe4cb696 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt2.q new file mode 100644 index 0000000000000..34fcdbfac4cb7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt20.q new file mode 100644 index 0000000000000..f217052881e24 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt3.q new file mode 100644 index 0000000000000..f6002ad49802a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt4.q new file mode 100644 index 0000000000000..ca83c446085fa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt5.q new file mode 100644 index 0000000000000..3d7884c5e3dcb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt6.q new file mode 100644 index 0000000000000..36cf8ceeaebb4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt7.q new file mode 100644 index 0000000000000..cf84f67b6a0fc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt8.q new file mode 100644 index 0000000000000..d0ac845f86581 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/skewjoinopt9.q new file mode 100644 index 0000000000000..04834033a11e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin9.q new file mode 100644 index 0000000000000..b9590227576c9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_1.q new file mode 100644 index 0000000000000..359513e424dba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_10.q new file mode 100644 index 0000000000000..a79ebf62d0693 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_11.q new file mode 100644 index 0000000000000..48ba8e80cccf6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_12.q new file mode 100644 index 0000000000000..9f9748c41f3ab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_13.q new file mode 100644 index 0000000000000..056bccd68999b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_14.q new file mode 100644 index 0000000000000..f03f92e469d53 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_15.q new file mode 100644 index 0000000000000..1e77a60670b73 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_16.q new file mode 100644 index 0000000000000..3a3a872427526 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_17.q new file mode 100644 index 0000000000000..238f7e011b72c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_18.q new file mode 100644 index 0000000000000..6880be340792a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_19.q new file mode 100644 index 0000000000000..ca48f61f9aaf8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_2.q new file mode 100644 index 0000000000000..9d86314879d68 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_20.q new file mode 100644 index 0000000000000..f70e7d5c86237 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_21.q new file mode 100644 index 0000000000000..09edfc10cd950 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_22.q new file mode 100644 index 0000000000000..676f46ae6d2aa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_25.q new file mode 100644 index 0000000000000..8b534e85aee1d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_3.q new file mode 100644 index 0000000000000..73b21fae250e0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_4.q new file mode 100644 index 0000000000000..83143b170ed5c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_5.q new file mode 100644 index 0000000000000..61ec084f64ffa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_6.q new file mode 100644 index 0000000000000..2884a11433d32 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_7.q new file mode 100644 index 0000000000000..1488b1f949527 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/smb_mapjoin_8.q new file mode 100644 index 0000000000000..6f282ed441bfa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort.q new file mode 100644 index 0000000000000..a2a54ee393f10 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_1.q new file mode 100644 index 0000000000000..8002ec55fa7f2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_2.q new file mode 100644 index 0000000000000..bd0cdb2b952b4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_3.q new file mode 100644 index 0000000000000..a109878f88585 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_4.q new file mode 100644 index 0000000000000..0bc507130c242 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_5.q new file mode 100644 index 0000000000000..3505db09dd3ee --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_6.q new file mode 100644 index 0000000000000..35b05353ccbdd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/sort_merge_join_desc_7.q new file mode 100644 index 0000000000000..65dc7f1d8eda9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q new file mode 100644 index 0000000000000..6fe3d211a00b0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/source.q @@ -0,0 +1 @@ +source ../data/files/source.txt; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/split_sample.q new file mode 100644 index 0000000000000..952eaf72f10c1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats0.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats0.q new file mode 100644 index 0000000000000..48b5c52600ec4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats1.q new file mode 100644 index 0000000000000..0b783de153b29 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats10.q new file mode 100644 index 0000000000000..a3f375e5a460a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats11.q new file mode 100644 index 0000000000000..6618c913ea700 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats12.q new file mode 100644 index 0000000000000..c1db34618e753 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats13.q new file mode 100644 index 0000000000000..beb7bfa42e10f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats14.q new file mode 100644 index 0000000000000..8e74df535d086 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats15.q new file mode 100644 index 0000000000000..9a557c6708544 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats16.q new file mode 100644 index 0000000000000..0557ce6e868c4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats18.q new file mode 100644 index 0000000000000..425de64c26e83 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats19.q new file mode 100644 index 0000000000000..da4af9655d16a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats2.q new file mode 100644 index 0000000000000..d6bb258307d8e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats20.q new file mode 100644 index 0000000000000..59701bde771dc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats3.q new file mode 100644 index 0000000000000..5962348d9c317 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats4.q new file mode 100644 index 0000000000000..62580042d4ded --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats5.q new file mode 100644 index 0000000000000..6b5d1384e065a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats6.q new file mode 100644 index 0000000000000..bef6e8809ef97 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats7.q new file mode 100644 index 0000000000000..19d0e1426930e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats8.q new file mode 100644 index 0000000000000..46002caf4a0ec --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats9.q new file mode 100644 index 0000000000000..48b20443a9f1b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_aggregator_error_1.q new file mode 100644 index 0000000000000..4e7d3dc547a30 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_empty_dyn_part.q new file mode 100644 index 0000000000000..38ee6f7f4f3fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_empty_partition.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_empty_partition.q new file mode 100644 index 0000000000000..305a97c929230 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_1.q new file mode 100644 index 0000000000000..02b4c2aabf039 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_noscan_2.q new file mode 100644 index 0000000000000..c934fb2051258 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1.q new file mode 100644 index 0000000000000..cdf92e44cf676 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_partscan_1_23.q new file mode 100644 index 0000000000000..1e5f360b20cbb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/stats_publisher_error_1.q new file mode 100644 index 0000000000000..6d383f213d1be --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/str_to_map.q new file mode 100644 index 0000000000000..c3b206bba6325 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq.q new file mode 100644 index 0000000000000..3fb1558a90f0b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/subq2.q new file mode 100644 index 0000000000000..7e174da4586fd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/symlink_text_input_format.q new file mode 100644 index 0000000000000..bb9d6f34ed8ec --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/table_access_keys_stats.q new file mode 100644 index 0000000000000..8b1a390149865 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tablename_with_select.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/tablename_with_select.q new file mode 100644 index 0000000000000..c48231b66c7b8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/test_boolean_whereclause.q new file mode 100644 index 0000000000000..d2da5ac174a15 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_1.q new file mode 100644 index 0000000000000..f2c3b596af44a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_2.q new file mode 100644 index 0000000000000..b93208f48c453 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_3.q new file mode 100644 index 0000000000000..cda724f9e8f43 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_comparison.q new file mode 100644 index 0000000000000..f64ae48b85db7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_lazy.q new file mode 100644 index 0000000000000..7a1005295eb28 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_null.q new file mode 100644 index 0000000000000..efd5bc4b78bf4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/timestamp_udf.q new file mode 100644 index 0000000000000..2620acefee7f7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/touch.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/touch.q new file mode 100644 index 0000000000000..8a661ef12104e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform1.q new file mode 100644 index 0000000000000..962077c2ca565 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform2.q new file mode 100644 index 0000000000000..ddbc567596410 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform_ppr1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform_ppr1.q new file mode 100644 index 0000000000000..07bb4d1dd08e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform_ppr2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/transform_ppr2.q new file mode 100644 index 0000000000000..85ef3acf824d4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column.q new file mode 100644 index 0000000000000..d756b47e464d0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_buckets.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_buckets.q new file mode 100644 index 0000000000000..a2ce21558cdd7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_list_bucket.q new file mode 100644 index 0000000000000..87289f0d3d619 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_column_merge.q new file mode 100644 index 0000000000000..a7aab357eaf2f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/truncate_table.q new file mode 100644 index 0000000000000..c0e81e9ac051f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_cast_1.q new file mode 100644 index 0000000000000..a1607320c7a2d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_conversions_1.q new file mode 100644 index 0000000000000..63dd66ebedd0c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/type_widening.q new file mode 100644 index 0000000000000..0d36bc44fe08b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_collect_set.q new file mode 100644 index 0000000000000..45aaa022dc8a4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_context_ngrams.q new file mode 100644 index 0000000000000..dda7aaa5f2041 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_corr.q new file mode 100644 index 0000000000000..6cc9ce2630dd1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_pop.q new file mode 100644 index 0000000000000..0f5d5f35bf02a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_covar_samp.q new file mode 100644 index 0000000000000..72b9c4bd40049 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_histogram_numeric.q new file mode 100644 index 0000000000000..18bcd1c070a44 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_ngrams.q new file mode 100644 index 0000000000000..31ffd29a88acd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_number_format.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_number_format.q new file mode 100644 index 0000000000000..4f2ab453dc976 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_20.q new file mode 100644 index 0000000000000..66c408d71bc1a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udaf_percentile_approx_23.q new file mode 100644 index 0000000000000..07bfb6e1fb2ab --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf1.q new file mode 100644 index 0000000000000..2292c3f5ec271 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf2.q new file mode 100644 index 0000000000000..5aa99e85d170b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf3.q new file mode 100644 index 0000000000000..dcffeea8dd4c8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf4.q new file mode 100644 index 0000000000000..16f5b2cce2b03 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf5.q new file mode 100644 index 0000000000000..782e23dbfaefc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf6.q new file mode 100644 index 0000000000000..fd540d610a97e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf7.q new file mode 100644 index 0000000000000..d12394e2e7265 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf8.q new file mode 100644 index 0000000000000..a9e696369a047 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf9.q new file mode 100644 index 0000000000000..3b179b338e991 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_10_trims.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_10_trims.q new file mode 100644 index 0000000000000..8a7e3a520751e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_E.q new file mode 100644 index 0000000000000..113af61062b0d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_PI.q new file mode 100644 index 0000000000000..1fde7df5d251e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_abs.q new file mode 100644 index 0000000000000..f4f227d0dc168 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_acos.q new file mode 100644 index 0000000000000..625a2aa5c6aa6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_add.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_add.q new file mode 100644 index 0000000000000..dbc3940f450ea --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_add.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION +; +DESCRIBE FUNCTION EXTENDED +; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array.q new file mode 100644 index 0000000000000..fca8fe8d1c00c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_array_contains.q new file mode 100644 index 0000000000000..937bb0be03bd3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ascii.q new file mode 100644 index 0000000000000..53b389fd38fd8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_asin.q new file mode 100644 index 0000000000000..f95a5f57df8f8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_atan.q new file mode 100644 index 0000000000000..d4ef03deb81b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_avg.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_avg.q new file mode 100644 index 0000000000000..5c95bc407b63c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_avg.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION avg; +DESCRIBE FUNCTION EXTENDED avg; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_between.q new file mode 100644 index 0000000000000..eb3ccea82e631 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bigint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bigint.q new file mode 100644 index 0000000000000..e430866bf0952 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bigint.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION bigint; +DESCRIBE FUNCTION EXTENDED bigint; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bin.q new file mode 100644 index 0000000000000..2b9ad62a39dbe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_and.q new file mode 100644 index 0000000000000..7ea50dac31d07 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_empty.q new file mode 100644 index 0000000000000..88e961683e289 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitmap_or.q new file mode 100644 index 0000000000000..0b71e681a53d5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_and.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_and.q new file mode 100644 index 0000000000000..8e40111c1ff84 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_and.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION &; +DESCRIBE FUNCTION EXTENDED &; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_not.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_not.q new file mode 100644 index 0000000000000..fdd5485446520 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_not.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ~; +DESCRIBE FUNCTION EXTENDED ~; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_or.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_or.q new file mode 100644 index 0000000000000..55375db9be5d7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_or.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION |; +DESCRIBE FUNCTION EXTENDED |; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_xor.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_xor.q new file mode 100644 index 0000000000000..9c4ee7366232f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_bitwise_xor.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ^; +DESCRIBE FUNCTION EXTENDED ^; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_boolean.q new file mode 100644 index 0000000000000..1dc5017b340f7 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_boolean.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION boolean; +DESCRIBE FUNCTION EXTENDED boolean; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case.q new file mode 100644 index 0000000000000..4f71e70e1f3bf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_column_pruning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_column_pruning.q new file mode 100644 index 0000000000000..da925515b30da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_case_thrift.q new file mode 100644 index 0000000000000..736bb053cddc7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ceil.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ceil.q new file mode 100644 index 0000000000000..34a91e18500fb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ceil.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ceil; +DESCRIBE FUNCTION EXTENDED ceil; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ceiling.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ceiling.q new file mode 100644 index 0000000000000..d830b4db4f67f --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ceiling.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ceiling; +DESCRIBE FUNCTION EXTENDED ceiling; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_coalesce.q new file mode 100644 index 0000000000000..48ca29cbc3ba6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_compare_java_string.q new file mode 100644 index 0000000000000..6c12f81304e46 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat.q new file mode 100644 index 0000000000000..f642f6a2d00df --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_insert1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_insert1.q new file mode 100644 index 0000000000000..496f40372d088 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_insert2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_insert2.q new file mode 100644 index 0000000000000..6d1353feacd2d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_concat_ws.q new file mode 100644 index 0000000000000..6a0ce20dc37b7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_context_aware.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_context_aware.q new file mode 100644 index 0000000000000..04cb512f4a3eb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_conv.q new file mode 100644 index 0000000000000..212bcfb57938f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_cos.q new file mode 100644 index 0000000000000..7887c4c2603f1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_count.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_count.q new file mode 100644 index 0000000000000..022d7a9f8dd4d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_date_add.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_date_add.q new file mode 100644 index 0000000000000..52e19ed3f417d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_date_sub.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_date_sub.q new file mode 100644 index 0000000000000..9fdef94456d06 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_datediff.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_datediff.q new file mode 100644 index 0000000000000..30718fe8e4d5c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_datediff.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION datediff; +DESCRIBE FUNCTION EXTENDED datediff; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_day.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_day.q new file mode 100644 index 0000000000000..fb9b57da015bb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_day.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION day; +DESCRIBE FUNCTION EXTENDED day; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_dayofmonth.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_dayofmonth.q new file mode 100644 index 0000000000000..ab30f1d69b4af --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_dayofmonth.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION dayofmonth; +DESCRIBE FUNCTION EXTENDED dayofmonth; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_degrees.q new file mode 100644 index 0000000000000..014ca1c6e6afb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_div.q new file mode 100644 index 0000000000000..4229e625a042a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_divide.q new file mode 100644 index 0000000000000..dc4b2e7884d68 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_double.q new file mode 100644 index 0000000000000..974dbf1595513 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_double.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION double; +DESCRIBE FUNCTION EXTENDED double; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_elt.q new file mode 100644 index 0000000000000..c32340ac89feb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_equal.q new file mode 100644 index 0000000000000..f5e9a7df4be00 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_exp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_exp.q new file mode 100644 index 0000000000000..612c9a75760f9 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_exp.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION exp; +DESCRIBE FUNCTION EXTENDED exp; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_explode.q new file mode 100644 index 0000000000000..19af288ff8404 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_field.q new file mode 100644 index 0000000000000..e995f5cf3c084 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_find_in_set.q new file mode 100644 index 0000000000000..eac2c6e91c3d1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_float.q new file mode 100644 index 0000000000000..abad4ed136ad0 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_float.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION float; +DESCRIBE FUNCTION EXTENDED float; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_floor.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_floor.q new file mode 100644 index 0000000000000..a9c304b14fc7a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_floor.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION floor; +DESCRIBE FUNCTION EXTENDED floor; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_format_number.q new file mode 100644 index 0000000000000..e2084cddf0ec7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_from_unixtime.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_from_unixtime.q new file mode 100644 index 0000000000000..e67803703fa92 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_get_json_object.q new file mode 100644 index 0000000000000..464f2df3dcd7a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthan.q new file mode 100644 index 0000000000000..aea110a9431cf --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_greaterthanorequal.q new file mode 100644 index 0000000000000..8de165b9ea9ed --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hash.q new file mode 100644 index 0000000000000..faf372218a107 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hex.q new file mode 100644 index 0000000000000..37e035ad42b00 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_hour.q new file mode 100644 index 0000000000000..c5c366daa8c83 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_if.q new file mode 100644 index 0000000000000..4f7c8b4a36bad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in.q new file mode 100644 index 0000000000000..75778138742d5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_in_file.q new file mode 100644 index 0000000000000..4da478908ddbd --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q new file mode 100644 index 0000000000000..9079d0e5743f6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_index.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION `index`; +DESCRIBE FUNCTION EXTENDED `index`; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_inline.q new file mode 100644 index 0000000000000..39006f4b9b117 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_instr.q new file mode 100644 index 0000000000000..20ed8e4ea0176 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_int.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_int.q new file mode 100644 index 0000000000000..64e94a1dc32e1 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_int.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION int; +DESCRIBE FUNCTION EXTENDED int; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnotnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnotnull.q new file mode 100644 index 0000000000000..75228617ae4f8 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnotnull.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION isnotnull; +DESCRIBE FUNCTION EXTENDED isnotnull; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull.q new file mode 100644 index 0000000000000..3b5df163f55b2 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION isnull; +DESCRIBE FUNCTION EXTENDED isnull; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_isnull_isnotnull.q new file mode 100644 index 0000000000000..d1569cc7f6dbb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_java_method.q new file mode 100644 index 0000000000000..2f28be1e9d0e1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lcase.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lcase.q new file mode 100644 index 0000000000000..c2a09b0f6570c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lcase.q @@ -0,0 +1,3 @@ +DESCRIBE FUNCTION lcase; +DESCRIBE FUNCTION EXTENDED lcase; + diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_length.q new file mode 100644 index 0000000000000..b84307970d630 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_lessthan.q new file mode 100644 index 0000000000000..a2577006a885d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_notop.q new file mode 100644 index 0000000000000..88396545de933 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_nvl.q new file mode 100644 index 0000000000000..0133b4b90bcbc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_or.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_or.q new file mode 100644 index 0000000000000..fbd4486d49d6d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_or.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION or; +DESCRIBE FUNCTION EXTENDED or; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_parse_url.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_parse_url.q new file mode 100644 index 0000000000000..62c67db20b1a2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_percentile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_percentile.q new file mode 100644 index 0000000000000..936a514b1e7bc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pmod.q new file mode 100644 index 0000000000000..9ff73d42b41e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_positive.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_positive.q new file mode 100644 index 0000000000000..e5fd17e2a0a68 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pow.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pow.q new file mode 100644 index 0000000000000..dc9ad9ddea600 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_pow.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION pow; +DESCRIBE FUNCTION EXTENDED pow; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_power.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_power.q new file mode 100644 index 0000000000000..c5a3f40e5f3bb --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_power.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION power; +DESCRIBE FUNCTION EXTENDED power; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_printf.q new file mode 100644 index 0000000000000..99e89ccbf773a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_radians.q new file mode 100644 index 0000000000000..001d1cf8b6464 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rand.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rand.q new file mode 100644 index 0000000000000..ff5380c545414 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rand.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION rand; +DESCRIBE FUNCTION EXTENDED rand; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect.q new file mode 100644 index 0000000000000..f357ff5049564 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reflect2.q new file mode 100644 index 0000000000000..9ffd755292f66 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp.q new file mode 100644 index 0000000000000..3aee10963d747 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp_extract.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp_extract.q new file mode 100644 index 0000000000000..7d7560d44d1f7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp_replace.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_regexp_replace.q new file mode 100644 index 0000000000000..db536a1efa4b9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_repeat.q new file mode 100644 index 0000000000000..162085f4c71c8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_reverse.q new file mode 100644 index 0000000000000..81f765ec5937c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rlike.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rlike.q new file mode 100644 index 0000000000000..27581046a8f10 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rlike.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION rlike; +DESCRIBE FUNCTION EXTENDED rlike; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round.q new file mode 100644 index 0000000000000..18ebba8708493 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_2.q new file mode 100644 index 0000000000000..6be30855aea1b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_round_3.q new file mode 100644 index 0000000000000..50a1f44419047 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rpad.q new file mode 100644 index 0000000000000..01e5fbd429b17 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rtrim.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rtrim.q new file mode 100644 index 0000000000000..33d09df284624 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_rtrim.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION rtrim; +DESCRIBE FUNCTION EXTENDED rtrim; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_second.q new file mode 100644 index 0000000000000..1943188086ea5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sentences.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sentences.q new file mode 100644 index 0000000000000..009e4cb48a1d4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sign.q new file mode 100644 index 0000000000000..abceb3439ec1e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sin.q new file mode 100644 index 0000000000000..abb7cac8dac9d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_size.q new file mode 100644 index 0000000000000..8aaa68a6e318a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_smallint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_smallint.q new file mode 100644 index 0000000000000..202a401eb705d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_smallint.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION smallint; +DESCRIBE FUNCTION EXTENDED smallint; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sort_array.q new file mode 100644 index 0000000000000..ef0973212a27c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_space.q new file mode 100644 index 0000000000000..cf6466fb63bad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_split.q new file mode 100644 index 0000000000000..f79901736cf7c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sqrt.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sqrt.q new file mode 100644 index 0000000000000..aec061ddd7b9c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sqrt.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION sqrt; +DESCRIBE FUNCTION EXTENDED sqrt; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_std.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_std.q new file mode 100644 index 0000000000000..9a20256e3e5fc --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_std.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION std; +DESCRIBE FUNCTION EXTENDED std; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev.q new file mode 100644 index 0000000000000..b3b8794c9d585 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION stddev; +DESCRIBE FUNCTION EXTENDED stddev; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev_pop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev_pop.q new file mode 100644 index 0000000000000..4abede7af8b55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev_samp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_stddev_samp.q new file mode 100644 index 0000000000000..30d2f24513864 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_string.q new file mode 100644 index 0000000000000..96e433e1751c5 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_string.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION string; +DESCRIBE FUNCTION EXTENDED string; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_struct.q new file mode 100644 index 0000000000000..3ee19c83699bb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substr.q new file mode 100644 index 0000000000000..32757bef51c1c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substring.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_substring.q new file mode 100644 index 0000000000000..d56b828b8b5c6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_subtract.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_subtract.q new file mode 100644 index 0000000000000..3bc583493487a --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_subtract.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION -; +DESCRIBE FUNCTION EXTENDED -; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sum.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_sum.q new file mode 100644 index 0000000000000..325bcabd6c9aa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tan.q new file mode 100644 index 0000000000000..f103da9ecb2de --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength.q new file mode 100644 index 0000000000000..322a061d645d7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_testlength2.q new file mode 100644 index 0000000000000..6de270902f7fb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tinyint.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tinyint.q new file mode 100644 index 0000000000000..56ea53af7766d --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_tinyint.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION tinyint; +DESCRIBE FUNCTION EXTENDED tinyint; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_boolean.q new file mode 100644 index 0000000000000..ca23f719f93e3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_byte.q new file mode 100644 index 0000000000000..ded930d1115a4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_date.q new file mode 100644 index 0000000000000..935609bbc3e55 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_double.q new file mode 100644 index 0000000000000..b0a248ad70bba --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_float.q new file mode 100644 index 0000000000000..c91d18cc2f5c7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_long.q new file mode 100644 index 0000000000000..45dc6f8bd32fa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_short.q new file mode 100644 index 0000000000000..7d843c1ea12ea --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_string.q new file mode 100644 index 0000000000000..3b585e7170c60 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_to_unix_timestamp.q new file mode 100644 index 0000000000000..3024074bba4b3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_translate.q new file mode 100644 index 0000000000000..cba6ff90f643f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_trim.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_trim.q new file mode 100644 index 0000000000000..1fa34e00c90cf --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_trim.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION trim; +DESCRIBE FUNCTION EXTENDED trim; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ucase.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ucase.q new file mode 100644 index 0000000000000..a61b9ba67a4d6 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_ucase.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION ucase; +DESCRIBE FUNCTION EXTENDED ucase; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unhex.q new file mode 100644 index 0000000000000..e80021c8b2c51 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_union.q new file mode 100644 index 0000000000000..9140d22fb58c7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_unix_timestamp.q new file mode 100644 index 0000000000000..89288a1193c87 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_upper.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_upper.q new file mode 100644 index 0000000000000..2a9d5ed4d6c90 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_upper.q @@ -0,0 +1,2 @@ +DESCRIBE FUNCTION upper; +DESCRIBE FUNCTION EXTENDED upper; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_var_pop.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_var_pop.q new file mode 100644 index 0000000000000..7761506f28cbb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_var_samp.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_var_samp.q new file mode 100644 index 0000000000000..de68e499005ef --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_variance.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_variance.q new file mode 100644 index 0000000000000..579cc8e68f49c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_weekofyear.q new file mode 100644 index 0000000000000..4b7b4ea55aa81 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_when.q new file mode 100644 index 0000000000000..d94a594f026c5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath.q new file mode 100644 index 0000000000000..fca1ba11d85b2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_boolean.q new file mode 100644 index 0000000000000..3a6e613eb6a0e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_double.q new file mode 100644 index 0000000000000..4328747f44497 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_float.q new file mode 100644 index 0000000000000..1f1482833c858 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_int.q new file mode 100644 index 0000000000000..9b50bd9eeb365 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_long.q new file mode 100644 index 0000000000000..04ee61b1f0e9d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_short.q new file mode 100644 index 0000000000000..6a1abdc0bdb21 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udf_xpath_string.q new file mode 100644 index 0000000000000..ebbc913b71bfb --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_explode.q new file mode 100644 index 0000000000000..638a4e9ca57e7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_json_tuple.q new file mode 100644 index 0000000000000..712d9598c755d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_parse_url_tuple.q new file mode 100644 index 0000000000000..055e39b2973a6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_stack.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/udtf_stack.q new file mode 100644 index 0000000000000..e7cbae0605954 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/unicode_notation.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/unicode_notation.q new file mode 100644 index 0000000000000..b3397363e1943 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union.q new file mode 100644 index 0000000000000..91bbd1b07f08b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union10.q new file mode 100644 index 0000000000000..8a5c5f78c8469 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union11.q new file mode 100644 index 0000000000000..77dc2efec5692 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union12.q new file mode 100644 index 0000000000000..b0893df97be67 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union13.q new file mode 100644 index 0000000000000..8bee1d7993570 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union14.q new file mode 100644 index 0000000000000..4437ad8d01e49 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union15.q new file mode 100644 index 0000000000000..3080b077e9d35 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union16.q new file mode 100644 index 0000000000000..1df68b037ebe5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union17.q new file mode 100644 index 0000000000000..34b0e8cc6335f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union18.q new file mode 100644 index 0000000000000..6207730963173 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union19.q new file mode 100644 index 0000000000000..dc61b74d02f0a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union2.q new file mode 100644 index 0000000000000..581cbebd861e9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union20.q new file mode 100644 index 0000000000000..267262e371e18 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union21.q new file mode 100644 index 0000000000000..8185994655228 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union22.q new file mode 100644 index 0000000000000..d0b125682f84e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union23.q new file mode 100644 index 0000000000000..41dca46ad2369 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union24.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union24.q new file mode 100644 index 0000000000000..e8f0e14076080 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union25.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union25.q new file mode 100644 index 0000000000000..04e782e2a5e34 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union26.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union26.q new file mode 100644 index 0000000000000..dce7bc3425117 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union27.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union27.q new file mode 100644 index 0000000000000..e0fccfcd72820 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union28.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union28.q new file mode 100644 index 0000000000000..6aebfeb41a3c7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union29.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union29.q new file mode 100644 index 0000000000000..ea99d4c9f29be --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union3.q new file mode 100644 index 0000000000000..b26a2e2799f7a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union30.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union30.q new file mode 100644 index 0000000000000..9763bc3c9a9c3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union31.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union31.q new file mode 100644 index 0000000000000..1d430fe1de815 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union32.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union32.q new file mode 100644 index 0000000000000..9abb09bf76d40 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union33.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union33.q new file mode 100644 index 0000000000000..69e46f44d7806 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union34.q new file mode 100644 index 0000000000000..36bc865be81fe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union4.q new file mode 100644 index 0000000000000..ee13e7ad3dd45 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union5.q new file mode 100644 index 0000000000000..9844127e4a4bc --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union6.q new file mode 100644 index 0000000000000..b793002821147 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union7.q new file mode 100644 index 0000000000000..d66d5960bd187 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union8.q new file mode 100644 index 0000000000000..6d5bf673e0881 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union9.q new file mode 100644 index 0000000000000..7d4c11b252ab5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_date.q new file mode 100644 index 0000000000000..e332a8af7bb1b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_lateralview.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_lateralview.q new file mode 100644 index 0000000000000..012296d7c3a51 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_null.q new file mode 100644 index 0000000000000..70147070feaca --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_ppr.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_ppr.q new file mode 100644 index 0000000000000..27d2c262def40 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_1.q new file mode 100644 index 0000000000000..c9f920c648a92 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_10.q new file mode 100644 index 0000000000000..a072fe3b56e43 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_11.q new file mode 100644 index 0000000000000..6250a20210f9f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_12.q new file mode 100644 index 0000000000000..168eac34a0301 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_13.q new file mode 100644 index 0000000000000..a88a83e152a5f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_14.q new file mode 100644 index 0000000000000..e588e8fd62ab9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_15.q new file mode 100644 index 0000000000000..237f2e7629057 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_16.q new file mode 100644 index 0000000000000..06d5043f9cf9d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_17.q new file mode 100644 index 0000000000000..65b8255c3e954 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_18.q new file mode 100644 index 0000000000000..265acfd35758f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_19.q new file mode 100644 index 0000000000000..1450caa2d07a7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_2.q new file mode 100644 index 0000000000000..015c146ecadf0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_20.q new file mode 100644 index 0000000000000..ac727889273d7 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_21.q new file mode 100644 index 0000000000000..f89744624116f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_22.q new file mode 100644 index 0000000000000..f01053d27d0c0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_23.q new file mode 100644 index 0000000000000..805dd76274c42 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_24.q new file mode 100644 index 0000000000000..36fd947d6035d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_3.q new file mode 100644 index 0000000000000..da0f1c0473ffe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_4.q new file mode 100644 index 0000000000000..18d4730a12024 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_5.q new file mode 100644 index 0000000000000..a6fbeb03ddcc3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_6.q new file mode 100644 index 0000000000000..7ae5af30c1bae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_7.q new file mode 100644 index 0000000000000..5a639ca11700d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_8.q new file mode 100644 index 0000000000000..79b84e81181d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_remove_9.q new file mode 100644 index 0000000000000..f6038384f3fa8 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_script.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_script.q new file mode 100644 index 0000000000000..df35859f597c5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/union_view.q new file mode 100644 index 0000000000000..4f8bafe37a809 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/uniquejoin.q new file mode 100644 index 0000000000000..51bcf22bfa0be --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/unset_table_view_property.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/unset_table_view_property.q new file mode 100644 index 0000000000000..f838cd1feb106 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/updateAccessTime.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/updateAccessTime.q new file mode 100644 index 0000000000000..908b6ecfa0778 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_1.q new file mode 100644 index 0000000000000..53273b3187a78 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_2.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_2.q new file mode 100644 index 0000000000000..1eb3b85d6a3b3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_cast.q new file mode 100644 index 0000000000000..550f3dc728df2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_comparison.q new file mode 100644 index 0000000000000..b6c6f406412fe --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_join1.q new file mode 100644 index 0000000000000..6a19efaa3c5a1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_nested_types.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_nested_types.q new file mode 100644 index 0000000000000..5c0989f7fc5ae --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_serde.q new file mode 100644 index 0000000000000..7351b688cf504 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_udf1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_udf1.q new file mode 100644 index 0000000000000..0a3012b5cd6d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/varchar_union1.q new file mode 100644 index 0000000000000..cf90eab33e2d0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view.q new file mode 100644 index 0000000000000..4e3d0572921d6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_cast.q new file mode 100644 index 0000000000000..b0b078ec628b3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_inputs.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/view_inputs.q new file mode 100644 index 0000000000000..7acc321e6f7d3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/virtual_column.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/virtual_column.q new file mode 100644 index 0000000000000..30fde395ebb04 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing.q new file mode 100644 index 0000000000000..a7297db6104c0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_adjust_rowcontainer_sz.q new file mode 100644 index 0000000000000..9c7625dcd786a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_columnPruning.q new file mode 100644 index 0000000000000..7c4ab386145d5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_expressions.q new file mode 100644 index 0000000000000..2c3339095f22b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_multipartitioning.q new file mode 100644 index 0000000000000..bb371e900975b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_navfn.q new file mode 100644 index 0000000000000..8a9d0012593e6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_ntile.q new file mode 100644 index 0000000000000..505c259f4b7b3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_rank.q new file mode 100644 index 0000000000000..bf76867813e4e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_udaf.q new file mode 100644 index 0000000000000..f22b992cd4386 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q b/sql/hive/src/test/resources/ql/src/test/queries/clientpositive/windowing_windowspec.q new file mode 100644 index 0000000000000..7cc1367306956 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/ambiguous_join_col.q new file mode 100644 index 0000000000000..e70aae46275b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/duplicate_alias.q new file mode 100644 index 0000000000000..5fd22460c037a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/garbage.q new file mode 100644 index 0000000000000..6c8c751f21c31 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/insert_wrong_number_columns.q new file mode 100644 index 0000000000000..aadfbde33836a --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q new file mode 100644 index 0000000000000..899bbd368b185 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_create_table.q @@ -0,0 +1,4 @@ +CREATE TABLE mytable ( + a INT + b STRING +); diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_dot.q new file mode 100644 index 0000000000000..36b9bd2a3b980 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_function_param2.q new file mode 100644 index 0000000000000..3543449b8870b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_index.q new file mode 100644 index 0000000000000..146bc5dc9f3b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index.q new file mode 100644 index 0000000000000..c40f079f60aaa --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_list_index2.q new file mode 100644 index 0000000000000..99d0b3d4162ac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index.q new file mode 100644 index 0000000000000..c2b9eab61b80e --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_map_index2.q new file mode 100644 index 0000000000000..5828f0709f53b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q new file mode 100644 index 0000000000000..fd1298577be8c --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/negative/invalid_select.q @@ -0,0 +1,4 @@ +SELECT + trim(trim(a)) + trim(b) +FROM src; diff --git a/sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/macro_reserved_word.q new file mode 100644 index 0000000000000..359eb9de93bac --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/missing_overwrite.q new file mode 100644 index 0000000000000..1bfeee382ea37 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/nonkey_groupby.q new file mode 100644 index 0000000000000..ad0f4415cbd8c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/quoted_string.q new file mode 100644 index 0000000000000..0252a9e11cdf5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column1.q new file mode 100644 index 0000000000000..429cead63beb1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column2.q new file mode 100644 index 0000000000000..3767dc4e65021 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column3.q new file mode 100644 index 0000000000000..2fc5f490f118d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column4.q new file mode 100644 index 0000000000000..8ad8dd12e46e2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column5.q new file mode 100644 index 0000000000000..766b0e5255fea --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_column6.q new file mode 100644 index 0000000000000..bb76c28623484 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function1.q new file mode 100644 index 0000000000000..d8ff6325b95f2 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function2.q new file mode 100644 index 0000000000000..f7d255934db5c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function3.q new file mode 100644 index 0000000000000..87d4edc987865 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_function4.q new file mode 100644 index 0000000000000..cfe70e4f2fdc4 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table1.q new file mode 100644 index 0000000000000..585ef6d7f2db1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/unknown_table2.q new file mode 100644 index 0000000000000..2c69c16be5905 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct1.q new file mode 100755 index 0000000000000..d92c3bb8df4b0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q b/sql/hive/src/test/resources/ql/src/test/queries/negative/wrong_distinct2.q new file mode 100755 index 0000000000000..53fb550b3d111 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/case_sensitivity.q new file mode 100644 index 0000000000000..d7f7371507662 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/cast1.q new file mode 100644 index 0000000000000..6269c6a4e76f9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby1.q new file mode 100755 index 0000000000000..96b29b05cc7ad --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby2.q new file mode 100755 index 0000000000000..d741eb60b6bb1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby3.q new file mode 100755 index 0000000000000..03b1248a11cb6 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby4.q new file mode 100755 index 0000000000000..85271a9caf6e3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby5.q new file mode 100755 index 0000000000000..ebd65b3069722 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q new file mode 100755 index 0000000000000..80654f2a9ce6b --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/positive/groupby6.q @@ -0,0 +1,2 @@ +FROM src +SELECT DISTINCT substr(src.value,5,1) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input1.q new file mode 100644 index 0000000000000..fdd290d6b136b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input2.q new file mode 100644 index 0000000000000..4e1612ea972e9 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input20.q new file mode 100644 index 0000000000000..f30cf27017d96 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input3.q new file mode 100644 index 0000000000000..fc53e94d39f02 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input4.q new file mode 100644 index 0000000000000..03e6de48faca3 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input5.q new file mode 100644 index 0000000000000..a46abc75833f0 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input6.q new file mode 100644 index 0000000000000..d6f25a935ae7d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q new file mode 100644 index 0000000000000..33a82953c26ed --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/positive/input7.q @@ -0,0 +1,2 @@ +FROM src1 +INSERT OVERWRITE TABLE dest1 SELECT NULL, src1.key diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q new file mode 100644 index 0000000000000..0843b9ba4e557 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/positive/input8.q @@ -0,0 +1,2 @@ +FROM src1 +SELECT 4 + NULL, src1.key - NULL, NULL + NULL diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input9.q new file mode 100644 index 0000000000000..2892f0b2dfc4f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_part1.q new file mode 100644 index 0000000000000..d45d1cd0b47ec --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testsequencefile.q new file mode 100755 index 0000000000000..cf9a092417e1d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath.q new file mode 100755 index 0000000000000..7699bff755520 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/input_testxpath2.q new file mode 100644 index 0000000000000..08abaf4fad8da --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join1.q new file mode 100644 index 0000000000000..739c39dd8f718 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join2.q new file mode 100644 index 0000000000000..a02d87f09f58c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join3.q new file mode 100644 index 0000000000000..b57c9569d7286 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join4.q new file mode 100644 index 0000000000000..2e5967fb7d854 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join5.q new file mode 100644 index 0000000000000..63a38f554a24f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join6.q new file mode 100644 index 0000000000000..110451cf3039b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join7.q new file mode 100644 index 0000000000000..65797b44a2cb1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/join8.q new file mode 100644 index 0000000000000..d215b07a6720f --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample1.q new file mode 100644 index 0000000000000..3a168b999d707 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample2.q new file mode 100644 index 0000000000000..b505b896fa2c5 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample3.q new file mode 100644 index 0000000000000..42d5a2bbec34b --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample4.q new file mode 100644 index 0000000000000..7b5ab03380ae1 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample5.q new file mode 100644 index 0000000000000..b9b48fdc71884 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample6.q new file mode 100644 index 0000000000000..0ee026f0f368c --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/sample7.q new file mode 100644 index 0000000000000..f17ce105c3572 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/subq.q new file mode 100644 index 0000000000000..6392dbcc4380d --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf1.q new file mode 100644 index 0000000000000..2ecf46e742c30 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf4.q new file mode 100644 index 0000000000000..f3a7598e17210 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q new file mode 100644 index 0000000000000..65791c41c1ff4 --- /dev/null +++ b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf6.q @@ -0,0 +1 @@ +FROM src SELECT CONCAT('a', 'b'), IF(TRUE, 1 ,2) diff --git a/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_case.q new file mode 100644 index 0000000000000..0c86da2198699 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/udf_when.q new file mode 100644 index 0000000000000..99ed09990b874 --- /dev/null +++ b/sql/hive/src/test/resources/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/sql/hive/src/test/resources/ql/src/test/queries/positive/union.q b/sql/hive/src/test/resources/ql/src/test/queries/positive/union.q new file mode 100644 index 0000000000000..6a6b9882aee71 --- /dev/null +++ b/sql/hive/src/test/resources/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.* From 2c0aa22e2e26ae35b7d4caa529bc6520e362cc3c Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 21 Mar 2014 16:07:22 -0700 Subject: [PATCH 118/397] SPARK-1279: Fix improper use of SimpleDateFormat `SimpleDateFormat` is not thread-safe. Some places use the same SimpleDateFormat object without safeguard in the multiple threads. It will cause that the Web UI displays improper date. This PR creates a new `SimpleDateFormat` every time when it's necessary. Another solution is using `ThreadLocal` to store a `SimpleDateFormat` in each thread. If this PR impacts the performance, I can change to the latter one. Author: zsxwing Closes #179 from zsxwing/SPARK-1278 and squashes the following commits: 21fabd3 [zsxwing] SPARK-1278: Fix improper use of SimpleDateFormat --- .../org/apache/spark/deploy/master/Master.scala | 6 +++--- .../apache/spark/deploy/master/ui/IndexPage.scala | 8 ++++---- .../org/apache/spark/deploy/worker/Worker.scala | 4 ++-- .../org/apache/spark/scheduler/JobLogger.scala | 6 ++++-- .../org/apache/spark/{deploy => ui}/WebUI.scala | 13 ++++++++----- .../org/apache/spark/ui/jobs/JobProgressUI.scala | 2 -- .../scala/org/apache/spark/ui/jobs/StagePage.scala | 5 ++--- .../scala/org/apache/spark/ui/jobs/StageTable.scala | 5 ++--- .../scala/org/apache/spark/util/FileLogger.scala | 7 +++++-- 9 files changed, 30 insertions(+), 26 deletions(-) rename core/src/main/scala/org/apache/spark/{deploy => ui}/WebUI.scala (74%) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 9ed49e01be639..95bd62e88db2b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -51,7 +51,7 @@ private[spark] class Master( val conf = new SparkConf - val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs + def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For application IDs val WORKER_TIMEOUT = conf.getLong("spark.worker.timeout", 60) * 1000 val RETAINED_APPLICATIONS = conf.getInt("spark.deploy.retainedApplications", 200) val REAPER_ITERATIONS = conf.getInt("spark.dead.worker.persistence", 15) @@ -682,7 +682,7 @@ private[spark] class Master( /** Generate a new app ID given a app's submission date */ def newApplicationId(submitDate: Date): String = { - val appId = "app-%s-%04d".format(DATE_FORMAT.format(submitDate), nextAppNumber) + val appId = "app-%s-%04d".format(createDateFormat.format(submitDate), nextAppNumber) nextAppNumber += 1 appId } @@ -706,7 +706,7 @@ private[spark] class Master( } def newDriverId(submitDate: Date): String = { - val appId = "driver-%s-%04d".format(DATE_FORMAT.format(submitDate), nextDriverNumber) + val appId = "driver-%s-%04d".format(createDateFormat.format(submitDate), nextDriverNumber) nextDriverNumber += 1 appId } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala index 7ec71eb80bfc0..8c1d6c7cce450 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala @@ -25,10 +25,10 @@ import scala.xml.Node import akka.pattern.ask import org.json4s.JValue -import org.apache.spark.deploy.{DeployWebUI, JsonProtocol} +import org.apache.spark.deploy.{JsonProtocol} import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUI, UIUtils} import org.apache.spark.util.Utils private[spark] class IndexPage(parent: MasterWebUI) { @@ -169,10 +169,10 @@ private[spark] class IndexPage(parent: MasterWebUI) {

    - + - + } diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 5e0fc31fff22f..8a71ddda4cb5e 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -56,7 +56,7 @@ private[spark] class Worker( Utils.checkHost(host, "Expected hostname") assert (port > 0) - val DATE_FORMAT = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs + def createDateFormat = new SimpleDateFormat("yyyyMMddHHmmss") // For worker and executor IDs // Send a heartbeat every (heartbeat timeout) / 4 milliseconds val HEARTBEAT_MILLIS = conf.getLong("spark.worker.timeout", 60) * 1000 / 4 @@ -319,7 +319,7 @@ private[spark] class Worker( } def generateWorkerId(): String = { - "worker-%s-%s-%d".format(DATE_FORMAT.format(new Date), host, port) + "worker-%s-%s-%d".format(createDateFormat.format(new Date), host, port) } override def postStop() { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index b3a67d7e17976..5cecf9416b32c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -55,7 +55,9 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener private val jobIdToPrintWriter = new HashMap[Int, PrintWriter] private val stageIdToJobId = new HashMap[Int, Int] private val jobIdToStageIds = new HashMap[Int, Seq[Int]] - private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val dateFormat = new ThreadLocal[SimpleDateFormat]() { + override def initialValue() = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + } private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent] createLogDir() @@ -128,7 +130,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener var writeInfo = info if (withTime) { val date = new Date(System.currentTimeMillis()) - writeInfo = DATE_FORMAT.format(date) + ": " + info + writeInfo = dateFormat.get.format(date) + ": " + info } jobIdToPrintWriter.get(jobId).foreach(_.println(writeInfo)) } diff --git a/core/src/main/scala/org/apache/spark/deploy/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala similarity index 74% rename from core/src/main/scala/org/apache/spark/deploy/WebUI.scala rename to core/src/main/scala/org/apache/spark/ui/WebUI.scala index ae258b58b9cc5..a7b872f3445a4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.deploy +package org.apache.spark.ui import java.text.SimpleDateFormat import java.util.Date @@ -23,12 +23,15 @@ import java.util.Date /** * Utilities used throughout the web UI. */ -private[spark] object DeployWebUI { - val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") +private[spark] object WebUI { + // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. + private val dateFormat = new ThreadLocal[SimpleDateFormat]() { + override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + } - def formatDate(date: Date): String = DATE_FORMAT.format(date) + def formatDate(date: Date): String = dateFormat.get.format(date) - def formatDate(timestamp: Long): String = DATE_FORMAT.format(new Date(timestamp)) + def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) def formatDuration(milliseconds: Long): String = { val seconds = milliseconds.toDouble / 1000 diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index ee4e9c69c1bd1..b2c67381cc3da 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -17,7 +17,6 @@ package org.apache.spark.ui.jobs -import java.text.SimpleDateFormat import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.servlet.ServletContextHandler @@ -32,7 +31,6 @@ import org.apache.spark.util.Utils private[ui] class JobProgressUI(parent: SparkUI) { val appName = parent.appName val basePath = parent.basePath - val dateFmt = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") val live = parent.live val sc = parent.sc diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index da7f20233063e..0c55f2ee7e944 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -23,14 +23,13 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUI, UIUtils} import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressUI) { private val appName = parent.appName private val basePath = parent.basePath - private val dateFmt = parent.dateFmt private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { @@ -253,7 +252,7 @@ private[ui] class StagePage(parent: JobProgressUI) { - + diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 68fef5234c9ab..5bf1c95cd5530 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -23,13 +23,12 @@ import scala.collection.mutable.HashMap import scala.xml.Node import org.apache.spark.scheduler.{StageInfo, TaskInfo} -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUI, UIUtils} import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { private val basePath = parent.basePath - private val dateFmt = parent.dateFmt private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler @@ -82,7 +81,7 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { val description = listener.stageIdToDescription.get(s.stageId) .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) val submissionTime = s.submissionTime match { - case Some(t) => dateFmt.format(new Date(t)) + case Some(t) => WebUI.formatDate(new Date(t)) case None => "Unknown" } val finishTime = s.completionTime.getOrElse(System.currentTimeMillis) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index f07962096a32c..a0c07e32fdc98 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -44,7 +44,10 @@ class FileLogger( overwrite: Boolean = true) extends Logging { - private val DATE_FORMAT = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + private val dateFormat = new ThreadLocal[SimpleDateFormat]() { + override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + } + private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) private var fileIndex = 0 @@ -111,7 +114,7 @@ class FileLogger( def log(msg: String, withTime: Boolean = false) { val writeInfo = if (!withTime) msg else { val date = new Date(System.currentTimeMillis()) - DATE_FORMAT.format(date) + ": " + msg + dateFormat.get.format(date) + ": " + msg } writer.foreach(_.print(writeInfo)) } From dab5439a083b5f771d5d5b462d0d517fa8e9aaf2 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 21 Mar 2014 16:53:18 -0700 Subject: [PATCH 119/397] Make SQL keywords case-insensitive This is a bit of a hack that allows all variations of a keyword, but it still seems to produce valid error messages and such. Author: Matei Zaharia Closes #193 from mateiz/case-insensitive-sql and squashes the following commits: 0ee4ace [Matei Zaharia] Removed unnecessary `+ ""` e3ed773 [Matei Zaharia] Make SQL keywords case-insensitive --- .../apache/spark/sql/catalyst/SqlParser.scala | 16 ++++++++++-- .../org/apache/spark/sql/SQLQuerySuite.scala | 25 ++++++++++++++++--- 2 files changed, 35 insertions(+), 6 deletions(-) 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 index d3b1070a58d58..919bf4dbc824c 100644 --- 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 @@ -51,7 +51,9 @@ class SqlParser extends StandardTokenParsers { } protected case class Keyword(str: String) - protected implicit def asParser(k: Keyword): Parser[String] = k.str + + protected implicit def asParser(k: Keyword): Parser[String] = + allCaseVersions(k.str).map(x => x : Parser[String]).reduce(_ | _) protected class SqlLexical extends StdLexical { case class FloatLit(chars: String) extends Token { @@ -133,7 +135,17 @@ class SqlParser extends StandardTokenParsers { .filter(_.getReturnType == classOf[Keyword]) .map(_.invoke(this).asInstanceOf[Keyword]) - lexical.reserved ++= reservedWords.map(_.str) + /** Generate all variations of upper and lower case of a given string */ + private def allCaseVersions(s: String, prefix: String = ""): Stream[String] = { + if (s == "") { + Stream(prefix) + } else { + allCaseVersions(s.tail, prefix + s.head.toLower) ++ + allCaseVersions(s.tail, prefix + s.head.toUpper) + } + } + + lexical.reserved ++= reservedWords.flatMap(w => allCaseVersions(w.str)) lexical.delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 656d89c644b70..6371fa296ac45 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -200,9 +200,9 @@ class SQLQuerySuite extends QueryTest { sql( """ |SELECT * FROM - | (SELECT * FROM upperCaseData WHERE N <= 4) left FULL OUTER JOIN - | (SELECT * FROM upperCaseData WHERE N >= 3) right - | ON left.N = right.N + | (SELECT * FROM upperCaseData WHERE N <= 4) leftTable FULL OUTER JOIN + | (SELECT * FROM upperCaseData WHERE N >= 3) rightTable + | ON leftTable.N = rightTable.N """.stripMargin), (1, "A", null, null) :: (2, "B", null, null) :: @@ -211,4 +211,21 @@ class SQLQuerySuite extends QueryTest { (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) } -} \ No newline at end of file + + test("mixed-case keywords") { + checkAnswer( + sql( + """ + |SeleCT * from + | (select * from upperCaseData WherE N <= 4) leftTable fuLL OUtER joiN + | (sElEcT * FROM upperCaseData whERe N >= 3) rightTable + | oN leftTable.N = rightTable.N + """.stripMargin), + (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) + } +} From d780983647376147ab8a54477dab83ed55676d34 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 21 Mar 2014 16:54:06 -0700 Subject: [PATCH 120/397] Add asCode function for dumping raw tree representations. Intended only for use by Catalyst developers. Author: Michael Armbrust Closes #200 from marmbrus/asCode and squashes the following commits: 7e8c1d9 [Michael Armbrust] Add asCode function for dumping raw tree representations. Intended only for use by Catalyst developers. --- .../spark/sql/catalyst/trees/TreeNode.scala | 15 +++++++++++++++ 1 file changed, 15 insertions(+) diff --git a/sql/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 index 76ede87e4e6c0..37e557441dc63 100644 --- a/sql/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 @@ -336,6 +336,21 @@ abstract class TreeNode[BaseType <: TreeNode[BaseType]] { children.foreach(_.generateTreeString(depth + 1, builder)) builder } + + /** + * Returns a 'scala code' representation of this `TreeNode` and its children. Intended for use + * when debugging where the prettier toString function is obfuscating the actual structure. In the + * case of 'pure' `TreeNodes` that only contain primitives and other TreeNodes, the result can be + * pasted in the REPL to build an equivalent Tree. + */ + def asCode: String = { + val args = productIterator.map { + case tn: TreeNode[_] => tn.asCode + case s: String => "\"" + s + "\"" + case other => other.toString + } + s"$nodeName(${args.mkString(",")})" + } } /** From 646e55405b433fdedc9601dab91f99832b641f87 Mon Sep 17 00:00:00 2001 From: Emtiaz Ahmed Date: Fri, 21 Mar 2014 18:05:53 -0700 Subject: [PATCH 121/397] Fix to Stage UI to display numbers on progress bar Fixes an issue on Stage UI to display numbers on progress bar which are today hidden behind the progress bar div. Please refer to the attached images to see the issue. ![screen shot 2014-03-21 at 4 48 46 pm](https://f.cloud.github.com/assets/563652/2489083/8c127e80-b153-11e3-807c-048ebd45104b.png) ![screen shot 2014-03-21 at 4 49 00 pm](https://f.cloud.github.com/assets/563652/2489084/8c12cf5c-b153-11e3-8747-9d93ff6fceb4.png) Author: Emtiaz Ahmed Closes #201 from emtiazahmed/master and squashes the following commits: a7964fe [Emtiaz Ahmed] Fix to Stage UI to display numbers on progress bar --- core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 5bf1c95cd5530..ac61568af52d2 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -63,7 +63,7 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { val startWidth = "width: %s%%".format((started.toDouble/total)*100)
    - + {completed}/{total} {failed}
    From abf6714e27cf07a13819b35a4ca50ff9bb28b65c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 23 Mar 2014 10:57:01 -0700 Subject: [PATCH 122/397] SPARK-1254. Supplemental fix for HTTPS on Maven Central It seems that HTTPS does not necessarily work on Maven Central, as it does not today at least. Back to HTTP. Both builds works from a clean repo. Author: Sean Owen Closes #209 from srowen/SPARK-1254Fix and squashes the following commits: bb7be47 [Sean Owen] Revert to HTTP for Maven Central repo, as it seems HTTPS does not necessarily work --- pom.xml | 3 ++- project/SparkBuild.scala | 3 ++- 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index 9db34a01ba87e..fa72d5f263e97 100644 --- a/pom.xml +++ b/pom.xml @@ -132,7 +132,8 @@ maven-repo Maven Repository - https://repo.maven.apache.org/maven2 + + http://repo.maven.apache.org/maven2 true diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index e4ad65912523e..b08fb26adfe68 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -182,7 +182,8 @@ object SparkBuild extends Build { concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), resolvers ++= Seq( - "Maven Repository" at "https://repo.maven.apache.org/maven2", + // HTTPS is unavailable for Maven Central + "Maven Repository" at "http://repo.maven.apache.org/maven2", "Apache Repository" at "https://repository.apache.org/content/repositories/releases", "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", From 57a4379c031e5d5901ba580422207d6aa2f19749 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 23 Mar 2014 12:08:55 -0700 Subject: [PATCH 123/397] [SPARK-1292] In-memory columnar representation for Spark SQL This PR is rebased from the Catalyst repository, and contains the first version of in-memory columnar representation for Spark SQL. Compression support is not included yet and will be added later in a separate PR. Author: Cheng Lian Author: Cheng Lian Closes #205 from liancheng/memColumnarSupport and squashes the following commits: 99dba41 [Cheng Lian] Restricted new objects/classes to `private[sql]' 0892ad8 [Cheng Lian] Addressed ScalaStyle issues af1ad5e [Cheng Lian] Fixed some minor issues introduced during rebasing 0dbf2fb [Cheng Lian] Make necessary renaming due to rebase a162d4d [Cheng Lian] Removed the unnecessary InMemoryColumnarRelation class 9bcae4b [Cheng Lian] Added Apache license 220ee1e [Cheng Lian] Added table scan operator for in-memory columnar support. c701c7a [Cheng Lian] Using SparkSqlSerializer for generic object SerDe causes error, made a workaround ed8608e [Cheng Lian] Added implicit conversion from DataType to ColumnType b8a645a [Cheng Lian] Replaced KryoSerializer with an updated SparkSqlSerializer b6c0a49 [Cheng Lian] Minor test suite refactoring 214be73 [Cheng Lian] Refactored BINARY and GENERIC to reduce duplicate code da2f4d5 [Cheng Lian] Added Apache license dbf7a38 [Cheng Lian] Added ColumnAccessor and test suite, refactored ColumnBuilder c01a177 [Cheng Lian] Added column builder classes and test suite f18ddc6 [Cheng Lian] Added ColumnTypes and test suite 2d09066 [Cheng Lian] Added KryoSerializer 34f3c19 [Cheng Lian] Added TypeTag field to all NativeTypes acc5c48 [Cheng Lian] Added Hive test files to .gitignore --- .../spark/sql/catalyst/types/dataTypes.scala | 14 +- .../spark/sql/columnar/ColumnAccessor.scala | 175 +++++++++++++++ .../spark/sql/columnar/ColumnBuilder.scala | 187 ++++++++++++++++ .../spark/sql/columnar/ColumnType.scala | 198 +++++++++++++++++ .../sql/columnar/NullableColumnAccessor.scala | 57 +++++ .../sql/columnar/NullableColumnBuilder.scala | 83 +++++++ .../columnar/inMemoryColumnarOperators.scala | 80 +++++++ .../apache/spark/sql/execution/Exchange.scala | 33 --- .../sql/execution/SparkSqlSerializer.scala | 73 +++++++ .../org/apache/spark/sql/QueryTest.scala | 2 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 204 ++++++++++++++++++ .../sql/columnar/ColumnarQuerySuite.scala | 34 +++ .../spark/sql/columnar/ColumnarTestData.scala | 55 +++++ .../NullableColumnAccessorSuite.scala | 61 ++++++ .../columnar/NullableColumnBuilderSuite.scala | 94 ++++++++ 15 files changed, 1315 insertions(+), 35 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala diff --git a/sql/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 index 6eb2b62eccb48..90a9f9f7e548d 100644 --- a/sql/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 @@ -19,7 +19,9 @@ package org.apache.spark.sql package catalyst package types -import expressions.Expression +import scala.reflect.runtime.universe.{typeTag, TypeTag} + +import org.apache.spark.sql.catalyst.expressions.Expression abstract class DataType { /** Matches any expression that evaluates to this DataType */ @@ -33,11 +35,13 @@ case object NullType extends DataType abstract class NativeType extends DataType { type JvmType + @transient val tag: TypeTag[JvmType] val ordering: Ordering[JvmType] } case object StringType extends NativeType { type JvmType = String + @transient lazy val tag = typeTag[JvmType] val ordering = implicitly[Ordering[JvmType]] } case object BinaryType extends DataType { @@ -45,6 +49,7 @@ case object BinaryType extends DataType { } case object BooleanType extends NativeType { type JvmType = Boolean + @transient lazy val tag = typeTag[JvmType] val ordering = implicitly[Ordering[JvmType]] } @@ -71,6 +76,7 @@ abstract class IntegralType extends NumericType { case object LongType extends IntegralType { type JvmType = Long + @transient lazy val tag = typeTag[JvmType] val numeric = implicitly[Numeric[Long]] val integral = implicitly[Integral[Long]] val ordering = implicitly[Ordering[JvmType]] @@ -78,6 +84,7 @@ case object LongType extends IntegralType { case object IntegerType extends IntegralType { type JvmType = Int + @transient lazy val tag = typeTag[JvmType] val numeric = implicitly[Numeric[Int]] val integral = implicitly[Integral[Int]] val ordering = implicitly[Ordering[JvmType]] @@ -85,6 +92,7 @@ case object IntegerType extends IntegralType { case object ShortType extends IntegralType { type JvmType = Short + @transient lazy val tag = typeTag[JvmType] val numeric = implicitly[Numeric[Short]] val integral = implicitly[Integral[Short]] val ordering = implicitly[Ordering[JvmType]] @@ -92,6 +100,7 @@ case object ShortType extends IntegralType { case object ByteType extends IntegralType { type JvmType = Byte + @transient lazy val tag = typeTag[JvmType] val numeric = implicitly[Numeric[Byte]] val integral = implicitly[Integral[Byte]] val ordering = implicitly[Ordering[JvmType]] @@ -110,6 +119,7 @@ abstract class FractionalType extends NumericType { case object DecimalType extends FractionalType { type JvmType = BigDecimal + @transient lazy val tag = typeTag[JvmType] val numeric = implicitly[Numeric[BigDecimal]] val fractional = implicitly[Fractional[BigDecimal]] val ordering = implicitly[Ordering[JvmType]] @@ -117,6 +127,7 @@ case object DecimalType extends FractionalType { case object DoubleType extends FractionalType { type JvmType = Double + @transient lazy val tag = typeTag[JvmType] val numeric = implicitly[Numeric[Double]] val fractional = implicitly[Fractional[Double]] val ordering = implicitly[Ordering[JvmType]] @@ -124,6 +135,7 @@ case object DoubleType extends FractionalType { case object FloatType extends FractionalType { type JvmType = Float + @transient lazy val tag = typeTag[JvmType] val numeric = implicitly[Numeric[Float]] val fractional = implicitly[Fractional[Float]] val ordering = implicitly[Ordering[JvmType]] diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala new file mode 100644 index 0000000000000..ddbeba6203aa4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -0,0 +1,175 @@ +/* + * 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 columnar + +import java.nio.{ByteOrder, ByteBuffer} + +import org.apache.spark.sql.catalyst.types.{BinaryType, NativeType, DataType} +import org.apache.spark.sql.catalyst.expressions.MutableRow +import org.apache.spark.sql.execution.SparkSqlSerializer + +/** + * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is + * extracted from the buffer, instead of directly returning it, the value is set into some field of + * a [[MutableRow]]. In this way, boxing cost can be avoided by leveraging the setter methods + * for primitive values provided by [[MutableRow]]. + */ +private[sql] trait ColumnAccessor { + initialize() + + protected def initialize() + + def hasNext: Boolean + + def extractTo(row: MutableRow, ordinal: Int) + + protected def underlyingBuffer: ByteBuffer +} + +private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType](buffer: ByteBuffer) + extends ColumnAccessor { + + protected def initialize() {} + + def columnType: ColumnType[T, JvmType] + + def hasNext = buffer.hasRemaining + + def extractTo(row: MutableRow, ordinal: Int) { + doExtractTo(row, ordinal) + } + + protected def doExtractTo(row: MutableRow, ordinal: Int) + + protected def underlyingBuffer = buffer +} + +private[sql] abstract class NativeColumnAccessor[T <: NativeType]( + buffer: ByteBuffer, + val columnType: NativeColumnType[T]) + extends BasicColumnAccessor[T, T#JvmType](buffer) + with NullableColumnAccessor + +private[sql] class BooleanColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, BOOLEAN) { + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + row.setBoolean(ordinal, columnType.extract(buffer)) + } +} + +private[sql] class IntColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, INT) { + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + row.setInt(ordinal, columnType.extract(buffer)) + } +} + +private[sql] class ShortColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, SHORT) { + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + row.setShort(ordinal, columnType.extract(buffer)) + } +} + +private[sql] class LongColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, LONG) { + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + row.setLong(ordinal, columnType.extract(buffer)) + } +} + +private[sql] class ByteColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, BYTE) { + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + row.setByte(ordinal, columnType.extract(buffer)) + } +} + +private[sql] class DoubleColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, DOUBLE) { + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + row.setDouble(ordinal, columnType.extract(buffer)) + } +} + +private[sql] class FloatColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, FLOAT) { + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + row.setFloat(ordinal, columnType.extract(buffer)) + } +} + +private[sql] class StringColumnAccessor(buffer: ByteBuffer) + extends NativeColumnAccessor(buffer, STRING) { + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + row.setString(ordinal, columnType.extract(buffer)) + } +} + +private[sql] class BinaryColumnAccessor(buffer: ByteBuffer) + extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer) + with NullableColumnAccessor { + + def columnType = BINARY + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + row(ordinal) = columnType.extract(buffer) + } +} + +private[sql] class GenericColumnAccessor(buffer: ByteBuffer) + extends BasicColumnAccessor[DataType, Array[Byte]](buffer) + with NullableColumnAccessor { + + def columnType = GENERIC + + override protected def doExtractTo(row: MutableRow, ordinal: Int) { + val serialized = columnType.extract(buffer) + row(ordinal) = SparkSqlSerializer.deserialize[Any](serialized) + } +} + +private[sql] object ColumnAccessor { + def apply(b: ByteBuffer): ColumnAccessor = { + // The first 4 bytes in the buffer indicates the column type. + val buffer = b.duplicate().order(ByteOrder.nativeOrder()) + val columnTypeId = buffer.getInt() + + columnTypeId match { + case INT.typeId => new IntColumnAccessor(buffer) + case LONG.typeId => new LongColumnAccessor(buffer) + case FLOAT.typeId => new FloatColumnAccessor(buffer) + case DOUBLE.typeId => new DoubleColumnAccessor(buffer) + case BOOLEAN.typeId => new BooleanColumnAccessor(buffer) + case BYTE.typeId => new ByteColumnAccessor(buffer) + case SHORT.typeId => new ShortColumnAccessor(buffer) + case STRING.typeId => new StringColumnAccessor(buffer) + case BINARY.typeId => new BinaryColumnAccessor(buffer) + case GENERIC.typeId => new GenericColumnAccessor(buffer) + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala new file mode 100644 index 0000000000000..6bd1841821875 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -0,0 +1,187 @@ +/* + * 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 columnar + +import java.nio.{ByteOrder, ByteBuffer} + +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.execution.SparkSqlSerializer + +private[sql] trait ColumnBuilder { + /** + * Initializes with an approximate lower bound on the expected number of elements in this column. + */ + def initialize(initialSize: Int, columnName: String = "") + + def appendFrom(row: Row, ordinal: Int) + + def build(): ByteBuffer +} + +private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends ColumnBuilder { + import ColumnBuilder._ + + private var columnName: String = _ + protected var buffer: ByteBuffer = _ + + def columnType: ColumnType[T, JvmType] + + override def initialize(initialSize: Int, columnName: String = "") = { + val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize + this.columnName = columnName + buffer = ByteBuffer.allocate(4 + 4 + size * columnType.defaultSize) + buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId) + } + + // Have to give a concrete implementation to make mixin possible + override def appendFrom(row: Row, ordinal: Int) { + doAppendFrom(row, ordinal) + } + + // Concrete `ColumnBuilder`s can override this method to append values + protected def doAppendFrom(row: Row, ordinal: Int) + + // Helper method to append primitive values (to avoid boxing cost) + protected def appendValue(v: JvmType) { + buffer = ensureFreeSpace(buffer, columnType.actualSize(v)) + columnType.append(v, buffer) + } + + override def build() = { + buffer.limit(buffer.position()).rewind() + buffer + } +} + +private[sql] abstract class NativeColumnBuilder[T <: NativeType]( + val columnType: NativeColumnType[T]) + extends BasicColumnBuilder[T, T#JvmType] + with NullableColumnBuilder + +private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(BOOLEAN) { + override def doAppendFrom(row: Row, ordinal: Int) { + appendValue(row.getBoolean(ordinal)) + } +} + +private[sql] class IntColumnBuilder extends NativeColumnBuilder(INT) { + override def doAppendFrom(row: Row, ordinal: Int) { + appendValue(row.getInt(ordinal)) + } +} + +private[sql] class ShortColumnBuilder extends NativeColumnBuilder(SHORT) { + override def doAppendFrom(row: Row, ordinal: Int) { + appendValue(row.getShort(ordinal)) + } +} + +private[sql] class LongColumnBuilder extends NativeColumnBuilder(LONG) { + override def doAppendFrom(row: Row, ordinal: Int) { + appendValue(row.getLong(ordinal)) + } +} + +private[sql] class ByteColumnBuilder extends NativeColumnBuilder(BYTE) { + override def doAppendFrom(row: Row, ordinal: Int) { + appendValue(row.getByte(ordinal)) + } +} + +private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(DOUBLE) { + override def doAppendFrom(row: Row, ordinal: Int) { + appendValue(row.getDouble(ordinal)) + } +} + +private[sql] class FloatColumnBuilder extends NativeColumnBuilder(FLOAT) { + override def doAppendFrom(row: Row, ordinal: Int) { + appendValue(row.getFloat(ordinal)) + } +} + +private[sql] class StringColumnBuilder extends NativeColumnBuilder(STRING) { + override def doAppendFrom(row: Row, ordinal: Int) { + appendValue(row.getString(ordinal)) + } +} + +private[sql] class BinaryColumnBuilder + extends BasicColumnBuilder[BinaryType.type, Array[Byte]] + with NullableColumnBuilder { + + def columnType = BINARY + + override def doAppendFrom(row: Row, ordinal: Int) { + appendValue(row(ordinal).asInstanceOf[Array[Byte]]) + } +} + +// TODO (lian) Add support for array, struct and map +private[sql] class GenericColumnBuilder + extends BasicColumnBuilder[DataType, Array[Byte]] + with NullableColumnBuilder { + + def columnType = GENERIC + + override def doAppendFrom(row: Row, ordinal: Int) { + val serialized = SparkSqlSerializer.serialize(row(ordinal)) + buffer = ColumnBuilder.ensureFreeSpace(buffer, columnType.actualSize(serialized)) + columnType.append(serialized, buffer) + } +} + +private[sql] object ColumnBuilder { + val DEFAULT_INITIAL_BUFFER_SIZE = 10 * 1024 * 104 + + private[columnar] def ensureFreeSpace(orig: ByteBuffer, size: Int) = { + if (orig.remaining >= size) { + orig + } else { + // grow in steps of initial size + val capacity = orig.capacity() + val newSize = capacity + size.max(capacity / 8 + 1) + val pos = orig.position() + + orig.clear() + ByteBuffer + .allocate(newSize) + .order(ByteOrder.nativeOrder()) + .put(orig.array(), 0, pos) + } + } + + def apply(typeId: Int, initialSize: Int = 0, columnName: String = ""): ColumnBuilder = { + val builder = (typeId match { + case INT.typeId => new IntColumnBuilder + case LONG.typeId => new LongColumnBuilder + case FLOAT.typeId => new FloatColumnBuilder + case DOUBLE.typeId => new DoubleColumnBuilder + case BOOLEAN.typeId => new BooleanColumnBuilder + case BYTE.typeId => new ByteColumnBuilder + case SHORT.typeId => new ShortColumnBuilder + case STRING.typeId => new StringColumnBuilder + case BINARY.typeId => new BinaryColumnBuilder + case GENERIC.typeId => new GenericColumnBuilder + }).asInstanceOf[ColumnBuilder] + + builder.initialize(initialSize, columnName) + builder + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala new file mode 100644 index 0000000000000..3b759a51cc695 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -0,0 +1,198 @@ +/* + * 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 columnar + +import java.nio.ByteBuffer + +import org.apache.spark.sql.catalyst.types._ + +/** + * An abstract class that represents type of a column. Used to append/extract Java objects into/from + * the underlying [[ByteBuffer]] of a column. + * + * @param typeId A unique ID representing the type. + * @param defaultSize Default size in bytes for one element of type T (e.g. 4 for `Int`). + * @tparam T Scala data type for the column. + * @tparam JvmType Underlying Java type to represent the elements. + */ +private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( + val typeId: Int, + val defaultSize: Int) { + + /** + * Extracts a value out of the buffer at the buffer's current position. + */ + def extract(buffer: ByteBuffer): JvmType + + /** + * Appends the given value v of type T into the given ByteBuffer. + */ + def append(v: JvmType, buffer: ByteBuffer) + + /** + * Returns the size of the value. This is used to calculate the size of variable length types + * such as byte arrays and strings. + */ + def actualSize(v: JvmType): Int = defaultSize + + /** + * Creates a duplicated copy of the value. + */ + def clone(v: JvmType): JvmType = v +} + +private[sql] abstract class NativeColumnType[T <: NativeType]( + val dataType: T, + typeId: Int, + defaultSize: Int) + extends ColumnType[T, T#JvmType](typeId, defaultSize) { + + /** + * Scala TypeTag. Can be used to create primitive arrays and hash tables. + */ + def scalaTag = dataType.tag +} + +private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { + def append(v: Int, buffer: ByteBuffer) { + buffer.putInt(v) + } + + def extract(buffer: ByteBuffer) = { + buffer.getInt() + } +} + +private[sql] object LONG extends NativeColumnType(LongType, 1, 8) { + override def append(v: Long, buffer: ByteBuffer) { + buffer.putLong(v) + } + + override def extract(buffer: ByteBuffer) = { + buffer.getLong() + } +} + +private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) { + override def append(v: Float, buffer: ByteBuffer) { + buffer.putFloat(v) + } + + override def extract(buffer: ByteBuffer) = { + buffer.getFloat() + } +} + +private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) { + override def append(v: Double, buffer: ByteBuffer) { + buffer.putDouble(v) + } + + override def extract(buffer: ByteBuffer) = { + buffer.getDouble() + } +} + +private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) { + override def append(v: Boolean, buffer: ByteBuffer) { + buffer.put(if (v) 1.toByte else 0.toByte) + } + + override def extract(buffer: ByteBuffer) = { + if (buffer.get() == 1) true else false + } +} + +private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) { + override def append(v: Byte, buffer: ByteBuffer) { + buffer.put(v) + } + + override def extract(buffer: ByteBuffer) = { + buffer.get() + } +} + +private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { + override def append(v: Short, buffer: ByteBuffer) { + buffer.putShort(v) + } + + override def extract(buffer: ByteBuffer) = { + buffer.getShort() + } +} + +private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { + override def actualSize(v: String): Int = v.getBytes.length + 4 + + override def append(v: String, buffer: ByteBuffer) { + val stringBytes = v.getBytes() + buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) + } + + override def extract(buffer: ByteBuffer) = { + val length = buffer.getInt() + val stringBytes = new Array[Byte](length) + buffer.get(stringBytes, 0, length) + new String(stringBytes) + } +} + +private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( + typeId: Int, + defaultSize: Int) + extends ColumnType[T, Array[Byte]](typeId, defaultSize) { + + override def actualSize(v: Array[Byte]) = v.length + 4 + + override def append(v: Array[Byte], buffer: ByteBuffer) { + buffer.putInt(v.length).put(v, 0, v.length) + } + + override def extract(buffer: ByteBuffer) = { + val length = buffer.getInt() + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + bytes + } +} + +private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) + +// Used to process generic objects (all types other than those listed above). Objects should be +// serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized +// byte array. +private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) + +private[sql] object ColumnType { + implicit def dataTypeToColumnType(dataType: DataType): ColumnType[_, _] = { + dataType match { + case IntegerType => INT + case LongType => LONG + case FloatType => FLOAT + case DoubleType => DOUBLE + case BooleanType => BOOLEAN + case ByteType => BYTE + case ShortType => SHORT + case StringType => STRING + case BinaryType => BINARY + case _ => GENERIC + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala new file mode 100644 index 0000000000000..2970c609b928d --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala @@ -0,0 +1,57 @@ +/* + * 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.columnar + +import java.nio.{ByteOrder, ByteBuffer} + +import org.apache.spark.sql.catalyst.expressions.MutableRow + +private[sql] trait NullableColumnAccessor extends ColumnAccessor { + private var nullsBuffer: ByteBuffer = _ + private var nullCount: Int = _ + private var seenNulls: Int = 0 + + private var nextNullIndex: Int = _ + private var pos: Int = 0 + + abstract override def initialize() { + nullsBuffer = underlyingBuffer.duplicate().order(ByteOrder.nativeOrder()) + nullCount = nullsBuffer.getInt() + nextNullIndex = if (nullCount > 0) nullsBuffer.getInt() else -1 + pos = 0 + + underlyingBuffer.position(underlyingBuffer.position + 4 + nullCount * 4) + super.initialize() + } + + abstract override def extractTo(row: MutableRow, ordinal: Int) { + if (pos == nextNullIndex) { + seenNulls += 1 + + if (seenNulls < nullCount) { + nextNullIndex = nullsBuffer.getInt() + } + + row.setNullAt(ordinal) + } else { + super.extractTo(row, ordinal) + } + + pos += 1 + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala new file mode 100644 index 0000000000000..1661c3f3ff4a9 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -0,0 +1,83 @@ +/* + * 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 columnar + +import java.nio.{ByteOrder, ByteBuffer} + +/** + * Builds a nullable column. The byte buffer of a nullable column contains: + * - 4 bytes for the null count (number of nulls) + * - positions for each null, in ascending order + * - the non-null data (column data type, compression type, data...) + */ +private[sql] trait NullableColumnBuilder extends ColumnBuilder { + private var nulls: ByteBuffer = _ + private var pos: Int = _ + private var nullCount: Int = _ + + abstract override def initialize(initialSize: Int, columnName: String) { + nulls = ByteBuffer.allocate(1024) + nulls.order(ByteOrder.nativeOrder()) + pos = 0 + nullCount = 0 + super.initialize(initialSize, columnName) + } + + abstract override def appendFrom(row: Row, ordinal: Int) { + if (row.isNullAt(ordinal)) { + nulls = ColumnBuilder.ensureFreeSpace(nulls, 4) + nulls.putInt(pos) + nullCount += 1 + } else { + super.appendFrom(row, ordinal) + } + pos += 1 + } + + abstract override def build(): ByteBuffer = { + val nonNulls = super.build() + val typeId = nonNulls.getInt() + val nullDataLen = nulls.position() + + nulls.limit(nullDataLen) + nulls.rewind() + + // Column type ID is moved to the front, follows the null count, then non-null data + // + // +---------+ + // | 4 bytes | Column type ID + // +---------+ + // | 4 bytes | Null count + // +---------+ + // | ... | Null positions (if null count is not zero) + // +---------+ + // | ... | Non-null part (without column type ID) + // +---------+ + val buffer = ByteBuffer + .allocate(4 + nullDataLen + nonNulls.limit) + .order(ByteOrder.nativeOrder()) + .putInt(typeId) + .putInt(nullCount) + .put(nulls) + .put(nonNulls) + + buffer.rewind() + buffer + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala new file mode 100644 index 0000000000000..c7efd30e87da4 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala @@ -0,0 +1,80 @@ +/* + * 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 columnar + +import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} +import org.apache.spark.sql.execution.{SparkPlan, LeafNode} + +private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan) + extends LeafNode { + + // For implicit conversion from `DataType` to `ColumnType` + import ColumnType._ + + override def output: Seq[Attribute] = attributes + + lazy val cachedColumnBuffers = { + val output = child.output + val cached = child.execute().mapPartitions { iterator => + val columnBuilders = output.map { a => + ColumnBuilder(a.dataType.typeId, 0, a.name) + }.toArray + + var row: Row = null + while (iterator.hasNext) { + row = iterator.next() + var i = 0 + while (i < row.length) { + columnBuilders(i).appendFrom(row, i) + i += 1 + } + } + + Iterator.single(columnBuilders.map(_.build())) + }.cache() + + cached.setName(child.toString) + // Force the materialization of the cached RDD. + cached.count() + cached + } + + override def execute() = { + cachedColumnBuffers.mapPartitions { iterator => + val columnBuffers = iterator.next() + assert(!iterator.hasNext) + + new Iterator[Row] { + val columnAccessors = columnBuffers.map(ColumnAccessor(_)) + val nextRow = new GenericMutableRow(columnAccessors.length) + + override def next() = { + var i = 0 + while (i < nextRow.length) { + columnAccessors(i).extractTo(nextRow, i) + i += 1 + } + nextRow + } + + override def hasNext = columnAccessors.head.hasNext + } + } + } +} 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 72dc5ec6ad33c..e934c4cf69ab5 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,14 +18,8 @@ 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 @@ -33,33 +27,6 @@ import catalyst.errors._ import catalyst.expressions._ import catalyst.plans.physical._ -private 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 - } -} - -private 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 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala new file mode 100644 index 0000000000000..ad7cd58b6aaaf --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -0,0 +1,73 @@ +/* + * 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 + +import java.nio.ByteBuffer + +import com.esotericsoftware.kryo.io.{Input, Output} +import com.esotericsoftware.kryo.{Serializer, Kryo} + +import org.apache.spark.{SparkEnv, SparkConf} +import org.apache.spark.serializer.KryoSerializer +import org.apache.spark.util.MutablePair + +class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { + override def newKryo(): Kryo = { + val kryo = new Kryo() + kryo.setRegistrationRequired(false) + 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 + } +} + +object SparkSqlSerializer { + // TODO (lian) Using KryoSerializer here is workaround, needs further investigation + // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization + // related error. + @transient lazy val ser: KryoSerializer = { + val sparkConf = Option(SparkEnv.get).map(_.conf).getOrElse(new SparkConf()) + new KryoSerializer(sparkConf) + } + + def serialize[T](o: T): Array[Byte] = { + ser.newInstance().serialize(o).array() + } + + def deserialize[T](bytes: Array[Byte]): T = { + ser.newInstance().deserialize[T](ByteBuffer.wrap(bytes)) + } +} + +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()) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 728fecededeb0..aa84211648db0 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -33,7 +33,7 @@ import TestSQLContext._ class QueryTest extends FunSuite { /** * Runs the plan and makes sure the answer matches the expected result. - * @param plan the query to be executed + * @param rdd the [[SchemaRDD]] to be executed * @param expectedAnswer the expected result, can either be an Any, Seq[Product], or Seq[ Seq[Any] ]. */ protected def checkAnswer(rdd: SchemaRDD, expectedAnswer: Any): Unit = { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala new file mode 100644 index 0000000000000..c7aaaae94ee5e --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -0,0 +1,204 @@ +package org.apache.spark.sql +package columnar + +import java.nio.ByteBuffer + +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.execution.SparkSqlSerializer + +class ColumnTypeSuite extends FunSuite { + val columnTypes = Seq(INT, SHORT, LONG, BYTE, DOUBLE, FLOAT, STRING, BINARY, GENERIC) + + test("defaultSize") { + val defaultSize = Seq(4, 2, 8, 1, 8, 4, 8, 16, 16) + + columnTypes.zip(defaultSize).foreach { case (columnType, size) => + assert(columnType.defaultSize === size) + } + } + + test("actualSize") { + val expectedSizes = Seq(4, 2, 8, 1, 8, 4, 4 + 5, 4 + 4, 4 + 11) + val actualSizes = Seq( + INT.actualSize(Int.MaxValue), + SHORT.actualSize(Short.MaxValue), + LONG.actualSize(Long.MaxValue), + BYTE.actualSize(Byte.MaxValue), + DOUBLE.actualSize(Double.MaxValue), + FLOAT.actualSize(Float.MaxValue), + STRING.actualSize("hello"), + BINARY.actualSize(new Array[Byte](4)), + GENERIC.actualSize(SparkSqlSerializer.serialize(Map(1 -> "a")))) + + expectedSizes.zip(actualSizes).foreach { case (expected, actual) => + assert(expected === actual) + } + } + + testNumericColumnType[BooleanType.type, Boolean]( + BOOLEAN, + Array.fill(4)(Random.nextBoolean()), + ByteBuffer.allocate(32), + (buffer: ByteBuffer, v: Boolean) => { + buffer.put((if (v) 1 else 0).toByte) + }, + (buffer: ByteBuffer) => { + buffer.get() == 1 + }) + + testNumericColumnType[IntegerType.type, Int]( + INT, + Array.fill(4)(Random.nextInt()), + ByteBuffer.allocate(32), + (_: ByteBuffer).putInt(_), + (_: ByteBuffer).getInt) + + testNumericColumnType[ShortType.type, Short]( + SHORT, + Array.fill(4)(Random.nextInt(Short.MaxValue).asInstanceOf[Short]), + ByteBuffer.allocate(32), + (_: ByteBuffer).putShort(_), + (_: ByteBuffer).getShort) + + testNumericColumnType[LongType.type, Long]( + LONG, + Array.fill(4)(Random.nextLong()), + ByteBuffer.allocate(64), + (_: ByteBuffer).putLong(_), + (_: ByteBuffer).getLong) + + testNumericColumnType[ByteType.type, Byte]( + BYTE, + Array.fill(4)(Random.nextInt(Byte.MaxValue).asInstanceOf[Byte]), + ByteBuffer.allocate(64), + (_: ByteBuffer).put(_), + (_: ByteBuffer).get) + + testNumericColumnType[DoubleType.type, Double]( + DOUBLE, + Array.fill(4)(Random.nextDouble()), + ByteBuffer.allocate(64), + (_: ByteBuffer).putDouble(_), + (_: ByteBuffer).getDouble) + + testNumericColumnType[FloatType.type, Float]( + FLOAT, + Array.fill(4)(Random.nextFloat()), + ByteBuffer.allocate(64), + (_: ByteBuffer).putFloat(_), + (_: ByteBuffer).getFloat) + + test("STRING") { + val buffer = ByteBuffer.allocate(128) + val seq = Array("hello", "world", "spark", "sql") + + seq.map(_.getBytes).foreach { bytes: Array[Byte] => + buffer.putInt(bytes.length).put(bytes) + } + + buffer.rewind() + seq.foreach { s => + assert(s === STRING.extract(buffer)) + } + + buffer.rewind() + seq.foreach(STRING.append(_, buffer)) + + buffer.rewind() + seq.foreach { s => + val length = buffer.getInt + assert(length === s.getBytes.length) + + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + assert(s === new String(bytes)) + } + } + + test("BINARY") { + val buffer = ByteBuffer.allocate(128) + val seq = Array.fill(4) { + val bytes = new Array[Byte](4) + Random.nextBytes(bytes) + bytes + } + + seq.foreach { bytes => + buffer.putInt(bytes.length).put(bytes) + } + + buffer.rewind() + seq.foreach { b => + assert(b === BINARY.extract(buffer)) + } + + buffer.rewind() + seq.foreach(BINARY.append(_, buffer)) + + buffer.rewind() + seq.foreach { b => + val length = buffer.getInt + assert(length === b.length) + + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + assert(b === bytes) + } + } + + test("GENERIC") { + val buffer = ByteBuffer.allocate(512) + val obj = Map(1 -> "spark", 2 -> "sql") + val serializedObj = SparkSqlSerializer.serialize(obj) + + GENERIC.append(SparkSqlSerializer.serialize(obj), buffer) + buffer.rewind() + + val length = buffer.getInt() + assert(length === serializedObj.length) + + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + assert(obj === SparkSqlSerializer.deserialize(bytes)) + + buffer.rewind() + buffer.putInt(serializedObj.length).put(serializedObj) + + buffer.rewind() + assert(obj === SparkSqlSerializer.deserialize(GENERIC.extract(buffer))) + } + + def testNumericColumnType[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType], + seq: Seq[JvmType], + buffer: ByteBuffer, + putter: (ByteBuffer, JvmType) => Unit, + getter: (ByteBuffer) => JvmType) { + + val columnTypeName = columnType.getClass.getSimpleName.stripSuffix("$") + + test(s"$columnTypeName.extract") { + buffer.rewind() + seq.foreach(putter(buffer, _)) + + buffer.rewind() + seq.foreach { i => + assert(i === columnType.extract(buffer)) + } + } + + test(s"$columnTypeName.append") { + buffer.rewind() + seq.foreach(columnType.append(_, buffer)) + + buffer.rewind() + seq.foreach { i => + assert(i === getter(buffer)) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala new file mode 100644 index 0000000000000..928851a385d41 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.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.columnar + +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.{TestData, DslQuerySuite} + +class ColumnarQuerySuite extends DslQuerySuite { + import TestData._ + import TestSQLContext._ + + test("simple columnar query") { + val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + + checkAnswer(scan, testData.collect().toSeq) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala new file mode 100644 index 0000000000000..ddcdede8d1a4a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala @@ -0,0 +1,55 @@ +/* + * 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.columnar + +import scala.util.Random + +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow + +// TODO Enrich test data +object ColumnarTestData { + object GenericMutableRow { + def apply(values: Any*) = { + val row = new GenericMutableRow(values.length) + row.indices.foreach { i => + row(i) = values(i) + } + row + } + } + + def randomBytes(length: Int) = { + val bytes = new Array[Byte](length) + Random.nextBytes(bytes) + bytes + } + + val nonNullRandomRow = GenericMutableRow( + Random.nextInt(), + Random.nextLong(), + Random.nextFloat(), + Random.nextDouble(), + Random.nextBoolean(), + Random.nextInt(Byte.MaxValue).asInstanceOf[Byte], + Random.nextInt(Short.MaxValue).asInstanceOf[Short], + Random.nextString(Random.nextInt(64)), + randomBytes(Random.nextInt(64)), + Map(Random.nextInt() -> Random.nextString(4))) + + val nullRow = GenericMutableRow(Seq.fill(10)(null): _*) +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala new file mode 100644 index 0000000000000..279607ccfaa5b --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -0,0 +1,61 @@ +/* + * 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 columnar + +import org.scalatest.FunSuite +import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow + +class NullableColumnAccessorSuite extends FunSuite { + import ColumnarTestData._ + + Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { + testNullableColumnAccessor(_) + } + + def testNullableColumnAccessor[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + + test(s"$typeName accessor: empty column") { + val builder = ColumnBuilder(columnType.typeId, 4) + val accessor = ColumnAccessor(builder.build()) + assert(!accessor.hasNext) + } + + test(s"$typeName accessor: access null values") { + val builder = ColumnBuilder(columnType.typeId, 4) + + (0 until 4).foreach { _ => + builder.appendFrom(nonNullRandomRow, columnType.typeId) + builder.appendFrom(nullRow, columnType.typeId) + } + + val accessor = ColumnAccessor(builder.build()) + val row = new GenericMutableRow(1) + + (0 until 4).foreach { _ => + accessor.extractTo(row, 0) + assert(row(0) === nonNullRandomRow(columnType.typeId)) + + accessor.extractTo(row, 0) + assert(row(0) === null) + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala new file mode 100644 index 0000000000000..3354da3fa3e0f --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -0,0 +1,94 @@ +/* + * 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 columnar + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.execution.SparkSqlSerializer + +class NullableColumnBuilderSuite extends FunSuite { + import ColumnarTestData._ + + Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { + testNullableColumnBuilder(_) + } + + def testNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { + val columnBuilder = ColumnBuilder(columnType.typeId) + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + + test(s"$typeName column builder: empty column") { + columnBuilder.initialize(4) + + val buffer = columnBuilder.build() + + // For column type ID + assert(buffer.getInt() === columnType.typeId) + // For null count + assert(buffer.getInt === 0) + assert(!buffer.hasRemaining) + } + + test(s"$typeName column builder: buffer size auto growth") { + columnBuilder.initialize(4) + + (0 until 4) foreach { _ => + columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId) + } + + val buffer = columnBuilder.build() + + // For column type ID + assert(buffer.getInt() === columnType.typeId) + // For null count + assert(buffer.getInt() === 0) + } + + test(s"$typeName column builder: null values") { + columnBuilder.initialize(4) + + (0 until 4) foreach { _ => + columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId) + columnBuilder.appendFrom(nullRow, columnType.typeId) + } + + val buffer = columnBuilder.build() + + // For column type ID + assert(buffer.getInt() === columnType.typeId) + // For null count + assert(buffer.getInt() === 4) + // For null positions + (1 to 7 by 2).foreach(i => assert(buffer.getInt() === i)) + + // For non-null values + (0 until 4).foreach { _ => + val actual = if (columnType == GENERIC) { + SparkSqlSerializer.deserialize[Any](GENERIC.extract(buffer)) + } else { + columnType.extract(buffer) + } + assert(actual === nonNullRandomRow(columnType.typeId)) + } + + assert(!buffer.hasRemaining) + } + } +} From 8265dc7739caccc59bc2456b2df055ca96337fe4 Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Sun, 23 Mar 2014 15:21:40 -0700 Subject: [PATCH 124/397] Fixed coding style issues in Spark SQL This PR addresses various coding style issues in Spark SQL, including but not limited to those mentioned by @mateiz in PR #146. As this PR affects lots of source files and may cause potential conflicts, it would be better to merge this as soon as possible *after* PR #205 (In-memory columnar representation for Spark SQL) is merged. Author: Cheng Lian Closes #208 from liancheng/fixCodingStyle and squashes the following commits: fc2b528 [Cheng Lian] Merge branch 'master' into fixCodingStyle b531273 [Cheng Lian] Fixed coding style issues in sql/hive 0b56f77 [Cheng Lian] Fixed coding style issues in sql/core fae7b02 [Cheng Lian] Addressed styling issues mentioned by @marmbrus 9265366 [Cheng Lian] Fixed coding style issues in sql/core 3dcbbbd [Cheng Lian] Fixed relative package imports for package catalyst --- .../spark/sql/catalyst/ScalaReflection.scala | 69 +++++++++++++++++++ .../apache/spark/sql/catalyst/SqlParser.scala | 28 ++++---- .../sql/catalyst/analysis/Analyzer.scala | 7 +- .../spark/sql/catalyst/analysis/Catalog.scala | 3 +- .../catalyst/analysis/FunctionRegistry.scala | 2 +- .../catalyst/analysis/HiveTypeCoercion.scala | 8 +-- .../analysis/MultiInstanceRelation.scala | 8 +-- .../spark/sql/catalyst/analysis/package.scala | 1 + .../sql/catalyst/analysis/unresolved.scala | 8 +-- .../spark/sql/catalyst/dsl/package.scala | 56 ++------------- .../spark/sql/catalyst/errors/package.scala | 7 +- .../catalyst/expressions/BoundAttribute.scala | 7 +- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/Expression.scala | 6 +- .../spark/sql/catalyst/expressions/Rand.scala | 2 +- .../spark/sql/catalyst/expressions/Row.scala | 2 +- .../sql/catalyst/expressions/ScalaUdf.scala | 2 +- .../catalyst/expressions/WrapDynamic.scala | 2 +- .../sql/catalyst/expressions/aggregates.scala | 2 +- .../sql/catalyst/expressions/arithmetic.scala | 4 +- .../catalyst/expressions/complexTypes.scala | 2 +- .../sql/catalyst/expressions/generators.scala | 2 +- .../sql/catalyst/expressions/literals.scala | 2 +- .../expressions/namedExpressions.scala | 6 +- .../catalyst/expressions/nullFunctions.scala | 2 +- .../sql/catalyst/expressions/predicates.scala | 4 +- .../expressions/stringOperations.scala | 2 +- .../sql/catalyst/optimizer/Optimizer.scala | 25 +++---- .../sql/catalyst/planning/QueryPlanner.scala | 5 +- .../sql/catalyst/planning/patterns.scala | 4 +- .../spark/sql/catalyst/plans/QueryPlan.scala | 4 +- .../catalyst/plans/logical/LogicalPlan.scala | 6 +- .../plans/logical/ScriptTransformation.scala | 2 +- .../catalyst/plans/logical/TestRelation.scala | 3 +- .../plans/logical/basicOperators.scala | 2 +- .../catalyst/plans/logical/partitioning.scala | 2 +- .../plans/physical/partitioning.scala | 4 +- .../spark/sql/catalyst/rules/Rule.scala | 2 +- .../sql/catalyst/rules/RuleExecutor.scala | 14 ++-- .../spark/sql/catalyst/trees/TreeNode.scala | 2 +- .../{ => analysis}/AnalysisSuite.scala | 16 ++--- .../HiveTypeCoercionSuite.scala | 3 +- .../ExpressionEvaluationSuite.scala | 5 +- .../optimizer/ConstantFoldingSuite.scala | 19 ++--- .../optimizer/FilterPushdownSuite.scala | 13 ++-- .../catalyst/optimizer/OptimizerTest.scala | 12 ++-- .../{ => trees}/RuleExecutorSuite.scala | 6 +- .../catalyst/{ => trees}/TreeNodeSuite.scala | 19 +++-- .../rdd/PartitionLocalRDDFunctions.scala | 7 +- .../org/apache/spark/sql/SQLContext.scala | 5 +- .../org/apache/spark/sql/SchemaRDD.scala | 4 +- .../apache/spark/sql/execution/Exchange.scala | 21 +++--- .../apache/spark/sql/execution/Generate.scala | 3 +- .../spark/sql/execution/SparkStrategies.scala | 16 ++--- .../spark/sql/execution/aggregates.scala | 9 ++- .../spark/sql/execution/basicOperators.scala | 13 ++-- .../apache/spark/sql/execution/joins.scala | 8 +-- .../apache/spark/sql/execution/package.scala | 3 +- .../spark/sql/parquet/ParquetRelation.scala | 39 +++++------ .../sql/parquet/ParquetTableOperations.scala | 28 ++++---- .../sql/parquet/ParquetTableSupport.scala | 20 +++--- .../spark/sql/parquet/ParquetTestData.scala | 11 ++- .../org/apache/spark/sql/DslQuerySuite.scala | 7 +- .../org/apache/spark/sql/QueryTest.scala | 8 +-- .../org/apache/spark/sql/SQLQuerySuite.scala | 26 ++----- .../scala/org/apache/spark/sql/TestData.scala | 14 ++-- .../sql/{ => execution}/PlannerSuite.scala | 8 +-- .../spark/sql/{ => execution}/TgfSuite.scala | 11 +-- .../spark/sql/parquet/ParquetQuerySuite.scala | 15 ++-- .../hadoop/mapred/SparkHadoopWriter.scala | 7 +- .../apache/spark/sql/hive/HiveContext.scala | 25 +++---- .../spark/sql/hive/HiveMetastoreCatalog.scala | 22 +++--- .../org/apache/spark/sql/hive/HiveQl.scala | 25 +++---- .../spark/sql/hive/HiveStrategies.scala | 11 ++- .../spark/sql/hive/ScriptTransformation.scala | 5 +- .../apache/spark/sql/hive/TableReader.scala | 26 ++++--- .../org/apache/spark/sql/hive/TestHive.scala | 21 +++--- .../apache/spark/sql/hive/hiveOperators.scala | 28 ++++---- .../org/apache/spark/sql/hive/hiveUdfs.scala | 22 +++--- .../hive/execution/ConcurrentHiveSuite.scala | 3 +- .../hive/execution/HiveComparisonTest.scala | 12 ++-- .../execution/HiveCompatibilitySuite.scala | 5 -- .../hive/execution/HiveQueryFileTest.scala | 6 +- .../sql/hive/execution/HiveQuerySuite.scala | 3 +- .../hive/execution/HiveResolutionSuite.scala | 4 +- .../sql/hive/execution/PruningSuite.scala | 5 +- .../spark/sql/parquet/HiveParquetSuite.scala | 16 +++-- 87 files changed, 448 insertions(+), 490 deletions(-) create mode 100644 sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/{ => analysis}/AnalysisSuite.scala (79%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/{ => analysis}/HiveTypeCoercionSuite.scala (98%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/{ => expressions}/ExpressionEvaluationSuite.scala (97%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/{ => trees}/RuleExecutorSuite.scala (92%) rename sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/{ => trees}/TreeNodeSuite.scala (86%) rename sql/core/src/test/scala/org/apache/spark/sql/{ => execution}/PlannerSuite.scala (96%) rename sql/core/src/test/scala/org/apache/spark/sql/{ => execution}/TgfSuite.scala (89%) diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala new file mode 100644 index 0000000000000..bf7318d2e078b --- /dev/null +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -0,0 +1,69 @@ +/* + * 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.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import org.apache.spark.sql.catalyst.plans.logical.LocalRelation +import org.apache.spark.sql.catalyst.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) + } + } +} 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 index 919bf4dbc824c..9dec4e3d9e4c2 100644 --- 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 @@ -17,21 +17,18 @@ package org.apache.spark.sql.catalyst -import scala.util.matching.Regex -import scala.util.parsing.combinator._ +import scala.util.parsing.combinator.lexical.StdLexical +import scala.util.parsing.combinator.syntactical.StandardTokenParsers import scala.util.parsing.input.CharArrayReader.EofCh -import lexical._ -import syntactical._ -import token._ -import analysis._ -import expressions._ -import plans._ -import plans.logical._ -import types._ +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.types._ /** - * A very simple SQL parser. Based loosly on: + * A very simple SQL parser. Based loosely on: * https://github.com/stephentu/scala-sql-parser/blob/master/src/main/scala/parser.scala * * Limitations: @@ -39,10 +36,9 @@ import types._ * - 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. + * for a SQL like language should checkout the HiveQL support in the sql/hive sub-project. */ class SqlParser extends StandardTokenParsers { - def apply(input: String): LogicalPlan = { phrase(query)(new lexical.Scanner(input)) match { case Success(r, x) => r @@ -196,7 +192,7 @@ class SqlParser extends StandardTokenParsers { protected lazy val from: Parser[LogicalPlan] = FROM ~> relations - // Based very loosly on the MySQL Grammar. + // Based very loosely 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) } | @@ -261,9 +257,9 @@ class SqlParser extends StandardTokenParsers { andExpression * (OR ^^^ { (e1: Expression, e2: Expression) => Or(e1,e2) }) protected lazy val andExpression: Parser[Expression] = - comparisionExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1,e2) }) + comparisonExpression * (AND ^^^ { (e1: Expression, e2: Expression) => And(e1,e2) }) - protected lazy val comparisionExpression: Parser[Expression] = + protected lazy val comparisonExpression: 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) } | diff --git a/sql/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 index 9eb992ee58f0a..fc76e76617670 100644 --- a/sql/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 @@ -19,9 +19,10 @@ package org.apache.spark.sql package catalyst package analysis -import expressions._ -import plans.logical._ -import rules._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ + /** * A trivial [[Analyzer]] with an [[EmptyCatalog]] and [[EmptyFunctionRegistry]]. Used for testing 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 71e4dcdb1567c..b77f0bbb2f21b 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 @@ -19,9 +19,10 @@ package org.apache.spark.sql package catalyst package analysis -import plans.logical.{LogicalPlan, Subquery} import scala.collection.mutable +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Subquery} + /** * An interface for looking up relations by name. Used by an [[Analyzer]]. */ 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 index a359eb541151d..eed058d3c7482 100644 --- 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package analysis -import expressions._ +import org.apache.spark.sql.catalyst.expressions.Expression /** A catalog for looking up user defined functions, used by an [[Analyzer]]. */ trait FunctionRegistry { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index a0105cd7cf4af..a6ecf6e2ebec5 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -19,10 +19,10 @@ package org.apache.spark.sql package catalyst package analysis -import expressions._ -import plans.logical._ -import rules._ -import types._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union} +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.types._ /** * A collection of [[catalyst.rules.Rule Rules]] that can be used to coerce differing types that diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala index fe18cc466fbff..3cad3a5d4db4e 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala @@ -18,14 +18,14 @@ package org.apache.spark.sql.catalyst package analysis -import plans.logical.LogicalPlan -import rules._ +import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan /** * A trait that should be mixed into query operators where an single instance might appear multiple * times in a logical query plan. It is invalid to have multiple copies of the same attribute - * produced by distinct operators in a query tree as this breaks the gurantee that expression - * ids, which are used to differentate attributes, are unique. + * produced by distinct operators in a query tree as this breaks the guarantee that expression + * ids, which are used to differentiate attributes, are unique. * * Before analysis, all operators that include this trait will be asked to produce a new version * of itself with globally unique expression ids. 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 index 375c99f48ea84..30c55bacc7427 100644 --- 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 @@ -15,6 +15,7 @@ * limitations under the License. */ +package org.apache.spark.sql package catalyst /** diff --git a/sql/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 index 2ed2af1352a16..04ae481102c74 100644 --- a/sql/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 @@ -19,9 +19,9 @@ package org.apache.spark.sql package catalyst package analysis -import expressions._ -import plans.logical.BaseRelation -import trees.TreeNode +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.plans.logical.BaseRelation +import org.apache.spark.sql.catalyst.trees.TreeNode /** * Thrown when an invalid attempt is made to access a property of a tree that has yet to be fully @@ -95,7 +95,7 @@ case class Star( // 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) + case Some(t) => input.filter(_.qualifiers contains t) } val mappedAttributes = expandedAttributes.map(mapFunction).zip(input).map { case (n: NamedExpression, _) => n 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 cd8de9d52fb74..e6255bcafa8aa 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,58 +19,12 @@ package org.apache.spark.sql package catalyst import scala.language.implicitConversions -import scala.reflect.runtime.universe.TypeTag -import analysis.UnresolvedAttribute -import expressions._ -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) - } - } -} +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} +import org.apache.spark.sql.catalyst.types._ /** * A collection of implicit conversions that create a DSL for constructing catalyst data structures. diff --git a/sql/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 index c253587f67c4e..d8b69946fab10 100644 --- a/sql/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 @@ -18,15 +18,16 @@ package org.apache.spark.sql package catalyst -import trees._ +import org.apache.spark.sql.catalyst.trees.TreeNode /** * Functions for attaching and retrieving trees that are associated with errors. */ package object errors { - class TreeNodeException[TreeType <: TreeNode[_]] - (tree: TreeType, msg: String, cause: Throwable) extends Exception(msg, cause) { + class TreeNodeException[TreeType <: TreeNode[_]]( + 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. 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 3b6bac16ff4e6..3fa4148f0dff2 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 @@ -19,10 +19,9 @@ package org.apache.spark.sql package catalyst package expressions -import rules._ -import errors._ - -import catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.errors.attachTree +import org.apache.spark.sql.catalyst.plans.QueryPlan +import org.apache.spark.sql.catalyst.rules.Rule /** * A bound reference points to a specific slot in the input tuple, allowing the actual value 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 608656d3a90f9..71f64ef95064e 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,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import types._ +import org.apache.spark.sql.catalyst.types._ /** Cast the child expression to the target data type. */ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { @@ -40,7 +40,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { 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 (BooleanType, ByteType) => { case null => null case true => 1.toByte case false => 0.toByte 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 78aaaeebbd631..2454a3355b11f 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,9 +19,9 @@ package org.apache.spark.sql package catalyst package expressions -import errors._ -import trees._ -import types._ +import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType} +import org.apache.spark.sql.catalyst.errors.TreeNodeException abstract class Expression extends TreeNode[Expression] { self: Product => 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 index a5d0ecf96410b..0d173afec8901 100644 --- 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import types.DoubleType +import org.apache.spark.sql.catalyst.types.DoubleType case object Rand extends LeafExpression { def dataType = DoubleType 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 352967546827d..79c91ebaa4746 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import types._ +import org.apache.spark.sql.catalyst.types.NativeType /** * Represents one row of output from a relational operator. Allows both generic access by ordinal, 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 a3c7ca1acdb0c..cc339480550a7 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import types._ +import org.apache.spark.sql.catalyst.types.DataType case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expression]) extends Expression { 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 2ad8d6f31d2e6..01b7a14d4aca6 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 @@ -21,7 +21,7 @@ package expressions import scala.language.dynamics -import types._ +import org.apache.spark.sql.catalyst.types.DataType case object DynamicType extends DataType 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 2287a849e6831..a16bb80df37af 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import catalyst.types._ +import org.apache.spark.sql.catalyst.types._ abstract class AggregateExpression extends Expression { self: Product => 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 db235645cd85c..81e4a487bdbf9 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 @@ -19,8 +19,8 @@ package org.apache.spark.sql package catalyst package expressions -import catalyst.analysis.UnresolvedException -import catalyst.types._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.types._ case class UnaryMinus(child: Expression) extends UnaryExpression { type EvaluatedType = Any 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 d3feb6c461b55..9ec0f6ade7467 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import types._ +import org.apache.spark.sql.catalyst.types._ /** * Returns the item at `ordinal` in the Array `child` or the Key `ordinal` in Map `child`. 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 c367de2a3e570..9097c635ee96f 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import catalyst.types._ +import org.apache.spark.sql.catalyst.types._ /** * An expression that produces zero or more rows given a single input row. 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 229d8f7f7b763..0d01312c71c70 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import types._ +import org.apache.spark.sql.catalyst.types._ object Literal { def apply(v: Any): Literal = v match { 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 0a06e85325a22..47b1241e714af 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 @@ -19,8 +19,8 @@ package org.apache.spark.sql package catalyst package expressions -import catalyst.analysis.UnresolvedAttribute -import types._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.types._ object NamedExpression { private val curId = new java.util.concurrent.atomic.AtomicLong() @@ -30,7 +30,7 @@ object NamedExpression { /** * 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. + * referenced in a subsequent computation. */ case class ExprId(id: Long) 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 e869a4d9b0a52..38e38371b1dec 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.analysis.UnresolvedException case class Coalesce(children: Seq[Expression]) extends Expression { type EvaluatedType = Any 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 561396eb4373c..e7f3e8ca60a25 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 @@ -19,8 +19,8 @@ package org.apache.spark.sql package catalyst package expressions -import types._ -import catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.types.{BooleanType, StringType} +import org.apache.spark.sql.catalyst.analysis.UnresolvedException trait Predicate extends Expression { self: Product => 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 index 6e585236b1b20..7584fe03cf745 100644 --- 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package expressions -import catalyst.types.BooleanType +import org.apache.spark.sql.catalyst.types.BooleanType case class Like(left: Expression, right: Expression) extends BinaryExpression { def dataType = BooleanType 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 4db280317345a..c1201971d9c0a 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 @@ -19,11 +19,11 @@ package org.apache.spark.sql package catalyst package optimizer -import catalyst.expressions._ -import catalyst.plans.logical._ -import catalyst.rules._ -import catalyst.types.BooleanType -import catalyst.plans.Inner +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.Inner +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.catalyst.types._ object Optimizer extends RuleExecutor[LogicalPlan] { val batches = @@ -73,7 +73,7 @@ object ConstantFolding extends Rule[LogicalPlan] { object BooleanSimplification extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan => q transformExpressionsUp { - case and @ And(left, right) => { + case and @ And(left, right) => (left, right) match { case (Literal(true, BooleanType), r) => r case (l, Literal(true, BooleanType)) => l @@ -81,8 +81,8 @@ object BooleanSimplification extends Rule[LogicalPlan] { case (_, Literal(false, BooleanType)) => Literal(false) case (_, _) => and } - } - case or @ Or(left, right) => { + + case or @ Or(left, right) => (left, right) match { case (Literal(true, BooleanType), _) => Literal(true) case (_, Literal(true, BooleanType)) => Literal(true) @@ -90,7 +90,6 @@ object BooleanSimplification extends Rule[LogicalPlan] { case (l, Literal(false, BooleanType)) => l case (_, _) => or } - } } } } @@ -101,7 +100,7 @@ object BooleanSimplification extends Rule[LogicalPlan] { */ 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) + case ff @ Filter(fc, nf @ Filter(nc, grandChild)) => Filter(And(nc, fc), grandChild) } } @@ -114,8 +113,10 @@ object CombineFilters extends Rule[LogicalPlan] { */ object PushPredicateThroughProject extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { - case filter@Filter(condition, project@Project(fields, grandChild)) => - val sourceAliases = fields.collect { case a@Alias(c, _) => a.toAttribute -> c }.toMap + case filter @ Filter(condition, project @ Project(fields, grandChild)) => + val sourceAliases = fields.collect { case a @ Alias(c, _) => + (a.toAttribute: Attribute) -> c + }.toMap project.copy(child = filter.copy( replaceAlias(condition, sourceAliases), grandChild)) diff --git a/sql/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 index 22f8ea005baf1..d50b963dfc25b 100644 --- a/sql/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 @@ -19,9 +19,8 @@ package org.apache.spark.sql package catalyst package planning - -import plans.logical.LogicalPlan -import trees._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees.TreeNode /** * Abstract class for transforming [[plans.logical.LogicalPlan LogicalPlan]]s into physical plans. diff --git a/sql/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 index 613b028ca8409..ff0ea90e54842 100644 --- a/sql/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 @@ -21,8 +21,8 @@ package planning import scala.annotation.tailrec -import expressions._ -import plans.logical._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical._ /** * A pattern that matches any number of filter operations on top of another relational operator. diff --git a/sql/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 index 20f230c5c4f2c..848db2452ae9c 100644 --- a/sql/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 @@ -19,8 +19,8 @@ package org.apache.spark.sql package catalyst package plans -import catalyst.expressions.{SortOrder, Attribute, Expression} -import catalyst.trees._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.trees.TreeNode abstract class QueryPlan[PlanType <: TreeNode[PlanType]] extends TreeNode[PlanType] { self: PlanType with Product => diff --git a/sql/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 index bc7b6871df524..225dd260fb9de 100644 --- a/sql/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 @@ -20,9 +20,9 @@ package catalyst package plans package logical -import catalyst.expressions._ -import catalyst.errors._ -import catalyst.types.StructType +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types.StructType abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index 1a1a2b9b88bd0..5a3ea9f0a0464 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -20,7 +20,7 @@ package catalyst package plans package logical -import expressions._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} /** * Transforms the input by forking and running the specified script. diff --git a/sql/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 index b5905a4456d12..ac7d2d6001779 100644 --- a/sql/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 @@ -20,8 +20,7 @@ package catalyst package plans package logical -import expressions._ -import rules._ +import org.apache.spark.sql.catalyst.expressions.Attribute object LocalRelation { def apply(output: Attribute*) = 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 8e98aab736432..6480cca30049e 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 @@ -20,7 +20,7 @@ package catalyst package plans package logical -import expressions._ +import org.apache.spark.sql.catalyst.expressions._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) 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 index f7fcdc5fdb950..775e50bbd5128 100644 --- 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 @@ -20,7 +20,7 @@ package catalyst package plans package logical -import expressions._ +import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} /** * Performs a physical redistribution of the data. Used when the consumer of the query diff --git a/sql/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 index 2d8f3ad33517d..20e2a45678c89 100644 --- a/sql/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 @@ -20,8 +20,8 @@ package catalyst package plans package physical -import expressions._ -import types._ +import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} +import org.apache.spark.sql.catalyst.types.IntegerType /** * Specifies how tuples that share common expressions will be distributed when a query is executed 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 index 6ff4891a3f8a6..c7632a62a00d2 100644 --- 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package rules -import trees._ +import org.apache.spark.sql.catalyst.trees.TreeNode abstract class Rule[TreeType <: TreeNode[_]] extends Logging { diff --git a/sql/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 index 68ae30cde1c57..9db96f89dd03c 100644 --- a/sql/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 @@ -19,8 +19,8 @@ package org.apache.spark.sql package catalyst package rules -import trees._ -import util._ +import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.util.sideBySide abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { @@ -52,19 +52,19 @@ abstract class RuleExecutor[TreeType <: TreeNode[_]] extends Logging { batches.foreach { batch => var iteration = 1 var lastPlan = curPlan - curPlan = batch.rules.foldLeft(curPlan) { case (curPlan, rule) => rule(curPlan) } + curPlan = batch.rules.foldLeft(curPlan) { case (plan, rule) => rule(plan) } // 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) => - val result = rule(curPlan) - if (!result.fastEquals(curPlan)) { + case (plan, rule) => + val result = rule(plan) + if (!result.fastEquals(plan)) { logger.debug( s""" |=== Applying Rule ${rule.ruleName} === - |${sideBySide(curPlan.treeString, result.treeString).mkString("\n")} + |${sideBySide(plan.treeString, result.treeString).mkString("\n")} """.stripMargin) } diff --git a/sql/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 index 37e557441dc63..89e27d81dad0b 100644 --- a/sql/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 @@ -19,7 +19,7 @@ package org.apache.spark.sql package catalyst package trees -import errors._ +import org.apache.spark.sql.catalyst.errors._ object TreeNode { private val currentId = new java.util.concurrent.atomic.AtomicLong 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/analysis/AnalysisSuite.scala similarity index 79% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/AnalysisSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 1fd0d26b6f8bb..78ec48ba77628 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/analysis/AnalysisSuite.scala @@ -21,13 +21,10 @@ package analysis import org.scalatest.FunSuite -import analysis._ -import expressions._ -import plans.logical._ -import types._ +import org.apache.spark.sql.catalyst.plans.logical._ -import dsl._ -import dsl.expressions._ +/* Implicit conversions */ +import org.apache.spark.sql.catalyst.dsl.expressions._ class AnalysisSuite extends FunSuite { val analyze = SimpleAnalyzer @@ -35,7 +32,8 @@ class AnalysisSuite extends FunSuite { val testRelation = LocalRelation('a.int) test("analyze project") { - assert(analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === Project(testRelation.output, testRelation)) - + assert( + analyze(Project(Seq(UnresolvedAttribute("a")), testRelation)) === + Project(testRelation.output, testRelation)) } -} \ No newline at end of file +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala similarity index 98% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index f595bf7e44af7..b85b72a2840f9 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -21,8 +21,7 @@ package analysis import org.scalatest.FunSuite -import catalyst.types._ - +import org.apache.spark.sql.catalyst.types._ class HiveTypeCoercionSuite 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/expressions/ExpressionEvaluationSuite.scala similarity index 97% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/ExpressionEvaluationSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index f06618ad11467..c8fd581aa7b47 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/expressions/ExpressionEvaluationSuite.scala @@ -21,10 +21,9 @@ package expressions import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types._ -/* Implict conversions */ +/* Implicit conversions */ import org.apache.spark.sql.catalyst.dsl.expressions._ class ExpressionEvaluationSuite extends FunSuite { @@ -112,4 +111,4 @@ class ExpressionEvaluationSuite extends FunSuite { } } } -} \ No newline at end of file +} 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 7ce42b2b0a057..2c107b865af19 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 @@ -19,13 +19,14 @@ package org.apache.spark.sql package catalyst package optimizer -import types.IntegerType -import util._ -import plans.logical.{LogicalPlan, LocalRelation} -import rules._ -import expressions._ -import dsl.plans._ -import dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.types.IntegerType + +// For implicit conversions +import org.apache.spark.sql.catalyst.dsl.expressions._ class ConstantFoldingSuite extends OptimizerTest { @@ -106,7 +107,7 @@ class ConstantFoldingSuite extends OptimizerTest { 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")) + 'a * Literal(7) as Symbol("c4")) .analyze comparePlans(optimized, correctAnswer) @@ -173,4 +174,4 @@ class ConstantFoldingSuite extends OptimizerTest { comparePlans(optimized, correctAnswer) } -} \ No newline at end of file +} 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 cd611b3fb3400..cfbef53de16b4 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 @@ -2,13 +2,12 @@ package org.apache.spark.sql package catalyst package optimizer -import expressions._ -import plans.logical._ -import rules._ -import util._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ -import dsl.plans._ -import dsl.expressions._ +/* Implicit conversions */ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.dsl.expressions._ class FilterPushdownSuite extends OptimizerTest { @@ -219,4 +218,4 @@ class FilterPushdownSuite extends OptimizerTest { comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) } -} \ No newline at end of file +} diff --git a/sql/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 index 7b3653d0f998a..8ec1d3d8c03ca 100644 --- a/sql/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 @@ -4,13 +4,9 @@ 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 */ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.util._ /** * Provides helper methods for comparing plans produced by optimization rules with the expected @@ -41,4 +37,4 @@ class OptimizerTest extends FunSuite { |${sideBySide(normalized1.treeString, normalized2.treeString).mkString("\n")} """.stripMargin) } -} \ No newline at end of file +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala similarity index 92% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala index ff7c15b7187b7..738cfa85fbc6a 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/RuleExecutorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala @@ -21,8 +21,8 @@ package trees import org.scalatest.FunSuite -import expressions._ -import rules._ +import org.apache.spark.sql.catalyst.expressions.{Expression, IntegerLiteral, Literal} +import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor} class RuleExecutorSuite extends FunSuite { object DecrementLiterals extends Rule[Expression] { @@ -54,4 +54,4 @@ class RuleExecutorSuite extends FunSuite { assert(ToFixedPoint(Literal(100)) === Literal(90)) } -} \ No newline at end of file +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala similarity index 86% rename from sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala rename to sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala index 98bb090c29d11..1ddc41a731ff5 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/TreeNodeSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/TreeNodeSuite.scala @@ -15,18 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package trees +package org.apache.spark.sql.catalyst.trees import scala.collection.mutable.ArrayBuffer -import expressions._ +import org.scalatest.FunSuite -import org.scalatest.{FunSuite} +import org.apache.spark.sql.catalyst.expressions._ class TreeNodeSuite extends FunSuite { - test("top node changed") { val after = Literal(1) transform { case Literal(1, _) => Literal(2) } assert(after === Literal(2)) @@ -60,8 +57,8 @@ class TreeNodeSuite extends FunSuite { val expected = Seq("+", "1", "*", "2", "-", "3", "4") val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) expression transformDown { - case b: BinaryExpression => {actual.append(b.symbol); b} - case l: Literal => {actual.append(l.toString); l} + case b: BinaryExpression => actual.append(b.symbol); b + case l: Literal => actual.append(l.toString); l } assert(expected === actual) @@ -72,10 +69,10 @@ class TreeNodeSuite extends FunSuite { val expected = Seq("1", "2", "3", "4", "-", "*", "+") val expression = Add(Literal(1), Multiply(Literal(2), Subtract(Literal(3), Literal(4)))) expression transformUp { - case b: BinaryExpression => {actual.append(b.symbol); b} - case l: Literal => {actual.append(l.toString); l} + 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 +} diff --git a/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala b/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala index b8b9e5839d0a5..f1230e7526ab1 100644 --- a/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala +++ b/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala @@ -22,11 +22,12 @@ import scala.language.implicitConversions import scala.reflect._ import scala.collection.mutable.ArrayBuffer -import org.apache.spark._ -import org.apache.spark.Aggregator -import org.apache.spark.SparkContext._ +import org.apache.spark.{Aggregator, InterruptibleIterator, Logging} import org.apache.spark.util.collection.AppendOnlyMap +/* Implicit conversions */ +import org.apache.spark.SparkContext._ + /** * Extra functions on RDDs that perform only local operations. These can be used when data has * already been partitioned correctly. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 587cc7487f31c..3e98bd3ca627d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -20,14 +20,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.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.Optimizer -import org.apache.spark.sql.catalyst.planning.QueryPlanner -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, NativeCommand, WriteToFile} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.execution._ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 91c3aaa2b8d5b..770cabcb31d13 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql -import org.apache.spark.{OneToOneDependency, Dependency, Partition, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType +import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} /** * ALPHA COMPONENT 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 e934c4cf69ab5..65d77e3a40cf4 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,14 +18,13 @@ package org.apache.spark.sql package execution -import org.apache.spark.{SparkConf, RangePartitioner, HashPartitioner} import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.sql.catalyst.errors.attachTree +import org.apache.spark.sql.catalyst.expressions.{MutableProjection, RowOrdering} +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair - -import catalyst.rules.Rule -import catalyst.errors._ -import catalyst.expressions._ -import catalyst.plans.physical._ +import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { @@ -35,7 +34,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una 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().mapPartitions { iter => val hashExpressions = new MutableProjection(expressions) @@ -46,8 +45,8 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una val shuffled = new ShuffledRDD[Row, Row, MutablePair[Row, Row]](rdd, part) shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._2) - } - case RangePartitioning(sortingExpressions, numPartitions) => { + + case RangePartitioning(sortingExpressions, numPartitions) => // TODO: RangePartitioner should take an Ordering. implicit val ordering = new RowOrdering(sortingExpressions) @@ -60,9 +59,9 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) shuffled.map(_._1) - } + case SinglePartition => - child.execute().coalesce(1, true) + child.execute().coalesce(1, shuffle = true) case _ => sys.error(s"Exchange not implemented for $newPartitioning") // TODO: Handle BroadcastPartitioning. 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 c1da3653c5b8e..7e50fda4ceac5 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 @@ -18,8 +18,7 @@ package org.apache.spark.sql package execution -import catalyst.expressions._ -import catalyst.types._ +import org.apache.spark.sql.catalyst.expressions._ /** * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the 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 85035b8118936..9eb1032113cd9 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 @@ -20,13 +20,13 @@ package execution import org.apache.spark.SparkContext -import catalyst.expressions._ -import catalyst.planning._ -import catalyst.plans._ -import catalyst.plans.logical.LogicalPlan -import catalyst.plans.physical._ -import parquet.ParquetRelation -import parquet.InsertIntoParquetTable +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.parquet.ParquetRelation +import org.apache.spark.sql.parquet.InsertIntoParquetTable abstract class SparkStrategies extends QueryPlanner[SparkPlan] { @@ -172,7 +172,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { - // TOOD: Set + // TODO: Set val numPartitions = 200 def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Distinct(child) => 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 51889c1988680..14e5ab628f23d 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 @@ -19,12 +19,11 @@ package org.apache.spark.sql package execution import org.apache.spark.SparkContext +import org.apache.spark.sql.catalyst.errors._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical._ -import catalyst.errors._ -import catalyst.expressions._ -import catalyst.plans.physical.{UnspecifiedDistribution, ClusteredDistribution, AllTuples} -import catalyst.types._ - +/* Implicit conversions */ import org.apache.spark.rdd.PartitionLocalRDDFunctions._ /** 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 c6d31d9abc9e8..e4f918b6787fe 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 @@ -23,18 +23,17 @@ import scala.reflect.runtime.universe.TypeTag 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 -import catalyst.ScalaReflection +import org.apache.spark.sql.catalyst.errors._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.physical.{OrderedDistribution, UnspecifiedDistribution} +import org.apache.spark.sql.catalyst.ScalaReflection case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) def execute() = child.execute().mapPartitions { iter => - @transient val resuableProjection = new MutableProjection(projectList) - iter.map(resuableProjection) + @transient val reusableProjection = new MutableProjection(projectList) + iter.map(reusableProjection) } } 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 5934fd1b03bfb..a6e3892e88703 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 @@ -23,10 +23,10 @@ import scala.collection.mutable 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.sql.catalyst.errors._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} import org.apache.spark.rdd.PartitionLocalRDDFunctions._ 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 index 67f6f43f904b1..e4a2dec33231d 100644 --- 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 @@ -25,5 +25,4 @@ package org.apache.spark.sql * documented here in order to make it easier for others to understand the performance * characteristics of query plans that are generated by Spark SQL. */ -package object execution { -} +package object execution 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 e87561fe1384f..011aaf7440cb1 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 @@ -19,28 +19,27 @@ package org.apache.spark.sql.parquet import java.io.{IOException, FileNotFoundException} -import org.apache.hadoop.fs.{Path, FileSystem} +import scala.collection.JavaConversions._ + import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.fs.permission.FsAction +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.mapreduce.Job -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, AttributeReference, Attribute} -import org.apache.spark.sql.catalyst.analysis.UnresolvedException - -import parquet.schema.{MessageTypeParser, MessageType} +import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} +import parquet.hadoop.util.ContextUtil +import parquet.hadoop.{Footer, ParquetFileReader, ParquetFileWriter} +import parquet.io.api.{Binary, RecordConsumer} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} +import parquet.schema.Type.Repetition +import parquet.schema.{MessageType, MessageTypeParser} 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, ParquetFileReader} -import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} -import parquet.hadoop.util.ContextUtil -import scala.collection.JavaConversions._ +import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} +import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.types._ /** * Relation that consists of data stored in a Parquet columnar format. @@ -55,7 +54,7 @@ import scala.collection.JavaConversions._ * @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 { +case class ParquetRelation(tableName: String, path: String) extends BaseRelation { /** Schema derived from ParquetFile **/ def parquetSchema: MessageType = @@ -145,11 +144,10 @@ object ParquetTypesConverter { case ParquetPrimitiveTypeName.FLOAT => FloatType case ParquetPrimitiveTypeName.INT32 => IntegerType case ParquetPrimitiveTypeName.INT64 => LongType - case ParquetPrimitiveTypeName.INT96 => { + 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") } @@ -186,11 +184,10 @@ object ParquetTypesConverter { def convertToAttributes(parquetSchema: MessageType) : Seq[Attribute] = { parquetSchema.getColumns.map { - case (desc) => { + case (desc) => val ctype = toDataType(desc.getType) val name: String = desc.getPath.mkString(".") new AttributeReference(name, ctype, false)() - } } } @@ -245,7 +242,7 @@ object ParquetTypesConverter { * 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. + * @param origPath The path at which we expect one (or more) Parquet files. * @return The `ParquetMetadata` containing among other things the schema. */ def readMetaData(origPath: Path): ParquetMetadata = { 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 61121103cbbfa..7285f5b88b9bf 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 @@ -17,24 +17,24 @@ package org.apache.spark.sql.parquet -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.{TaskContext, SerializableWritable, SparkContext} -import org.apache.spark.sql.catalyst.expressions.{Row, Attribute, Expression} -import org.apache.spark.sql.execution.{SparkPlan, UnaryNode, LeafNode} +import java.io.IOException +import java.text.SimpleDateFormat +import java.util.Date -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 org.apache.hadoop.mapreduce._ +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} -import java.io.IOException -import java.text.SimpleDateFormat -import java.util.Date +import parquet.hadoop.util.ContextUtil +import parquet.hadoop.{ParquetInputFormat, ParquetOutputFormat} +import parquet.io.InvalidRecordException +import parquet.schema.MessageType + +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} +import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} +import org.apache.spark.{SerializableWritable, SparkContext, TaskContext} /** * Parquet table scan operator. Imports the file that backs 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 c2ae18b882d19..91b4848fe403f 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 @@ -19,16 +19,15 @@ package org.apache.spark.sql.parquet import org.apache.hadoop.conf.Configuration -import org.apache.spark.Logging - -import parquet.io.api._ -import parquet.schema.{MessageTypeParser, MessageType} -import parquet.hadoop.api.{WriteSupport, ReadSupport} -import parquet.hadoop.api.ReadSupport.ReadContext -import parquet.hadoop.ParquetOutputFormat import parquet.column.ParquetProperties +import parquet.hadoop.ParquetOutputFormat +import parquet.hadoop.api.ReadSupport.ReadContext +import parquet.hadoop.api.{ReadSupport, WriteSupport} +import parquet.io.api._ +import parquet.schema.{MessageType, MessageTypeParser} -import org.apache.spark.sql.catalyst.expressions.{Row, Attribute} +import org.apache.spark.Logging +import org.apache.spark.sql.catalyst.expressions.{Attribute, Row} import org.apache.spark.sql.catalyst.types._ /** @@ -95,8 +94,7 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { } def getSchema(configuration: Configuration): MessageType = { - return MessageTypeParser.parseMessageType( - configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) + MessageTypeParser.parseMessageType(configuration.get(RowWriteSupport.PARQUET_ROW_SCHEMA)) } private var schema: MessageType = null @@ -108,7 +106,7 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { attributes = ParquetTypesConverter.convertToAttributes(schema) new WriteSupport.WriteContext( schema, - new java.util.HashMap[java.lang.String, java.lang.String]()); + new java.util.HashMap[java.lang.String, java.lang.String]()) } override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { 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 bbe409fb9c7e0..3340c3ff81f0a 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,17 +17,16 @@ package org.apache.spark.sql.parquet -import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.Job -import parquet.schema.{MessageTypeParser, MessageType} -import parquet.hadoop.util.ContextUtil import parquet.hadoop.ParquetWriter +import parquet.hadoop.util.ContextUtil +import parquet.schema.{MessageType, MessageTypeParser} -import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.catalyst.expressions.GenericRow -import java.nio.charset.Charset +import org.apache.spark.sql.catalyst.util.getTempFilePath object ParquetTestData { @@ -69,7 +68,7 @@ object ParquetTestData { lazy val testData = new ParquetRelation("testData", testFile.toURI.toString) - def writeFile = { + def writeFile() = { testFile.delete val path: Path = new Path(testFile.toURI) val job = new Job() diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 37c90a18a0148..2524a37cbac13 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -17,14 +17,9 @@ package org.apache.spark.sql -import org.scalatest.{BeforeAndAfterAll, FunSuite} - import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.test._ /* Implicits */ @@ -198,4 +193,4 @@ class DslQuerySuite extends QueryTest { (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) } -} \ No newline at end of file +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index aa84211648db0..5c8cb086ee3b8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -17,18 +17,12 @@ package org.apache.spark.sql -import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.analysis._ -import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.util._ -import org.apache.spark.sql.test._ /* Implicits */ -import TestSQLContext._ class QueryTest extends FunSuite { /** diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 6371fa296ac45..fa4a1d5189ea6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -17,13 +17,7 @@ package org.apache.spark.sql -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.test._ /* Implicits */ @@ -37,8 +31,7 @@ class SQLQuerySuite extends QueryTest { test("agg") { checkAnswer( sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"), - Seq((1,3),(2,3),(3,3)) - ) + Seq((1,3),(2,3),(3,3))) } test("select *") { @@ -88,13 +81,11 @@ class SQLQuerySuite extends QueryTest { ignore("null count") { checkAnswer( sql("SELECT a, COUNT(b) FROM testData3"), - Seq((1,0), (2, 1)) - ) + Seq((1,0), (2, 1))) checkAnswer( testData3.groupBy()(Count('a), Count('b), Count(1), CountDistinct('a :: Nil), CountDistinct('b :: Nil)), - (2, 1, 2, 2, 1) :: Nil - ) + (2, 1, 2, 2, 1) :: Nil) } test("inner join where, one match per row") { @@ -104,8 +95,7 @@ class SQLQuerySuite extends QueryTest { (1, "A", 1, "a"), (2, "B", 2, "b"), (3, "C", 3, "c"), - (4, "D", 4, "d") - )) + (4, "D", 4, "d"))) } test("inner join ON, one match per row") { @@ -115,8 +105,7 @@ class SQLQuerySuite extends QueryTest { (1, "A", 1, "a"), (2, "B", 2, "b"), (3, "C", 3, "c"), - (4, "D", 4, "d") - )) + (4, "D", 4, "d"))) } test("inner join, where, multiple matches") { @@ -129,8 +118,7 @@ class SQLQuerySuite extends QueryTest { (1,1,1,1) :: (1,1,1,2) :: (1,2,1,1) :: - (1,2,1,2) :: Nil - ) + (1,2,1,2) :: Nil) } test("inner join, no matches") { @@ -164,7 +152,7 @@ class SQLQuerySuite extends QueryTest { row => Seq.fill(16)((row ++ row).toSeq)).collect().toSeq) } - ignore("cartisian product join") { + ignore("cartesian product join") { checkAnswer( testData3.join(testData3), (1, null, 1, null) :: diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 640292571b125..0bb13cf442b59 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -37,16 +37,14 @@ object TestData { TestData2(2, 1) :: TestData2(2, 2) :: TestData2(3, 1) :: - TestData2(3, 2) :: Nil - ) + TestData2(3, 2) :: Nil) testData2.registerAsTable("testData2") // TODO: There is no way to express null primitives as case classes currently... val testData3 = logical.LocalRelation('a.int, 'b.int).loadData( (1, null) :: - (2, 2) :: Nil - ) + (2, 2) :: Nil) case class UpperCaseData(N: Int, L: String) val upperCaseData = @@ -56,8 +54,7 @@ object TestData { UpperCaseData(3, "C") :: UpperCaseData(4, "D") :: UpperCaseData(5, "E") :: - UpperCaseData(6, "F") :: Nil - ) + UpperCaseData(6, "F") :: Nil) upperCaseData.registerAsTable("upperCaseData") case class LowerCaseData(n: Int, l: String) @@ -66,7 +63,6 @@ object TestData { LowerCaseData(1, "a") :: LowerCaseData(2, "b") :: LowerCaseData(3, "c") :: - LowerCaseData(4, "d") :: Nil - ) + LowerCaseData(4, "d") :: Nil) lowerCaseData.registerAsTable("lowerCaseData") -} \ No newline at end of file +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala similarity index 96% rename from sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala index 83908edf5a040..658ff0927aa85 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/PlannerSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/PlannerSuite.scala @@ -15,20 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import org.scalatest.FunSuite +import org.apache.spark.sql.TestData._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical -import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.execution import org.apache.spark.sql.test.TestSQLContext._ import org.apache.spark.sql.test.TestSQLContext.planner._ class PlannerSuite extends FunSuite { - - test("unions are collapsed") { val query = testData.unionAll(testData).unionAll(testData).logicalPlan val planned = BasicOperators(query).head diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala similarity index 89% rename from sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala index 08265b7a6a2bf..93b2a308a410a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala @@ -18,17 +18,12 @@ package org.apache.spark.sql package execution -import org.scalatest.{BeforeAndAfterAll, FunSuite} - -import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.catalyst.types._ -import org.apache.spark.sql.test._ - -import TestSQLContext._ +/* Implicit conversions */ +import org.apache.spark.sql.test.TestSQLContext._ /** * This is an example TGF that uses UnresolvedAttributes 'name and 'age to access specific columns diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 8b2ccb52d82c3..71caa709afca6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -19,21 +19,20 @@ package org.apache.spark.sql.parquet import org.scalatest.{BeforeAndAfterAll, FunSuite} +import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.mapreduce.Job +import parquet.hadoop.ParquetFileWriter +import parquet.hadoop.util.ContextUtil +import parquet.schema.MessageTypeParser + import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.test.TestSQLContext -import org.apache.hadoop.mapreduce.Job -import org.apache.hadoop.fs.{Path, FileSystem} - -import parquet.schema.MessageTypeParser -import parquet.hadoop.ParquetFileWriter -import parquet.hadoop.util.ContextUtil - class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { override def beforeAll() { - ParquetTestData.writeFile + ParquetTestData.writeFile() } override def afterAll() { diff --git a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala index 08d390e8871ab..0b3873191985b 100644 --- a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala @@ -22,15 +22,14 @@ import java.text.NumberFormat import java.util.Date import org.apache.hadoop.fs.Path +import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} +import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} +import org.apache.hadoop.hive.ql.plan.FileSinkDesc import org.apache.hadoop.io.Writable import org.apache.spark.Logging import org.apache.spark.SerializableWritable -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 - /** * Internal helper class that saves an RDD using a Hive OutputFormat. * It is based on [[SparkHadoopWriter]]. 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 4aad876cc0364..491b3a62712c6 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 @@ -18,25 +18,26 @@ package org.apache.spark.sql package hive -import java.io.{PrintStream, InputStreamReader, BufferedReader, File} -import java.util.{ArrayList => JArrayList} import scala.language.implicitConversions -import org.apache.spark.SparkContext +import java.io.{BufferedReader, File, InputStreamReader, PrintStream} +import java.util.{ArrayList => JArrayList} + 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 org.apache.hadoop.hive.ql.processors._ +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.spark.SparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog} +import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand, ExplainCommand} +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution._ +/* Implicit conversions */ import scala.collection.JavaConversions._ /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala index e4d50722ce9a8..a5db283765c27 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 @@ -27,12 +27,12 @@ import org.apache.hadoop.hive.ql.plan.TableDesc import org.apache.hadoop.hive.ql.session.SessionState import org.apache.hadoop.hive.serde2.Deserializer -import catalyst.analysis.Catalog -import catalyst.expressions._ -import catalyst.plans.logical -import catalyst.plans.logical._ -import catalyst.rules._ -import catalyst.types._ +import org.apache.spark.sql.catalyst.analysis.Catalog +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.rules._ +import org.apache.spark.sql.catalyst.types._ import scala.collection.JavaConversions._ @@ -45,7 +45,7 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { db: Option[String], tableName: String, alias: Option[String]): LogicalPlan = { - val databaseName = db.getOrElse(hive.sessionState.getCurrentDatabase()) + val databaseName = db.getOrElse(hive.sessionState.getCurrentDatabase) val table = client.getTable(databaseName, tableName) val partitions: Seq[Partition] = if (table.isPartitioned) { @@ -91,7 +91,7 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { object CreateTables extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case InsertIntoCreatedTable(db, tableName, child) => - val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase()) + val databaseName = db.getOrElse(SessionState.get.getCurrentDatabase) createTable(databaseName, tableName, child.output) @@ -123,8 +123,8 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { } 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, output) if input.dataType != output.dataType => + Alias(Cast(input, output.dataType), input.name)() case (input, _) => input } @@ -135,7 +135,7 @@ 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]]. + * For now, if this functionality is desired mix in the in-memory [[OverrideCatalog]]. */ override def registerTable( databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = ??? 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 4f33a293c3964..8e76a7348e957 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 @@ -18,18 +18,19 @@ package org.apache.spark.sql package hive -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._ -import catalyst.plans._ -import catalyst.plans.logical -import catalyst.plans.logical._ -import catalyst.types._ +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.catalyst.types._ + +/* Implicit conversions */ +import scala.collection.JavaConversions._ /** * Used when we need to start parsing the AST before deciding that we are going to pass the command @@ -48,7 +49,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. */ +/** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ object HiveQl { protected val nativeCommands = Seq( "TOK_DESCFUNCTION", @@ -150,13 +151,13 @@ object HiveQl { } /** - * Returns a scala.Seq equivilent to [s] or Nil if [s] is null. + * Returns a scala.Seq equivalent to [s] or Nil if [s] is null. */ private def nilIfEmpty[A](s: java.util.List[A]): Seq[A] = 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) @@ -667,7 +668,7 @@ object HiveQl { case Token(allJoinTokens(joinToken), relation1 :: relation2 :: other) => - assert(other.size <= 1, s"Unhandled join child ${other}") + assert(other.size <= 1, s"Unhandled join child $other") val joinType = joinToken match { case "TOK_JOIN" => Inner case "TOK_RIGHTOUTERJOIN" => RightOuter 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 92d84208ab3f9..c71141c4194f6 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 @@ -18,13 +18,12 @@ package org.apache.spark.sql package hive -import catalyst.expressions._ -import catalyst.planning._ -import catalyst.plans._ -import catalyst.plans.logical.{BaseRelation, LogicalPlan} - +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning._ +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} import org.apache.spark.sql.execution._ -import org.apache.spark.sql.parquet.{ParquetRelation, InsertIntoParquetTable, ParquetTableScan} +import org.apache.spark.sql.parquet.{InsertIntoParquetTable, ParquetRelation, ParquetTableScan} trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. 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 f20e9d4de4cf2..dc4181ec9905c 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 @@ -18,11 +18,12 @@ package org.apache.spark.sql package hive -import java.io.{InputStreamReader, BufferedReader} +import java.io.{BufferedReader, InputStreamReader} -import catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ +/* Implicit conversions */ import scala.collection.JavaConversions._ /** diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 71d751cbc424d..99dc85ec19868 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -19,19 +19,18 @@ package org.apache.spark.sql package hive import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{Path, PathFilter} import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants._ +import org.apache.hadoop.hive.ql.exec.Utilities import org.apache.hadoop.hive.ql.metadata.{Partition => HivePartition, Table => HiveTable} import org.apache.hadoop.hive.ql.plan.TableDesc 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.hadoop.mapred.{FileInputFormat, InputFormat, JobConf} import org.apache.spark.SerializableWritable import org.apache.spark.broadcast.Broadcast -import org.apache.spark.rdd.{HadoopRDD, UnionRDD, EmptyRDD, RDD} - +import org.apache.spark.rdd.{EmptyRDD, HadoopRDD, RDD, UnionRDD} /** * A trait for subclasses that handle table scans. @@ -40,7 +39,6 @@ private[hive] sealed trait TableReader { def makeRDDForTable(hiveTable: HiveTable): RDD[_] def makeRDDForPartitionedTable(partitions: Seq[HivePartition]): RDD[_] - } @@ -57,7 +55,6 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon private val _minSplitsPerRDD = math.max( sc.hiveconf.getInt("mapred.map.tasks", 1), sc.sparkContext.defaultMinSplits) - // TODO: set aws s3 credentials. private val _broadcastedHiveConf = @@ -85,8 +82,8 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon def makeRDDForTable( hiveTable: HiveTable, deserializerClass: Class[_ <: Deserializer], - filterOpt: Option[PathFilter]): RDD[_] = - { + 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.""") @@ -115,6 +112,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon sys.error(s"Unable to deserialize non-Writable: $value of ${value.getClass.getName}") } } + deserializedHadoopRDD } @@ -136,8 +134,8 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon */ def makeRDDForPartitionedTable( partitionToDeserializer: Map[HivePartition, Class[_ <: Deserializer]], - filterOpt: Option[PathFilter]): RDD[_] = - { + filterOpt: Option[PathFilter]): RDD[_] = { + val hivePartitionRDDs = partitionToDeserializer.map { case (partition, partDeserializer) => val partDesc = Utilities.getPartitionDesc(partition) val partPath = partition.getPartitionPath @@ -178,6 +176,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon } } }.toSeq + // Even if we don't use any partitions, we still need an empty RDD if (hivePartitionRDDs.size == 0) { new EmptyRDD[Object](sc.sparkContext) @@ -207,8 +206,8 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon private def createHadoopRdd( tableDesc: TableDesc, path: String, - inputFormatClass: Class[InputFormat[Writable, Writable]]) - : RDD[Writable] = { + inputFormatClass: Class[InputFormat[Writable, Writable]]): RDD[Writable] = { + val initializeJobConfFunc = HadoopTableReader.initializeLocalJobConfFunc(path, tableDesc) _ val rdd = new HadoopRDD( @@ -227,7 +226,6 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon } private[hive] object HadoopTableReader { - /** * Curried. After given an argument for 'path', the resulting JobConf => Unit closure is used to * instantiate a HadoopRDD. diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 17ae4ef63c054..a26b0ff2319d4 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -22,23 +22,22 @@ import java.io.File 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 import org.apache.hadoop.hive.ql.exec.FunctionRegistry -import org.apache.hadoop.hive.ql.io.avro.{AvroContainerOutputFormat, AvroContainerInputFormat} +import org.apache.hadoop.hive.ql.io.avro.{AvroContainerInputFormat, AvroContainerOutputFormat} 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 +import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe +import org.apache.hadoop.hive.serde2.avro.AvroSerDe -import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, NativeCommand} +import org.apache.spark.sql.catalyst.util._ -import catalyst.analysis._ -import catalyst.plans.logical.{LogicalPlan, NativeCommand} -import catalyst.util._ +/* Implicit conversions */ +import scala.collection.JavaConversions._ object TestHive extends TestHiveContext(new SparkContext("local", "TestSQLContext", new SparkConf())) @@ -52,7 +51,7 @@ object TestHive * * 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 TestHive must be serialized. + * test cases that rely on TestHive must be serialized. */ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { self => 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 d20fd87f34f48..9aa9e173a8367 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 @@ -24,24 +24,18 @@ 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._ import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveDecimalObjectInspector import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharObjectInspector - -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.spark.{TaskContext, SparkException} -import catalyst.expressions.Cast import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ - -import scala.Some -import scala.collection.immutable.ListMap +import org.apache.spark.{TaskContext, SparkException} /* Implicits */ import scala.collection.JavaConversions._ @@ -194,20 +188,26 @@ case class InsertIntoHiveTable( * 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 (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 { + row.zip(oi.getAllStructFieldRefs: Seq[StructField]).foreach { case (data, field) => oi.setStructFieldData(struct, field, wrap(data, field.getFieldObjectInspector)) } struct + case (s: Seq[_], oi: ListObjectInspector) => val wrappedSeq = s.map(wrap(_, oi.getListElementObjectInspector)) seqAsJavaList(wrappedSeq) - case (obj, _) => obj + + case (obj, _) => + obj } def saveAsHiveFile( @@ -324,7 +324,7 @@ case class InsertIntoHiveTable( case (key, Some(value)) => key -> value case (key, None) => key -> "" // Should not reach here right now. } - val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols(), partitionSpec) + val partVals = MetaStoreUtils.getPvals(table.hiveQlTable.getPartCols, partitionSpec) db.validatePartitionNameCharacters(partVals) // inheritTableSpecs is set to true. It should be set to false for a IMPORT query // which is currently considered as a Hive native command. 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 5e775d6a048de..72ccd4f4a4302 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 @@ -18,22 +18,24 @@ package org.apache.spark.sql package hive -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.UDF import org.apache.hadoop.hive.ql.exec.{FunctionInfo, FunctionRegistry} import org.apache.hadoop.hive.ql.udf.generic._ -import org.apache.hadoop.hive.ql.exec.UDF +import org.apache.hadoop.hive.serde2.objectinspector._ +import org.apache.hadoop.hive.serde2.objectinspector.primitive._ +import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} -import catalyst.analysis -import catalyst.expressions._ -import catalyst.types -import catalyst.types._ +import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types +import org.apache.spark.sql.catalyst.types._ + +/* Implicit conversions */ +import scala.collection.JavaConversions._ object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionFactory with HiveInspectors { @@ -148,7 +150,7 @@ abstract class HiveUdf } case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { - import HiveFunctionRegistry._ + import org.apache.spark.sql.hive.HiveFunctionRegistry._ type UDFType = UDF @transient 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 index a12ab23946ed1..02ee2a0ebc354 100644 --- 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 @@ -20,7 +20,6 @@ package sql package hive package execution - import org.scalatest.{FunSuite, BeforeAndAfterAll} class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { @@ -35,4 +34,4 @@ class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { } } } -} \ No newline at end of file +} 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 8a5b97b7a05ef..e8fcc272354fa 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 @@ -20,12 +20,11 @@ package hive package execution import java.io._ -import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import catalyst.plans.logical.{ExplainCommand, NativeCommand} -import catalyst.plans._ -import catalyst.util._ +import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} +import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeCommand} +import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.Sort /** @@ -38,7 +37,8 @@ import org.apache.spark.sql.execution.Sort * 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 { +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 @@ -376,4 +376,4 @@ abstract class HiveComparisonTest extends FunSuite with BeforeAndAfterAll with G } } } -} \ No newline at end of file +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index d010023f789d4..16bcded8a4c65 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -19,11 +19,6 @@ package org.apache.spark.sql package hive package execution - -import java.io._ - -import util._ - /** * Runs the test cases that are included in the hive distribution. */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index f0a4ec3c02fa4..2d2f13333a65c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -19,9 +19,9 @@ package org.apache.spark.sql package hive package execution -import java.io._ +import java.io.File -import catalyst.util._ +import org.apache.spark.sql.catalyst.util._ /** * A framework for running the query tests that are listed as a set of text files. @@ -67,4 +67,4 @@ abstract class HiveQueryFileTest extends HiveComparisonTest { ignore(testCaseName) {} } } -} \ No newline at end of file +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 28a5d260b3185..b804634db1f5f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -19,7 +19,6 @@ package org.apache.spark.sql 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. */ @@ -141,4 +140,4 @@ class HiveQuerySuite extends HiveComparisonTest { sql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") } -} \ No newline at end of file +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 0dd79faa15396..996bd4efecd4c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -23,8 +23,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 HiveResolutionSuite extends HiveComparisonTest { - import TestHive._ - createQueryTest("table.attr", "SELECT src.key FROM src ORDER BY key LIMIT 1") @@ -62,4 +60,4 @@ class HiveResolutionSuite extends HiveComparisonTest { createQueryTest("tableName.attr from aliased subquery", "SELECT src.key FROM (SELECT * FROM src ORDER BY key LIMIT 1) a") */ -} \ No newline at end of file +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 8542f42aa9f43..bb65c91e2a651 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -19,10 +19,11 @@ package org.apache.spark.sql package hive package execution -import scala.collection.JavaConversions._ - import org.apache.spark.sql.hive.TestHive +/* Implicit conversions */ +import scala.collection.JavaConversions._ + /** * A set of test cases that validate partition and column pruning. */ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index ee90061c7cd0e..05ad85b622ac8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -19,21 +19,23 @@ package org.apache.spark.sql.parquet import java.io.File -import org.scalatest.{BeforeAndAfterEach, BeforeAndAfterAll, FunSuite} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} +import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.hive.TestHive - class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { - 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] = { + 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 = TestHive .parseSql(querystr) @@ -90,7 +92,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft override def beforeAll() { // write test data - ParquetTestData.writeFile + ParquetTestData.writeFile() // Override initial Parquet test table TestHive.catalog.registerTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) } @@ -151,7 +153,7 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft (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") + 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 80c29689ae3b589254a571da3ddb5f9c866ae534 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Sun, 23 Mar 2014 17:34:02 -0700 Subject: [PATCH 125/397] [SPARK-1212] Adding sparse data support and update KMeans Continue our discussions from https://github.com/apache/incubator-spark/pull/575 This PR is WIP because it depends on a SNAPSHOT version of breeze. Per previous discussions and benchmarks, I switched to breeze for linear algebra operations. @dlwh and I made some improvements to breeze to keep its performance comparable to the bare-bone implementation, including norm computation and squared distance. This is why this PR needs to depend on a SNAPSHOT version of breeze. @fommil , please find the notice of using netlib-core in `NOTICE`. This is following Apache's instructions on appropriate labeling. I'm going to update this PR to include: 1. Fast distance computation: using `\|a\|_2^2 + \|b\|_2^2 - 2 a^T b` when it doesn't introduce too much numerical error. The squared norms are pre-computed. Otherwise, computing the distance between the center (dense) and a point (possibly sparse) always takes O(n) time. 2. Some numbers about the performance. 3. A released version of breeze. @dlwh, a minor release of breeze will help this PR get merged early. Do you mind sharing breeze's release plan? Thanks! Author: Xiangrui Meng Closes #117 from mengxr/sparse-kmeans and squashes the following commits: 67b368d [Xiangrui Meng] fix SparseVector.toArray 5eda0de [Xiangrui Meng] update NOTICE 67abe31 [Xiangrui Meng] move ArrayRDDs to mllib.rdd 1da1033 [Xiangrui Meng] remove dependency on commons-math3 and compute EPSILON directly 9bb1b31 [Xiangrui Meng] optimize SparseVector.toArray 226d2cd [Xiangrui Meng] update Java friendly methods in Vectors 238ba34 [Xiangrui Meng] add VectorRDDs with a converter from RDD[Array[Double]] b28ba2f [Xiangrui Meng] add toArray to Vector e69b10c [Xiangrui Meng] remove examples/JavaKMeans.java, which is replaced by mllib/examples/JavaKMeans.java 72bde33 [Xiangrui Meng] clean up code for distance computation 712cb88 [Xiangrui Meng] make Vectors.sparse Java friendly 27858e4 [Xiangrui Meng] update breeze version to 0.7 07c3cf2 [Xiangrui Meng] change Mahout to breeze in doc use a simple lower bound to avoid unnecessary distance computation 6f5cdde [Xiangrui Meng] fix a bug in filtering finished runs 42512f2 [Xiangrui Meng] Merge branch 'master' into sparse-kmeans d6e6c07 [Xiangrui Meng] add predict(RDD[Vector]) to KMeansModel 42b4e50 [Xiangrui Meng] line feed at the end a4ace73 [Xiangrui Meng] Merge branch 'fast-dist' into sparse-kmeans 3ed1a24 [Xiangrui Meng] add doc to BreezeVectorWithSquaredNorm 0107e19 [Xiangrui Meng] update NOTICE 87bc755 [Xiangrui Meng] tuned the KMeans code: changed some for loops to while, use view to avoid copying arrays 0ff8046 [Xiangrui Meng] update KMeans to use fastSquaredDistance f355411 [Xiangrui Meng] add BreezeVectorWithSquaredNorm case class ab74f67 [Xiangrui Meng] add fastSquaredDistance for KMeans 4e7d5ca [Xiangrui Meng] minor style update 07ffaf2 [Xiangrui Meng] add dense/sparse vector data models and conversions to/from breeze vectors use breeze to implement KMeans in order to support both dense and sparse data --- NOTICE | 9 + .../org/apache/spark/examples/JavaKMeans.java | 138 ----------- .../spark/mllib/examples/JavaKMeans.java | 23 +- mllib/pom.xml | 5 + .../mllib/api/python/PythonMLLibAPI.scala | 12 +- .../spark/mllib/clustering/KMeans.scala | 233 ++++++++++++------ .../spark/mllib/clustering/KMeansModel.scala | 24 +- .../spark/mllib/clustering/LocalKMeans.scala | 58 +++-- .../apache/spark/mllib/linalg/Vectors.scala | 177 +++++++++++++ .../apache/spark/mllib/rdd/VectorRDDs.scala | 32 +++ .../org/apache/spark/mllib/util/MLUtils.scala | 61 ++++- .../mllib/clustering/JavaKMeansSuite.java | 88 +++---- .../spark/mllib/linalg/JavaVectorsSuite.java | 44 ++++ .../spark/mllib/clustering/KMeansSuite.scala | 175 +++++++------ .../linalg/BreezeVectorConversionSuite.scala | 58 +++++ .../spark/mllib/linalg/VectorsSuite.scala | 85 +++++++ .../spark/mllib/rdd/VectorRDDsSuite.scala | 33 +++ .../spark/mllib/util/LocalSparkContext.scala | 17 ++ .../spark/mllib/util/MLUtilsSuite.scala | 52 ++++ project/SparkBuild.scala | 3 +- 20 files changed, 930 insertions(+), 397 deletions(-) delete mode 100644 examples/src/main/java/org/apache/spark/examples/JavaKMeans.java create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala create mode 100644 mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala diff --git a/NOTICE b/NOTICE index dce0c4eaf31ed..42f6c3a835725 100644 --- a/NOTICE +++ b/NOTICE @@ -3,3 +3,12 @@ Copyright 2014 The Apache Software Foundation. This product includes software developed at The Apache Software Foundation (http://www.apache.org/). + +In addition, this product includes: + +- JUnit (http://www.junit.org) is a testing framework for Java. We included it + under the terms of the Eclipse Public License v1.0. + +- JTransforms (https://sites.google.com/site/piotrwendykier/software/jtransforms) + provides fast transforms in Java. It is tri-licensed, and we included it under + the terms of the Mozilla Public License v1.1. diff --git a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java deleted file mode 100644 index 2d797279d5bcc..0000000000000 --- a/examples/src/main/java/org/apache/spark/examples/JavaKMeans.java +++ /dev/null @@ -1,138 +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.examples; - -import scala.Tuple2; -import org.apache.spark.api.java.JavaPairRDD; -import org.apache.spark.api.java.JavaRDD; -import org.apache.spark.api.java.JavaSparkContext; -import org.apache.spark.api.java.function.Function; -import org.apache.spark.api.java.function.PairFunction; -import org.apache.spark.util.Vector; - -import java.util.List; -import java.util.Map; -import java.util.regex.Pattern; - -/** - * K-means clustering using Java API. - */ -public final class JavaKMeans { - - private static final Pattern SPACE = Pattern.compile(" "); - - /** Parses numbers split by whitespace to a vector */ - static Vector parseVector(String line) { - String[] splits = SPACE.split(line); - double[] data = new double[splits.length]; - int i = 0; - for (String s : splits) { - data[i] = Double.parseDouble(s); - i++; - } - return new Vector(data); - } - - /** Computes the vector to which the input vector is closest using squared distance */ - static int closestPoint(Vector p, List centers) { - int bestIndex = 0; - double closest = Double.POSITIVE_INFINITY; - for (int i = 0; i < centers.size(); i++) { - double tempDist = p.squaredDist(centers.get(i)); - if (tempDist < closest) { - closest = tempDist; - bestIndex = i; - } - } - return bestIndex; - } - - /** Computes the mean across all vectors in the input set of vectors */ - static Vector average(List ps) { - int numVectors = ps.size(); - Vector out = new Vector(ps.get(0).elements()); - // start from i = 1 since we already copied index 0 above - for (int i = 1; i < numVectors; i++) { - out.addInPlace(ps.get(i)); - } - return out.divide(numVectors); - } - - public static void main(String[] args) throws Exception { - if (args.length < 4) { - System.err.println("Usage: JavaKMeans "); - System.exit(1); - } - JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", - System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); - String path = args[1]; - int K = Integer.parseInt(args[2]); - double convergeDist = Double.parseDouble(args[3]); - - JavaRDD data = sc.textFile(path).map( - new Function() { - @Override - public Vector call(String line) { - return parseVector(line); - } - } - ).cache(); - - final List centroids = data.takeSample(false, K, 42); - - double tempDist; - do { - // allocate each vector to closest centroid - JavaPairRDD closest = data.mapToPair( - new PairFunction() { - @Override - public Tuple2 call(Vector vector) { - return new Tuple2( - closestPoint(vector, centroids), vector); - } - } - ); - - // group by cluster id and average the vectors within each cluster to compute centroids - JavaPairRDD> pointsGroup = closest.groupByKey(); - Map newCentroids = pointsGroup.mapValues( - new Function, Vector>() { - @Override - public Vector call(List ps) { - return average(ps); - } - }).collectAsMap(); - tempDist = 0.0; - for (int i = 0; i < K; i++) { - tempDist += centroids.get(i).squaredDist(newCentroids.get(i)); - } - for (Map.Entry t: newCentroids.entrySet()) { - centroids.set(t.getKey(), t.getValue()); - } - System.out.println("Finished iteration (delta = " + tempDist + ")"); - } while (tempDist > convergeDist); - - System.out.println("Final centers:"); - for (Vector c : centroids) { - System.out.println(c); - } - - System.exit(0); - - } -} diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index 76ebdccfd6b67..7b0ec36424e97 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -17,32 +17,33 @@ package org.apache.spark.mllib.examples; +import java.util.regex.Pattern; + import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.Function; import org.apache.spark.mllib.clustering.KMeans; import org.apache.spark.mllib.clustering.KMeansModel; - -import java.util.Arrays; -import java.util.regex.Pattern; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; /** * Example using MLLib KMeans from Java. */ public final class JavaKMeans { - static class ParsePoint implements Function { + private static class ParsePoint implements Function { private static final Pattern SPACE = Pattern.compile(" "); @Override - public double[] call(String line) { + public Vector call(String line) { String[] tok = SPACE.split(line); double[] point = new double[tok.length]; for (int i = 0; i < tok.length; ++i) { point[i] = Double.parseDouble(tok[i]); } - return point; + return Vectors.dense(point); } } @@ -65,15 +66,15 @@ public static void main(String[] args) { JavaSparkContext sc = new JavaSparkContext(args[0], "JavaKMeans", System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaKMeans.class)); - JavaRDD lines = sc.textFile(args[1]); + JavaRDD lines = sc.textFile(inputFile); - JavaRDD points = lines.map(new ParsePoint()); + JavaRDD points = lines.map(new ParsePoint()); - KMeansModel model = KMeans.train(points.rdd(), k, iterations, runs); + KMeansModel model = KMeans.train(points.rdd(), k, iterations, runs, KMeans.K_MEANS_PARALLEL()); System.out.println("Cluster centers:"); - for (double[] center : model.clusterCenters()) { - System.out.println(" " + Arrays.toString(center)); + for (Vector center : model.clusterCenters()) { + System.out.println(" " + center); } double cost = model.computeCost(points.rdd()); System.out.println("Cost: " + cost); diff --git a/mllib/pom.xml b/mllib/pom.xml index 9b65cb4b4ce3f..fec1cc94b2642 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -60,6 +60,11 @@ jblas 1.2.3 + + org.scalanlp + breeze_${scala.binary.version} + 0.7 + org.scalatest scalatest_${scala.binary.version} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index efe99a31beac4..3449c698da60b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -16,14 +16,16 @@ */ package org.apache.spark.mllib.api.python + +import java.nio.{ByteBuffer, ByteOrder} + import org.apache.spark.api.java.JavaRDD -import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.recommendation._ +import org.apache.spark.mllib.regression._ import org.apache.spark.rdd.RDD -import java.nio.ByteBuffer -import java.nio.ByteOrder /** * The Java stubs necessary for the Python mllib bindings. @@ -205,10 +207,10 @@ class PythonMLLibAPI extends Serializable { def trainKMeansModel(dataBytesJRDD: JavaRDD[Array[Byte]], k: Int, maxIterations: Int, runs: Int, initializationMode: String): java.util.List[java.lang.Object] = { - val data = dataBytesJRDD.rdd.map(xBytes => deserializeDoubleVector(xBytes)) + val data = dataBytesJRDD.rdd.map(xBytes => Vectors.dense(deserializeDoubleVector(xBytes))) val model = KMeans.train(data, k, maxIterations, runs, initializationMode) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleMatrix(model.clusterCenters)) + ret.add(serializeDoubleMatrix(model.clusterCenters.map(_.toArray))) ret } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index e508b76c3f8c5..b412738e3f00a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -19,16 +19,15 @@ package org.apache.spark.mllib.clustering import scala.collection.mutable.ArrayBuffer -import org.jblas.DoubleMatrix +import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} -import org.apache.spark.SparkContext +import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD -import org.apache.spark.Logging +import org.apache.spark.mllib.linalg.{Vector, Vectors} import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom - /** * K-means clustering with support for multiple parallel runs and a k-means++ like initialization * mode (the k-means|| algorithm by Bahmani et al). When multiple concurrent runs are requested, @@ -44,10 +43,7 @@ class KMeans private ( var initializationMode: String, var initializationSteps: Int, var epsilon: Double) - extends Serializable with Logging -{ - private type ClusterCenters = Array[Array[Double]] - + extends Serializable with Logging { def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) /** Set the number of clusters to create (k). Default: 2. */ @@ -113,28 +109,50 @@ class KMeans private ( * Train a K-means model on the given set of points; `data` should be cached for high * performance, because this is an iterative algorithm. */ - def run(data: RDD[Array[Double]]): KMeansModel = { - // TODO: check whether data is persistent; this needs RDD.storageLevel to be publicly readable + def run(data: RDD[Vector]): KMeansModel = { + // Compute squared norms and cache them. + val norms = data.map(v => breezeNorm(v.toBreeze, 2.0)) + norms.persist() + val breezeData = data.map(_.toBreeze).zip(norms).map { case (v, norm) => + new BreezeVectorWithNorm(v, norm) + } + val model = runBreeze(breezeData) + norms.unpersist() + model + } + + /** + * Implementation of K-Means using breeze. + */ + private def runBreeze(data: RDD[BreezeVectorWithNorm]): KMeansModel = { val sc = data.sparkContext + val initStartTime = System.nanoTime() + val centers = if (initializationMode == KMeans.RANDOM) { initRandom(data) } else { initKMeansParallel(data) } + val initTimeInSeconds = (System.nanoTime() - initStartTime) / 1e9 + logInfo(s"Initialization with $initializationMode took " + "%.3f".format(initTimeInSeconds) + + " seconds.") + val active = Array.fill(runs)(true) val costs = Array.fill(runs)(0.0) var activeRuns = new ArrayBuffer[Int] ++ (0 until runs) var iteration = 0 + val iterationStartTime = System.nanoTime() + // Execute iterations of Lloyd's algorithm until all runs have converged while (iteration < maxIterations && !activeRuns.isEmpty) { - type WeightedPoint = (DoubleMatrix, Long) + type WeightedPoint = (BV[Double], Long) def mergeContribs(p1: WeightedPoint, p2: WeightedPoint): WeightedPoint = { - (p1._1.addi(p2._1), p1._2 + p2._2) + (p1._1 += p2._1, p1._2 + p2._2) } val activeCenters = activeRuns.map(r => centers(r)).toArray @@ -144,16 +162,18 @@ class KMeans private ( val totalContribs = data.mapPartitions { points => val runs = activeCenters.length val k = activeCenters(0).length - val dims = activeCenters(0)(0).length + val dims = activeCenters(0)(0).vector.length - val sums = Array.fill(runs, k)(new DoubleMatrix(dims)) + val sums = Array.fill(runs, k)(BDV.zeros[Double](dims).asInstanceOf[BV[Double]]) val counts = Array.fill(runs, k)(0L) - for (point <- points; (centers, runIndex) <- activeCenters.zipWithIndex) { - val (bestCenter, cost) = KMeans.findClosest(centers, point) - costAccums(runIndex) += cost - sums(runIndex)(bestCenter).addi(new DoubleMatrix(point)) - counts(runIndex)(bestCenter) += 1 + points.foreach { point => + (0 until runs).foreach { i => + val (bestCenter, cost) = KMeans.findClosest(activeCenters(i), point) + costAccums(i) += cost + sums(i)(bestCenter) += point.vector + counts(i)(bestCenter) += 1 + } } val contribs = for (i <- 0 until runs; j <- 0 until k) yield { @@ -165,15 +185,18 @@ class KMeans private ( // Update the cluster centers and costs for each active run for ((run, i) <- activeRuns.zipWithIndex) { var changed = false - for (j <- 0 until k) { + var j = 0 + while (j < k) { val (sum, count) = totalContribs((i, j)) if (count != 0) { - val newCenter = sum.divi(count).data - if (MLUtils.squaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { + sum /= count.toDouble + val newCenter = new BreezeVectorWithNorm(sum) + if (KMeans.fastSquaredDistance(newCenter, centers(run)(j)) > epsilon * epsilon) { changed = true } centers(run)(j) = newCenter } + j += 1 } if (!changed) { active(run) = false @@ -186,17 +209,32 @@ class KMeans private ( iteration += 1 } - val bestRun = costs.zipWithIndex.min._2 - new KMeansModel(centers(bestRun)) + val iterationTimeInSeconds = (System.nanoTime() - iterationStartTime) / 1e9 + logInfo(s"Iterations took " + "%.3f".format(iterationTimeInSeconds) + " seconds.") + + if (iteration == maxIterations) { + logInfo(s"KMeans reached the max number of iterations: $maxIterations.") + } else { + logInfo(s"KMeans converged in $iteration iterations.") + } + + val (minCost, bestRun) = costs.zipWithIndex.min + + logInfo(s"The cost for the best run is $minCost.") + + new KMeansModel(centers(bestRun).map(c => Vectors.fromBreeze(c.vector))) } /** * Initialize `runs` sets of cluster centers at random. */ - private def initRandom(data: RDD[Array[Double]]): Array[ClusterCenters] = { + private def initRandom(data: RDD[BreezeVectorWithNorm]) + : Array[Array[BreezeVectorWithNorm]] = { // Sample all the cluster centers in one pass to avoid repeated scans val sample = data.takeSample(true, runs * k, new XORShiftRandom().nextInt()).toSeq - Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).toArray) + Array.tabulate(runs)(r => sample.slice(r * k, (r + 1) * k).map { v => + new BreezeVectorWithNorm(v.vector.toDenseVector, v.norm) + }.toArray) } /** @@ -208,38 +246,43 @@ class KMeans private ( * * The original paper can be found at http://theory.stanford.edu/~sergei/papers/vldb12-kmpar.pdf. */ - private def initKMeansParallel(data: RDD[Array[Double]]): Array[ClusterCenters] = { + private def initKMeansParallel(data: RDD[BreezeVectorWithNorm]) + : Array[Array[BreezeVectorWithNorm]] = { // Initialize each run's center to a random point val seed = new XORShiftRandom().nextInt() val sample = data.takeSample(true, runs, seed).toSeq - val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r))) + val centers = Array.tabulate(runs)(r => ArrayBuffer(sample(r).toDense)) // On each step, sample 2 * k points on average for each run with probability proportional // to their squared distance from that run's current centers - for (step <- 0 until initializationSteps) { - val centerArrays = centers.map(_.toArray) + var step = 0 + while (step < initializationSteps) { val sumCosts = data.flatMap { point => - for (r <- 0 until runs) yield (r, KMeans.pointCost(centerArrays(r), point)) + (0 until runs).map { r => + (r, KMeans.pointCost(centers(r), point)) + } }.reduceByKey(_ + _).collectAsMap() val chosen = data.mapPartitionsWithIndex { (index, points) => val rand = new XORShiftRandom(seed ^ (step << 16) ^ index) - for { - p <- points - r <- 0 until runs - if rand.nextDouble() < KMeans.pointCost(centerArrays(r), p) * 2 * k / sumCosts(r) - } yield (r, p) + points.flatMap { p => + (0 until runs).filter { r => + rand.nextDouble() < 2.0 * KMeans.pointCost(centers(r), p) * k / sumCosts(r) + }.map((_, p)) + } }.collect() - for ((r, p) <- chosen) { - centers(r) += p + chosen.foreach { case (r, p) => + centers(r) += p.toDense } + step += 1 } // Finally, we might have a set of more than k candidate centers for each run; weigh each // candidate by the number of points in the dataset mapping to it and run a local k-means++ // on the weighted centers to pick just k of them - val centerArrays = centers.map(_.toArray) val weightMap = data.flatMap { p => - for (r <- 0 until runs) yield ((r, KMeans.findClosest(centerArrays(r), p)._1), 1.0) + (0 until runs).map { r => + ((r, KMeans.findClosest(centers(r), p)._1), 1.0) + } }.reduceByKey(_ + _).collectAsMap() val finalCenters = (0 until runs).map { r => val myCenters = centers(r).toArray @@ -256,63 +299,75 @@ class KMeans private ( * Top-level methods for calling K-means clustering. */ object KMeans { + // Initialization mode names val RANDOM = "random" val K_MEANS_PARALLEL = "k-means||" + /** + * Trains a k-means model using the given set of parameters. + * + * @param data training points stored as `RDD[Array[Double]]` + * @param k number of clusters + * @param maxIterations max number of iterations + * @param runs number of parallel runs, defaults to 1. The best model is returned. + * @param initializationMode initialization model, either "random" or "k-means||" (default). + */ def train( - data: RDD[Array[Double]], + data: RDD[Vector], k: Int, maxIterations: Int, - runs: Int, - initializationMode: String) - : KMeansModel = - { + runs: Int = 1, + initializationMode: String = K_MEANS_PARALLEL): KMeansModel = { new KMeans().setK(k) - .setMaxIterations(maxIterations) - .setRuns(runs) - .setInitializationMode(initializationMode) - .run(data) - } - - def train(data: RDD[Array[Double]], k: Int, maxIterations: Int, runs: Int): KMeansModel = { - train(data, k, maxIterations, runs, K_MEANS_PARALLEL) - } - - def train(data: RDD[Array[Double]], k: Int, maxIterations: Int): KMeansModel = { - train(data, k, maxIterations, 1, K_MEANS_PARALLEL) + .setMaxIterations(maxIterations) + .setRuns(runs) + .setInitializationMode(initializationMode) + .run(data) } /** - * Return the index of the closest point in `centers` to `point`, as well as its distance. + * Returns the index of the closest center to the given point, as well as the squared distance. */ - private[mllib] def findClosest(centers: Array[Array[Double]], point: Array[Double]) - : (Int, Double) = - { + private[mllib] def findClosest( + centers: TraversableOnce[BreezeVectorWithNorm], + point: BreezeVectorWithNorm): (Int, Double) = { var bestDistance = Double.PositiveInfinity var bestIndex = 0 - for (i <- 0 until centers.length) { - val distance = MLUtils.squaredDistance(point, centers(i)) - if (distance < bestDistance) { - bestDistance = distance - bestIndex = i + var i = 0 + centers.foreach { center => + // Since `\|a - b\| \geq |\|a\| - \|b\||`, we can use this lower bound to avoid unnecessary + // distance computation. + var lowerBoundOfSqDist = center.norm - point.norm + lowerBoundOfSqDist = lowerBoundOfSqDist * lowerBoundOfSqDist + if (lowerBoundOfSqDist < bestDistance) { + val distance: Double = fastSquaredDistance(center, point) + if (distance < bestDistance) { + bestDistance = distance + bestIndex = i + } } + i += 1 } (bestIndex, bestDistance) } /** - * Return the K-means cost of a given point against the given cluster centers. + * Returns the K-means cost of a given point against the given cluster centers. */ - private[mllib] def pointCost(centers: Array[Array[Double]], point: Array[Double]): Double = { - var bestDistance = Double.PositiveInfinity - for (i <- 0 until centers.length) { - val distance = MLUtils.squaredDistance(point, centers(i)) - if (distance < bestDistance) { - bestDistance = distance - } - } - bestDistance + private[mllib] def pointCost( + centers: TraversableOnce[BreezeVectorWithNorm], + point: BreezeVectorWithNorm): Double = + findClosest(centers, point)._2 + + /** + * Returns the squared Euclidean distance between two vectors computed by + * [[org.apache.spark.mllib.util.MLUtils#fastSquaredDistance]]. + */ + private[clustering] + def fastSquaredDistance(v1: BreezeVectorWithNorm, v2: BreezeVectorWithNorm) + : Double = { + MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } def main(args: Array[String]) { @@ -323,14 +378,34 @@ object KMeans { val (master, inputFile, k, iters) = (args(0), args(1), args(2).toInt, args(3).toInt) val runs = if (args.length >= 5) args(4).toInt else 1 val sc = new SparkContext(master, "KMeans") - val data = sc.textFile(inputFile).map(line => line.split(' ').map(_.toDouble)).cache() + val data = sc.textFile(inputFile) + .map(line => Vectors.dense(line.split(' ').map(_.toDouble))) + .cache() val model = KMeans.train(data, k, iters, runs) val cost = model.computeCost(data) println("Cluster centers:") for (c <- model.clusterCenters) { - println(" " + c.mkString(" ")) + println(" " + c) } println("Cost: " + cost) System.exit(0) } } + +/** + * A breeze vector with its norm for fast distance computation. + * + * @see [[org.apache.spark.mllib.clustering.KMeans#fastSquaredDistance]] + */ +private[clustering] +class BreezeVectorWithNorm(val vector: BV[Double], val norm: Double) extends Serializable { + + def this(vector: BV[Double]) = this(vector, breezeNorm(vector, 2.0)) + + def this(array: Array[Double]) = this(new BDV[Double](array)) + + def this(v: Vector) = this(v.toBreeze) + + /** Converts the vector to a dense vector. */ + def toDense = new BreezeVectorWithNorm(vector.toDenseVector, norm) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala index 980be931576dc..18abbf2758b86 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeansModel.scala @@ -19,24 +19,36 @@ package org.apache.spark.mllib.clustering import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector /** * A clustering model for K-means. Each point belongs to the cluster with the closest center. */ -class KMeansModel(val clusterCenters: Array[Array[Double]]) extends Serializable { +class KMeansModel(val clusterCenters: Array[Vector]) extends Serializable { + /** Total number of clusters. */ def k: Int = clusterCenters.length - /** Return the cluster index that a given point belongs to. */ - def predict(point: Array[Double]): Int = { - KMeans.findClosest(clusterCenters, point)._1 + /** Returns the cluster index that a given point belongs to. */ + def predict(point: Vector): Int = { + KMeans.findClosest(clusterCentersWithNorm, new BreezeVectorWithNorm(point))._1 + } + + /** Maps given points to their cluster indices. */ + def predict(points: RDD[Vector]): RDD[Int] = { + val centersWithNorm = clusterCentersWithNorm + points.map(p => KMeans.findClosest(centersWithNorm, new BreezeVectorWithNorm(p))._1) } /** * Return the K-means cost (sum of squared distances of points to their nearest center) for this * model on the given data. */ - def computeCost(data: RDD[Array[Double]]): Double = { - data.map(p => KMeans.pointCost(clusterCenters, p)).sum() + def computeCost(data: RDD[Vector]): Double = { + val centersWithNorm = clusterCentersWithNorm + data.map(p => KMeans.pointCost(centersWithNorm, new BreezeVectorWithNorm(p))).sum() } + + private def clusterCentersWithNorm: Iterable[BreezeVectorWithNorm] = + clusterCenters.map(new BreezeVectorWithNorm(_)) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala index baf8251d8fc53..2e3a4ce783de7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/LocalKMeans.scala @@ -19,35 +19,37 @@ package org.apache.spark.mllib.clustering import scala.util.Random -import org.jblas.{DoubleMatrix, SimpleBlas} +import breeze.linalg.{Vector => BV, DenseVector => BDV, norm => breezeNorm} + +import org.apache.spark.Logging /** * An utility object to run K-means locally. This is private to the ML package because it's used * in the initialization of KMeans but not meant to be publicly exposed. */ -private[mllib] object LocalKMeans { +private[mllib] object LocalKMeans extends Logging { + /** * Run K-means++ on the weighted point set `points`. This first does the K-means++ - * initialization procedure and then roudns of Lloyd's algorithm. + * initialization procedure and then rounds of Lloyd's algorithm. */ def kMeansPlusPlus( seed: Int, - points: Array[Array[Double]], + points: Array[BreezeVectorWithNorm], weights: Array[Double], k: Int, - maxIterations: Int) - : Array[Array[Double]] = - { + maxIterations: Int + ): Array[BreezeVectorWithNorm] = { val rand = new Random(seed) - val dimensions = points(0).length - val centers = new Array[Array[Double]](k) + val dimensions = points(0).vector.length + val centers = new Array[BreezeVectorWithNorm](k) - // Initialize centers by sampling using the k-means++ procedure - centers(0) = pickWeighted(rand, points, weights) + // Initialize centers by sampling using the k-means++ procedure. + centers(0) = pickWeighted(rand, points, weights).toDense for (i <- 1 until k) { // Pick the next center with a probability proportional to cost under current centers - val curCenters = centers.slice(0, i) - val sum = points.zip(weights).map { case (p, w) => + val curCenters = centers.view.take(i) + val sum = points.view.zip(weights).map { case (p, w) => w * KMeans.pointCost(curCenters, p) }.sum val r = rand.nextDouble() * sum @@ -57,7 +59,7 @@ private[mllib] object LocalKMeans { cumulativeScore += weights(j) * KMeans.pointCost(curCenters, points(j)) j += 1 } - centers(i) = points(j-1) + centers(i) = points(j-1).toDense } // Run up to maxIterations iterations of Lloyd's algorithm @@ -66,29 +68,43 @@ private[mllib] object LocalKMeans { var moved = true while (moved && iteration < maxIterations) { moved = false - val sums = Array.fill(k)(new DoubleMatrix(dimensions)) val counts = Array.fill(k)(0.0) - for ((p, i) <- points.zipWithIndex) { + val sums = Array.fill(k)( + BDV.zeros[Double](dimensions).asInstanceOf[BV[Double]] + ) + var i = 0 + while (i < points.length) { + val p = points(i) val index = KMeans.findClosest(centers, p)._1 - SimpleBlas.axpy(weights(i), new DoubleMatrix(p), sums(index)) + breeze.linalg.axpy(weights(i), p.vector, sums(index)) counts(index) += weights(i) if (index != oldClosest(i)) { moved = true oldClosest(i) = index } + i += 1 } // Update centers - for (i <- 0 until k) { - if (counts(i) == 0.0) { + var j = 0 + while (j < k) { + if (counts(j) == 0.0) { // Assign center to a random point - centers(i) = points(rand.nextInt(points.length)) + centers(j) = points(rand.nextInt(points.length)).toDense } else { - centers(i) = sums(i).divi(counts(i)).data + sums(j) /= counts(j) + centers(j) = new BreezeVectorWithNorm(sums(j)) } + j += 1 } iteration += 1 } + if (iteration == maxIterations) { + logInfo(s"Local KMeans++ reached the max number of iterations: $maxIterations.") + } else { + logInfo(s"Local KMeans++ converged in $iteration iterations.") + } + centers } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala new file mode 100644 index 0000000000000..01c1501548f87 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -0,0 +1,177 @@ +/* + * 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.mllib.linalg + +import java.lang.{Iterable => JavaIterable, Integer => JavaInteger, Double => JavaDouble} +import java.util.Arrays + +import scala.annotation.varargs +import scala.collection.JavaConverters._ + +import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV} + +/** + * Represents a numeric vector, whose index type is Int and value type is Double. + */ +trait Vector extends Serializable { + + /** + * Size of the vector. + */ + def size: Int + + /** + * Converts the instance to a double array. + */ + def toArray: Array[Double] + + override def equals(other: Any): Boolean = { + other match { + case v: Vector => + Arrays.equals(this.toArray, v.toArray) + case _ => false + } + } + + override def hashCode(): Int = Arrays.hashCode(this.toArray) + + /** + * Converts the instance to a breeze vector. + */ + private[mllib] def toBreeze: BV[Double] +} + +/** + * Factory methods for [[org.apache.spark.mllib.linalg.Vector]]. + */ +object Vectors { + + /** + * Creates a dense vector. + */ + @varargs + def dense(firstValue: Double, otherValues: Double*): Vector = + new DenseVector((firstValue +: otherValues).toArray) + + // A dummy implicit is used to avoid signature collision with the one generated by @varargs. + /** + * Creates a dense vector from a double array. + */ + def dense(values: Array[Double]): Vector = new DenseVector(values) + + /** + * Creates a sparse vector providing its index array and value array. + * + * @param size vector size. + * @param indices index array, must be strictly increasing. + * @param values value array, must have the same length as indices. + */ + def sparse(size: Int, indices: Array[Int], values: Array[Double]): Vector = + new SparseVector(size, indices, values) + + /** + * Creates a sparse vector using unordered (index, value) pairs. + * + * @param size vector size. + * @param elements vector elements in (index, value) pairs. + */ + def sparse(size: Int, elements: Seq[(Int, Double)]): Vector = { + require(size > 0) + + val (indices, values) = elements.sortBy(_._1).unzip + var prev = -1 + indices.foreach { i => + require(prev < i, s"Found duplicate indices: $i.") + prev = i + } + require(prev < size) + + new SparseVector(size, indices.toArray, values.toArray) + } + + /** + * Creates a sparse vector using unordered (index, value) pairs in a Java friendly way. + * + * @param size vector size. + * @param elements vector elements in (index, value) pairs. + */ + def sparse(size: Int, elements: JavaIterable[(JavaInteger, JavaDouble)]): Vector = { + sparse(size, elements.asScala.map { case (i, x) => + (i.intValue(), x.doubleValue()) + }.toSeq) + } + + /** + * Creates a vector instance from a breeze vector. + */ + private[mllib] def fromBreeze(breezeVector: BV[Double]): Vector = { + breezeVector match { + case v: BDV[Double] => + require(v.offset == 0, s"Do not support non-zero offset ${v.offset}.") + require(v.stride == 1, s"Do not support stride other than 1, but got ${v.stride}.") + new DenseVector(v.data) + case v: BSV[Double] => + new SparseVector(v.length, v.index, v.data) + case v: BV[_] => + sys.error("Unsupported Breeze vector type: " + v.getClass.getName) + } + } +} + +/** + * A dense vector represented by a value array. + */ +class DenseVector(val values: Array[Double]) extends Vector { + + override def size: Int = values.length + + override def toString: String = values.mkString("[", ",", "]") + + override def toArray: Array[Double] = values + + private[mllib] override def toBreeze: BV[Double] = new BDV[Double](values) +} + +/** + * A sparse vector represented by an index array and an value array. + * + * @param n size of the vector. + * @param indices index array, assume to be strictly increasing. + * @param values value array, must have the same length as the index array. + */ +class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double]) extends Vector { + + override def size: Int = n + + override def toString: String = { + "(" + n + "," + indices.zip(values).mkString("[", "," ,"]") + ")" + } + + override def toArray: Array[Double] = { + val data = new Array[Double](n) + var i = 0 + val nnz = indices.length + while (i < nnz) { + data(indices(i)) = values(i) + i += 1 + } + data + } + + private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, n) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.scala new file mode 100644 index 0000000000000..9096d6a1a16d6 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/VectorRDDs.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.mllib.rdd + +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Vectors, Vector} + +/** + * Factory methods for `RDD[Vector]`. + */ +object VectorRDDs { + + /** + * Converts an `RDD[Array[Double]]` to `RDD[Vector]`. + */ + def fromArrayRDD(rdd: RDD[Array[Double]]): RDD[Vector] = rdd.map(v => Vectors.dense(v)) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 64c6136a8b89d..08cd9ab05547b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -22,13 +22,24 @@ import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.jblas.DoubleMatrix + import org.apache.spark.mllib.regression.LabeledPoint +import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} + /** * Helper methods to load, save and pre-process data used in ML Lib. */ object MLUtils { + private[util] lazy val EPSILON = { + var eps = 1.0 + while ((1.0 + (eps / 2.0)) != 1.0) { + eps /= 2.0 + } + eps + } + /** * Load labeled data from a file. The data format used here is * , ... @@ -106,18 +117,46 @@ object MLUtils { } /** - * Return the squared Euclidean distance between two vectors. + * Returns the squared Euclidean distance between two vectors. The following formula will be used + * if it does not introduce too much numerical error: + *
    +   *   \|a - b\|_2^2 = \|a\|_2^2 + \|b\|_2^2 - 2 a^T b.
    +   * 
    + * When both vector norms are given, this is faster than computing the squared distance directly, + * especially when one of the vectors is a sparse vector. + * + * @param v1 the first vector + * @param norm1 the norm of the first vector, non-negative + * @param v2 the second vector + * @param norm2 the norm of the second vector, non-negative + * @param precision desired relative precision for the squared distance + * @return squared distance between v1 and v2 within the specified precision */ - def squaredDistance(v1: Array[Double], v2: Array[Double]): Double = { - if (v1.length != v2.length) { - throw new IllegalArgumentException("Vector sizes don't match") - } - var i = 0 - var sum = 0.0 - while (i < v1.length) { - sum += (v1(i) - v2(i)) * (v1(i) - v2(i)) - i += 1 + private[mllib] def fastSquaredDistance( + v1: BV[Double], + norm1: Double, + v2: BV[Double], + norm2: Double, + precision: Double = 1e-6): Double = { + val n = v1.size + require(v2.size == n) + require(norm1 >= 0.0 && norm2 >= 0.0) + val sumSquaredNorm = norm1 * norm1 + norm2 * norm2 + val normDiff = norm1 - norm2 + var sqDist = 0.0 + val precisionBound1 = 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) + if (precisionBound1 < precision) { + sqDist = sumSquaredNorm - 2.0 * v1.dot(v2) + } else if (v1.isInstanceOf[BSV[Double]] || v2.isInstanceOf[BSV[Double]]) { + val dot = v1.dot(v2) + sqDist = math.max(sumSquaredNorm - 2.0 * dot, 0.0) + val precisionBound2 = EPSILON * (sumSquaredNorm + 2.0 * math.abs(dot)) / (sqDist + EPSILON) + if (precisionBound2 > precision) { + sqDist = breezeSquaredDistance(v1, v2) + } + } else { + sqDist = breezeSquaredDistance(v1, v2) } - sum + sqDist } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java index 33b99f4bd3bcf..49a614bd90cab 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/clustering/JavaKMeansSuite.java @@ -18,16 +18,19 @@ package org.apache.spark.mllib.clustering; import java.io.Serializable; -import java.util.ArrayList; import java.util.List; import org.junit.After; -import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import static org.junit.Assert.*; + +import com.google.common.collect.Lists; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; public class JavaKMeansSuite implements Serializable { private transient JavaSparkContext sc; @@ -44,72 +47,45 @@ public void tearDown() { System.clearProperty("spark.driver.port"); } - // L1 distance between two points - double distance1(double[] v1, double[] v2) { - double distance = 0.0; - for (int i = 0; i < v1.length; ++i) { - distance = Math.max(distance, Math.abs(v1[i] - v2[i])); - } - return distance; - } - - // Assert that two sets of points are equal, within EPSILON tolerance - void assertSetsEqual(double[][] v1, double[][] v2) { - double EPSILON = 1e-4; - Assert.assertTrue(v1.length == v2.length); - for (int i = 0; i < v1.length; ++i) { - double minDistance = Double.MAX_VALUE; - for (int j = 0; j < v2.length; ++j) { - minDistance = Math.min(minDistance, distance1(v1[i], v2[j])); - } - Assert.assertTrue(minDistance <= EPSILON); - } - - for (int i = 0; i < v2.length; ++i) { - double minDistance = Double.MAX_VALUE; - for (int j = 0; j < v1.length; ++j) { - minDistance = Math.min(minDistance, distance1(v2[i], v1[j])); - } - Assert.assertTrue(minDistance <= EPSILON); - } - } - - @Test public void runKMeansUsingStaticMethods() { - List points = new ArrayList(); - points.add(new double[]{1.0, 2.0, 6.0}); - points.add(new double[]{1.0, 3.0, 0.0}); - points.add(new double[]{1.0, 4.0, 6.0}); + List points = Lists.newArrayList( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) + ); - double[][] expectedCenter = { {1.0, 3.0, 4.0} }; + Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0); - JavaRDD data = sc.parallelize(points, 2); - KMeansModel model = KMeans.train(data.rdd(), 1, 1); - assertSetsEqual(model.clusterCenters(), expectedCenter); + JavaRDD data = sc.parallelize(points, 2); + KMeansModel model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.K_MEANS_PARALLEL()); + assertEquals(1, model.clusterCenters().length); + assertEquals(expectedCenter, model.clusterCenters()[0]); model = KMeans.train(data.rdd(), 1, 1, 1, KMeans.RANDOM()); - assertSetsEqual(model.clusterCenters(), expectedCenter); + assertEquals(expectedCenter, model.clusterCenters()[0]); } @Test public void runKMeansUsingConstructor() { - List points = new ArrayList(); - points.add(new double[]{1.0, 2.0, 6.0}); - points.add(new double[]{1.0, 3.0, 0.0}); - points.add(new double[]{1.0, 4.0, 6.0}); + List points = Lists.newArrayList( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) + ); - double[][] expectedCenter = { {1.0, 3.0, 4.0} }; + Vector expectedCenter = Vectors.dense(1.0, 3.0, 4.0); - JavaRDD data = sc.parallelize(points, 2); + JavaRDD data = sc.parallelize(points, 2); KMeansModel model = new KMeans().setK(1).setMaxIterations(5).run(data.rdd()); - assertSetsEqual(model.clusterCenters(), expectedCenter); - - model = new KMeans().setK(1) - .setMaxIterations(1) - .setRuns(1) - .setInitializationMode(KMeans.RANDOM()) - .run(data.rdd()); - assertSetsEqual(model.clusterCenters(), expectedCenter); + assertEquals(1, model.clusterCenters().length); + assertEquals(expectedCenter, model.clusterCenters()[0]); + + model = new KMeans() + .setK(1) + .setMaxIterations(1) + .setInitializationMode(KMeans.RANDOM()) + .run(data.rdd()); + assertEquals(expectedCenter, model.clusterCenters()[0]); } } diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java new file mode 100644 index 0000000000000..2c4d795f96e4e --- /dev/null +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -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.mllib.linalg; + +import java.io.Serializable; + +import com.google.common.collect.Lists; + +import scala.Tuple2; + +import org.junit.Test; +import static org.junit.Assert.*; + +public class JavaVectorsSuite implements Serializable { + + @Test + public void denseArrayConstruction() { + Vector v = Vectors.dense(1.0, 2.0, 3.0); + assertArrayEquals(new double[]{1.0, 2.0, 3.0}, v.toArray(), 0.0); + } + + @Test + public void sparseArrayConstruction() { + Vector v = Vectors.sparse(3, Lists.newArrayList( + new Tuple2(0, 2.0), + new Tuple2(2, 3.0))); + assertArrayEquals(new double[]{2.0, 0.0, 3.0}, v.toArray(), 0.0); + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala index 4ef1d1f64ff06..560a4ad71a4de 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/clustering/KMeansSuite.scala @@ -17,127 +17,139 @@ package org.apache.spark.mllib.clustering - -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.Vectors class KMeansSuite extends FunSuite with LocalSparkContext { - val EPSILON = 1e-4 - import KMeans.{RANDOM, K_MEANS_PARALLEL} - def prettyPrint(point: Array[Double]): String = point.mkString("(", ", ", ")") - - def prettyPrint(points: Array[Array[Double]]): String = { - points.map(prettyPrint).mkString("(", "; ", ")") - } - - // L1 distance between two points - def distance1(v1: Array[Double], v2: Array[Double]): Double = { - v1.zip(v2).map{ case (a, b) => math.abs(a-b) }.max - } - - // Assert that two vectors are equal within tolerance EPSILON - def assertEqual(v1: Array[Double], v2: Array[Double]) { - def errorMessage = prettyPrint(v1) + " did not equal " + prettyPrint(v2) - assert(v1.length == v2.length, errorMessage) - assert(distance1(v1, v2) <= EPSILON, errorMessage) - } - - // Assert that two sets of points are equal, within EPSILON tolerance - def assertSetsEqual(set1: Array[Array[Double]], set2: Array[Array[Double]]) { - def errorMessage = prettyPrint(set1) + " did not equal " + prettyPrint(set2) - assert(set1.length == set2.length, errorMessage) - for (v <- set1) { - val closestDistance = set2.map(w => distance1(v, w)).min - if (closestDistance > EPSILON) { - fail(errorMessage) - } - } - for (v <- set2) { - val closestDistance = set1.map(w => distance1(v, w)).min - if (closestDistance > EPSILON) { - fail(errorMessage) - } - } - } - test("single cluster") { val data = sc.parallelize(Array( - Array(1.0, 2.0, 6.0), - Array(1.0, 3.0, 0.0), - Array(1.0, 4.0, 6.0) + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) )) + val center = Vectors.dense(1.0, 3.0, 4.0) + // No matter how many runs or iterations we use, we should get one cluster, // centered at the mean of the points var model = KMeans.train(data, k=1, maxIterations=1) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=2) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train( data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) } test("single cluster with big dataset") { val smallData = Array( - Array(1.0, 2.0, 6.0), - Array(1.0, 3.0, 0.0), - Array(1.0, 4.0, 6.0) + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0) ) val data = sc.parallelize((1 to 100).flatMap(_ => smallData), 4) // No matter how many runs or iterations we use, we should get one cluster, // centered at the mean of the points + val center = Vectors.dense(1.0, 3.0, 4.0) + var model = KMeans.train(data, k=1, maxIterations=1) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.size === 1) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=2) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=5) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) - assertSetsEqual(model.clusterCenters, Array(Array(1.0, 3.0, 4.0))) + assert(model.clusterCenters.head === center) + } + + test("single cluster with sparse data") { + + val n = 10000 + val data = sc.parallelize((1 to 100).flatMap { i => + val x = i / 1000.0 + Array( + Vectors.sparse(n, Seq((0, 1.0 + x), (1, 2.0), (2, 6.0))), + Vectors.sparse(n, Seq((0, 1.0 - x), (1, 2.0), (2, 6.0))), + Vectors.sparse(n, Seq((0, 1.0), (1, 3.0 + x))), + Vectors.sparse(n, Seq((0, 1.0), (1, 3.0 - x))), + Vectors.sparse(n, Seq((0, 1.0), (1, 4.0), (2, 6.0 + x))), + Vectors.sparse(n, Seq((0, 1.0), (1, 4.0), (2, 6.0 - x))) + ) + }, 4) + + data.persist() + + // No matter how many runs or iterations we use, we should get one cluster, + // centered at the mean of the points + + val center = Vectors.sparse(n, Seq((0, 1.0), (1, 3.0), (2, 4.0))) + + var model = KMeans.train(data, k=1, maxIterations=1) + assert(model.clusterCenters.head === center) + + model = KMeans.train(data, k=1, maxIterations=2) + assert(model.clusterCenters.head === center) + + model = KMeans.train(data, k=1, maxIterations=5) + assert(model.clusterCenters.head === center) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assert(model.clusterCenters.head === center) + + model = KMeans.train(data, k=1, maxIterations=1, runs=5) + assert(model.clusterCenters.head === center) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=RANDOM) + assert(model.clusterCenters.head === center) + + model = KMeans.train(data, k=1, maxIterations=1, runs=1, initializationMode=K_MEANS_PARALLEL) + assert(model.clusterCenters.head === center) + + data.unpersist() } test("k-means|| initialization") { - val points = Array( - Array(1.0, 2.0, 6.0), - Array(1.0, 3.0, 0.0), - Array(1.0, 4.0, 6.0), - Array(1.0, 0.0, 1.0), - Array(1.0, 1.0, 1.0) + val points = Seq( + Vectors.dense(1.0, 2.0, 6.0), + Vectors.dense(1.0, 3.0, 0.0), + Vectors.dense(1.0, 4.0, 6.0), + Vectors.dense(1.0, 0.0, 1.0), + Vectors.dense(1.0, 1.0, 1.0) ) val rdd = sc.parallelize(points) @@ -146,14 +158,39 @@ class KMeansSuite extends FunSuite with LocalSparkContext { // unselected point as long as it hasn't yet selected all of them var model = KMeans.train(rdd, k=5, maxIterations=1) - assertSetsEqual(model.clusterCenters, points) + assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Iterations of Lloyd's should not change the answer either model = KMeans.train(rdd, k=5, maxIterations=10) - assertSetsEqual(model.clusterCenters, points) + assert(Set(model.clusterCenters: _*) === Set(points: _*)) // Neither should more runs model = KMeans.train(rdd, k=5, maxIterations=10, runs=5) - assertSetsEqual(model.clusterCenters, points) + assert(Set(model.clusterCenters: _*) === Set(points: _*)) + } + + test("two clusters") { + val points = Seq( + Vectors.dense(0.0, 0.0), + Vectors.dense(0.0, 0.1), + Vectors.dense(0.1, 0.0), + Vectors.dense(9.0, 0.0), + Vectors.dense(9.0, 0.2), + Vectors.dense(9.2, 0.0) + ) + val rdd = sc.parallelize(points, 3) + + for (initMode <- Seq(RANDOM, K_MEANS_PARALLEL)) { + // Two iterations are sufficient no matter where the initial centers are. + val model = KMeans.train(rdd, k = 2, maxIterations = 2, runs = 1, initMode) + + val predicts = model.predict(rdd).collect() + + assert(predicts(0) === predicts(1)) + assert(predicts(0) === predicts(2)) + assert(predicts(3) === predicts(4)) + assert(predicts(3) === predicts(5)) + assert(predicts(0) != predicts(3)) + } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala new file mode 100644 index 0000000000000..aacaa300849aa --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeVectorConversionSuite.scala @@ -0,0 +1,58 @@ +/* + * 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.mllib.linalg + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} + +/** + * Test Breeze vector conversions. + */ +class BreezeVectorConversionSuite extends FunSuite { + + val arr = Array(0.1, 0.2, 0.3, 0.4) + val n = 20 + val indices = Array(0, 3, 5, 10, 13) + val values = Array(0.1, 0.5, 0.3, -0.8, -1.0) + + test("dense to breeze") { + val vec = Vectors.dense(arr) + assert(vec.toBreeze === new BDV[Double](arr)) + } + + test("sparse to breeze") { + val vec = Vectors.sparse(n, indices, values) + assert(vec.toBreeze === new BSV[Double](indices, values, n)) + } + + test("dense breeze to vector") { + val breeze = new BDV[Double](arr) + val vec = Vectors.fromBreeze(breeze).asInstanceOf[DenseVector] + assert(vec.size === arr.length) + assert(vec.values.eq(arr), "should not copy data") + } + + test("sparse breeze to vector") { + val breeze = new BSV[Double](indices, values, n) + val vec = Vectors.fromBreeze(breeze).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices.eq(indices), "should not copy data") + assert(vec.values.eq(values), "should not copy data") + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.scala new file mode 100644 index 0000000000000..8a200310e0bb1 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/VectorsSuite.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.mllib.linalg + +import org.scalatest.FunSuite + +class VectorsSuite extends FunSuite { + + val arr = Array(0.1, 0.0, 0.3, 0.4) + val n = 4 + val indices = Array(0, 2, 3) + val values = Array(0.1, 0.3, 0.4) + + test("dense vector construction with varargs") { + val vec = Vectors.dense(arr).asInstanceOf[DenseVector] + assert(vec.size === arr.length) + assert(vec.values.eq(arr)) + } + + test("dense vector construction from a double array") { + val vec = Vectors.dense(arr).asInstanceOf[DenseVector] + assert(vec.size === arr.length) + assert(vec.values.eq(arr)) + } + + test("sparse vector construction") { + val vec = Vectors.sparse(n, indices, values).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices.eq(indices)) + assert(vec.values.eq(values)) + } + + test("sparse vector construction with unordered elements") { + val vec = Vectors.sparse(n, indices.zip(values).reverse).asInstanceOf[SparseVector] + assert(vec.size === n) + assert(vec.indices === indices) + assert(vec.values === values) + } + + test("dense to array") { + val vec = Vectors.dense(arr).asInstanceOf[DenseVector] + assert(vec.toArray.eq(arr)) + } + + test("sparse to array") { + val vec = Vectors.sparse(n, indices, values).asInstanceOf[SparseVector] + assert(vec.toArray === arr) + } + + test("vector equals") { + val dv1 = Vectors.dense(arr.clone()) + val dv2 = Vectors.dense(arr.clone()) + val sv1 = Vectors.sparse(n, indices.clone(), values.clone()) + val sv2 = Vectors.sparse(n, indices.clone(), values.clone()) + + val vectors = Seq(dv1, dv2, sv1, sv2) + + for (v <- vectors; u <- vectors) { + assert(v === u) + assert(v.## === u.##) + } + + val another = Vectors.dense(0.1, 0.2, 0.3, 0.4) + + for (v <- vectors) { + assert(v != another) + assert(v.## != another.##) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.scala new file mode 100644 index 0000000000000..692f025e959ae --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/VectorRDDsSuite.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.mllib.rdd + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.LocalSparkContext + +class VectorRDDsSuite extends FunSuite with LocalSparkContext { + + test("from array rdd") { + val data = Seq(Array(1.0, 2.0), Array(3.0, 4.0)) + val arrayRdd = sc.parallelize(data, 2) + val vectorRdd = VectorRDDs.fromArrayRDD(arrayRdd) + assert(arrayRdd.collect().map(v => Vectors.dense(v)) === vectorRdd.collect()) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala index 7d840043e5c6b..212fbe9288f0d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LocalSparkContext.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.mllib.util import org.scalatest.Suite diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala new file mode 100644 index 0000000000000..60f053b381305 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -0,0 +1,52 @@ +/* + * 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.mllib.util + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNorm, + squaredDistance => breezeSquaredDistance} + +import org.apache.spark.mllib.util.MLUtils._ + +class MLUtilsSuite extends FunSuite { + + test("epsilon computation") { + assert(1.0 + EPSILON > 1.0, s"EPSILON is too small: $EPSILON.") + assert(1.0 + EPSILON / 2.0 === 1.0, s"EPSILON is too big: $EPSILON.") + } + + test("fast squared distance") { + val a = (30 to 0 by -1).map(math.pow(2.0, _)).toArray + val n = a.length + val v1 = new BDV[Double](a) + val norm1 = breezeNorm(v1, 2.0) + val precision = 1e-6 + for (m <- 0 until n) { + val indices = (0 to m).toArray + val values = indices.map(i => a(i)) + val v2 = new BSV[Double](indices, values, n) + val norm2 = breezeNorm(v2, 2.0) + val squaredDist = breezeSquaredDistance(v1, v2) + val fastSquaredDist1 = fastSquaredDistance(v1, norm1, v2, norm2, precision) + assert((fastSquaredDist1 - squaredDist) <= precision * squaredDist, s"failed with m = $m") + val fastSquaredDist2 = fastSquaredDistance(v1, norm1, v2.toDenseVector, norm2, precision) + assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m") + } + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b08fb26adfe68..1969486f79df1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -366,7 +366,8 @@ object SparkBuild extends Build { def mllibSettings = sharedSettings ++ Seq( name := "spark-mllib", libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3" + "org.jblas" % "jblas" % "1.2.3", + "org.scalanlp" %% "breeze" % "0.7" ) ) From 21109fbab0ded4f73cf4f8816859c76f125adfea Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Mon, 24 Mar 2014 08:44:12 -0700 Subject: [PATCH 126/397] SPARK-1144 Added license and RAT to check licenses. Author: Prashant Sharma Closes #125 from ScrapCodes/rat-integration and squashes the following commits: 64f7c7d [Prashant Sharma] added license headers. fcf28b1 [Prashant Sharma] Review feedback. c0648db [Prashant Sharma] SPARK-1144 Added license and RAT to check licenses. --- .gitignore | 1 + .rat-excludes | 39 +++++++++ dev/check-license | 81 +++++++++++++++++++ dev/run-tests | 7 +- .../optimizer/FilterPushdownSuite.scala | 17 ++++ .../catalyst/optimizer/OptimizerTest.scala | 17 ++++ .../spark/sql/columnar/ColumnTypeSuite.scala | 17 ++++ 7 files changed, 178 insertions(+), 1 deletion(-) create mode 100644 .rat-excludes create mode 100755 dev/check-license diff --git a/.gitignore b/.gitignore index 3d178992123da..e9d8dc3147aa8 100644 --- a/.gitignore +++ b/.gitignore @@ -45,3 +45,4 @@ dist/ spark-*-bin.tar.gz unit-tests.log /lib/ +rat-results.txt diff --git a/.rat-excludes b/.rat-excludes new file mode 100644 index 0000000000000..5c9915de91163 --- /dev/null +++ b/.rat-excludes @@ -0,0 +1,39 @@ +target +.gitignore +.project +.classpath +.rat-excludes +.*md +derby.log +TAGS +RELEASE +control +docs +fairscheduler.xml.template +log4j.properties +log4j.properties.template +metrics.properties.template +slaves +spark-env.sh +spark-env.sh.template +log4j-defaults.properties +sorttable.js +.*txt +.*data +.*log +cloudpickle.py +join.py +SparkExprTyper.scala +SparkILoop.scala +SparkILoopInit.scala +SparkIMain.scala +SparkImports.scala +SparkJLineCompletion.scala +SparkJLineReader.scala +SparkMemberHandlers.scala +sbt +sbt-launch-lib.bash +plugins.sbt +work +.*\.q +golden diff --git a/dev/check-license b/dev/check-license new file mode 100755 index 0000000000000..fbd2dd465bb18 --- /dev/null +++ b/dev/check-license @@ -0,0 +1,81 @@ +#!/usr/bin/env 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. +# + + +acquire_rat_jar () { + + URL1="http://search.maven.org/remotecontent?filepath=org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" + URL2="http://repo1.maven.org/maven2/org/apache/rat/apache-rat/${RAT_VERSION}/apache-rat-${RAT_VERSION}.jar" + + JAR=$rat_jar + + if [[ ! -f "$rat_jar" ]]; then + # Download rat launch jar if it hasn't been downloaded yet + if [ ! -f ${JAR} ]; then + # Download + printf "Attempting to fetch rat\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 rat manually.\n" + exit -1 + fi + fi + if [ ! -f ${JAR} ]; then + # We failed to download + printf "Our attempt to download rat locally to ${JAR} failed. Please install rat manually.\n" + exit -1 + fi + printf "Launching rat from ${JAR}\n" + fi +} + +# Go to the Spark project root directory +FWDIR="$(cd `dirname $0`/..; pwd)" +cd $FWDIR + +if test -x "$JAVA_HOME/bin/java"; then + declare java_cmd="$JAVA_HOME/bin/java" +else + declare java_cmd=java +fi + +export RAT_VERSION=0.10 +export rat_jar=$FWDIR/lib/apache-rat-${RAT_VERSION}.jar +mkdir -p $FWDIR/lib + +[[ -f "$rat_jar" ]] || acquire_rat_jar || { + echo "Download failed. Obtain the rat jar manually and place it at $rat_jar" + exit 1 +} + +$java_cmd -jar $rat_jar -E $FWDIR/.rat-excludes -d $FWDIR > rat-results.txt + +ERRORS=$(cat rat-results.txt | grep -e "??") + +if test ! -z "$ERRORS"; then + echo "Could not find Apache license headers in the following files:" + echo "$ERRORS" + exit 1 +else + echo -e "RAT checks passed." +fi diff --git a/dev/run-tests b/dev/run-tests index cf0b940c09a81..6e78cad995000 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -34,8 +34,13 @@ else fi JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') -[ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run, because JDK version is < 1.8." +[ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run because JDK version is < 1.8." +echo "=========================================================================" +echo "Running Apache RAT checks" +echo "=========================================================================" + +dev/check-license echo "=========================================================================" echo "Running Scala style checks" 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 cfbef53de16b4..0aa7e4d04d707 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 @@ -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/sql/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 index 8ec1d3d8c03ca..0c6e4d5acee83 100644 --- a/sql/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 @@ -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/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index c7aaaae94ee5e..1997d8638cb61 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.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 columnar From 56db8a2f053625fa0c5ba1b0dc64de7d4fc80400 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Mar 2014 13:38:07 -0700 Subject: [PATCH 127/397] HOT FIX: Exclude test files from RAT --- .rat-excludes | 2 ++ 1 file changed, 2 insertions(+) diff --git a/.rat-excludes b/.rat-excludes index 5c9915de91163..17cf6d0ed1cf3 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -2,6 +2,7 @@ target .gitignore .project .classpath +.mima-excludes .rat-excludes .*md derby.log @@ -37,3 +38,4 @@ plugins.sbt work .*\.q golden +test.out/* From 8043b7bc74ff3640743ffc3f1be386dc42f3f44c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Mar 2014 19:24:22 -0700 Subject: [PATCH 128/397] SPARK-1294 Fix resolution of uppercase field names using a HiveContext. Fixing this bug required the following: - Creation of a new logical node that converts a schema to lowercase. - Generalization of the subquery eliding rule to also elide this new node - Fixing of several places where too tight assumptions were made on the types of `InsertIntoTable` children. - I also removed an API that was left in by accident that exposed catalyst data structures, and fix the logic that pushes down filters into hive tables scans to correctly compare attribute references. Author: Michael Armbrust Closes #202 from marmbrus/upperCaseFieldNames and squashes the following commits: 15e5265 [Michael Armbrust] Support for resolving mixed case fields from a reflected schema using HiveQL. 5aa5035 [Michael Armbrust] Remove API that exposes internal catalyst data structures. 9d99cb6 [Michael Armbrust] Attributes should be compared using exprId, not TreeNode.id. --- .../sql/catalyst/analysis/Analyzer.scala | 19 +++++++++++- .../sql/catalyst/optimizer/Optimizer.scala | 14 --------- .../plans/logical/basicOperators.scala | 30 ++++++++++++++++++- .../optimizer/ConstantFoldingSuite.scala | 7 +++-- .../optimizer/FilterPushdownSuite.scala | 13 ++++---- .../apache/spark/sql/hive/HiveContext.scala | 18 ++++++----- .../spark/sql/hive/HiveMetastoreCatalog.scala | 6 ++-- .../spark/sql/hive/HiveStrategies.scala | 4 +-- .../hive/execution/HiveResolutionSuite.scala | 13 ++++++++ 9 files changed, 87 insertions(+), 37 deletions(-) diff --git a/sql/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 index fc76e76617670..161d28eba070e 100644 --- a/sql/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 @@ -55,7 +55,9 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool StarExpansion :: ResolveFunctions :: GlobalAggregates :: - typeCoercionRules :_*) + typeCoercionRules :_*), + Batch("AnalysisOperators", fixedPoint, + EliminateAnalysisOperators) ) /** @@ -80,6 +82,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool case s: Star => s.copy(table = s.table.map(_.toLowerCase)) case UnresolvedAttribute(name) => UnresolvedAttribute(name.toLowerCase) case Alias(c, name) => Alias(c, name.toLowerCase)() + case GetField(c, name) => GetField(c, name.toLowerCase) } } } @@ -184,3 +187,17 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool exprs.collect { case _: Star => true }.nonEmpty } } + +/** + * 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. Similarly, this node also removes + * [[catalyst.plans.logical.LowerCaseSchema LowerCaseSchema]] operators. + */ +object EliminateAnalysisOperators extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case Subquery(_, child) => child + case LowerCaseSchema(child) => child + } +} + 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 c1201971d9c0a..07ebbc90fce71 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 @@ -27,30 +27,16 @@ import org.apache.spark.sql.catalyst.types._ object Optimizer extends RuleExecutor[LogicalPlan] { val batches = - Batch("Subqueries", Once, - EliminateSubqueries) :: Batch("ConstantFolding", Once, ConstantFolding, BooleanSimplification, SimplifyCasts) :: Batch("Filter Pushdown", Once, - EliminateSubqueries, CombineFilters, PushPredicateThroughProject, PushPredicateThroughInnerJoin) :: Nil } -/** - * 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 { - case Subquery(_, child) => child - } -} - /** * Replaces [[catalyst.expressions.Expression Expressions]] that can be statically evaluated with * equivalent [[catalyst.expressions.Literal Literal]] values. 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 6480cca30049e..61481de65e76e 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 @@ -21,6 +21,7 @@ package plans package logical import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { def output = projectList.map(_.toAttribute) @@ -86,7 +87,7 @@ case class Join( } case class InsertIntoTable( - table: BaseRelation, + table: LogicalPlan, partition: Map[String, Option[String]], child: LogicalPlan, overwrite: Boolean) @@ -141,6 +142,33 @@ case class Subquery(alias: String, child: LogicalPlan) extends UnaryNode { def references = Set.empty } +/** + * Converts the schema of `child` to all lowercase, together with LowercaseAttributeReferences + * this allows for optional case insensitive attribute resolution. This node can be elided after + * analysis. + */ +case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { + protected def lowerCaseSchema(dataType: DataType): DataType = dataType match { + case StructType(fields) => + StructType(fields.map(f => + StructField(f.name.toLowerCase(), lowerCaseSchema(f.dataType), f.nullable))) + case ArrayType(elemType) => ArrayType(lowerCaseSchema(elemType)) + case otherType => otherType + } + + val output = child.output.map { + case a: AttributeReference => + AttributeReference( + a.name.toLowerCase, + lowerCaseSchema(a.dataType), + a.nullable)( + a.exprId, + a.qualifiers) + } + + def references = Set.empty +} + case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: LogicalPlan) extends UnaryNode { 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 2c107b865af19..53f760fb4ceb2 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 @@ -19,21 +19,22 @@ package org.apache.spark.sql package catalyst package optimizer -import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor import org.apache.spark.sql.catalyst.types.IntegerType // For implicit conversions +import org.apache.spark.sql.catalyst.dsl.plans._ import org.apache.spark.sql.catalyst.dsl.expressions._ class ConstantFoldingSuite extends OptimizerTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = - Batch("Subqueries", Once, - EliminateSubqueries) :: + Batch("AnalysisNodes", Once, + EliminateAnalysisOperators) :: Batch("ConstantFolding", Once, ConstantFolding, BooleanSimplification) :: Nil 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 0aa7e4d04d707..ae1b2b13dd8f1 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 @@ -19,6 +19,8 @@ package org.apache.spark.sql package catalyst package optimizer + +import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ @@ -31,9 +33,8 @@ class FilterPushdownSuite extends OptimizerTest { object Optimize extends RuleExecutor[LogicalPlan] { val batches = Batch("Subqueries", Once, - EliminateSubqueries) :: + EliminateAnalysisOperators) :: Batch("Filter Pushdown", Once, - EliminateSubqueries, CombineFilters, PushPredicateThroughProject, PushPredicateThroughInnerJoin) :: Nil @@ -172,7 +173,7 @@ class FilterPushdownSuite extends OptimizerTest { } val optimized = Optimize(originalQuery.analyze) - comparePlans(optimizer.EliminateSubqueries(originalQuery.analyze), optimized) + comparePlans(analysis.EliminateAnalysisOperators(originalQuery.analyze), optimized) } test("joins: conjunctive predicates") { @@ -191,7 +192,7 @@ class FilterPushdownSuite extends OptimizerTest { left.join(right, condition = Some("x.b".attr === "y.b".attr)) .analyze - comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) + comparePlans(optimized, analysis.EliminateAnalysisOperators(correctAnswer)) } test("joins: conjunctive predicates #2") { @@ -210,7 +211,7 @@ class FilterPushdownSuite extends OptimizerTest { left.join(right, condition = Some("x.b".attr === "y.b".attr)) .analyze - comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) + comparePlans(optimized, analysis.EliminateAnalysisOperators(correctAnswer)) } test("joins: conjunctive predicates #3") { @@ -233,6 +234,6 @@ class FilterPushdownSuite extends OptimizerTest { condition = Some("z.a".attr === "x.b".attr)) .analyze - comparePlans(optimized, optimizer.EliminateSubqueries(correctAnswer)) + comparePlans(optimized, analysis.EliminateAnalysisOperators(correctAnswer)) } } 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 491b3a62712c6..af35c919df308 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 @@ -32,7 +32,7 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan, LowerCaseSchema} import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand, ExplainCommand} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution._ @@ -108,18 +108,20 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog + override lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog { + override def lookupRelation( + databaseName: Option[String], + tableName: String, + alias: Option[String] = None): LogicalPlan = { + + LowerCaseSchema(super.lookupRelation(databaseName, tableName, alias)) + } + } /* An analyzer that uses the Hive metastore. */ @transient override lazy 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. */ 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 a5db283765c27..1667a217297b1 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 @@ -27,7 +27,8 @@ 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.spark.sql.catalyst.analysis.Catalog + +import org.apache.spark.sql.catalyst.analysis.{Catalog, EliminateAnalysisOperators} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ @@ -96,7 +97,8 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { createTable(databaseName, tableName, child.output) InsertIntoTable( - lookupRelation(Some(databaseName), tableName, None).asInstanceOf[BaseRelation], + EliminateAnalysisOperators( + lookupRelation(Some(databaseName), tableName, None)), Map.empty, child, overwrite = false) 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 c71141c4194f6..3dd0530225be3 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 @@ -73,11 +73,11 @@ trait HiveStrategies { case p @ FilteredOperation(predicates, relation: MetastoreRelation) if relation.isPartitioned => - val partitionKeyIds = relation.partitionKeys.map(_.id).toSet + val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet // Filter out all predicates that only deal with partition keys val (pruningPredicates, otherPredicates) = predicates.partition { - _.references.map(_.id).subsetOf(partitionKeyIds) + _.references.map(_.exprId).subsetOf(partitionKeyIds) } val scan = HiveTableScan( diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 996bd4efecd4c..4bdea214677ad 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -19,6 +19,11 @@ package org.apache.spark.sql package hive package execution +import TestHive._ + +case class Data(a: Int, B: Int, n: Nested) +case class Nested(a: Int, B: Int) + /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. */ @@ -47,6 +52,14 @@ class HiveResolutionSuite extends HiveComparisonTest { createQueryTest("alias.*", "SELECT a.* FROM src a ORDER BY key LIMIT 1") + test("case insensitivity with scala reflection") { + // Test resolution with Scala Reflection + TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2)) :: Nil) + .registerAsTable("caseSensitivityTest") + + sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + } + /** * Negative examples. Currently only left here for documentation purposes. * TODO(marmbrus): Test that catalyst fails on these queries. From dc126f2121d0cd1dc0caa50ae0c4cb9137d42562 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 24 Mar 2014 21:20:23 -0700 Subject: [PATCH 129/397] SPARK-1094 Support MiMa for reporting binary compatibility accross versions. This adds some changes on top of the initial work by @scrapcodes in #20: The goal here is to do automated checking of Spark commits to determine whether they break binary compatibility. 1. Special case for inner classes of package-private objects. 2. Made tools classes accessible when running `spark-class`. 3. Made some declared types in MLLib more general. 4. Various other improvements to exclude-generation script. 5. In-code documentation. Author: Patrick Wendell Author: Prashant Sharma Author: Prashant Sharma Closes #207 from pwendell/mima and squashes the following commits: 22ae267 [Patrick Wendell] New binary changes after upmerge 6c2030d [Patrick Wendell] Merge remote-tracking branch 'apache/master' into mima 3666cf1 [Patrick Wendell] Minor style change 0e0f570 [Patrick Wendell] Small fix and removing directory listings 647c547 [Patrick Wendell] Reveiw feedback. c39f3b5 [Patrick Wendell] Some enhancements to binary checking. 4c771e0 [Prashant Sharma] Added a tool to generate mima excludes and also adapted build to pick automatically. b551519 [Prashant Sharma] adding a new exclude after rebasing with master 651844c [Prashant Sharma] Support MiMa for reporting binary compatibility accross versions. --- .gitignore | 1 + bin/compute-classpath.sh | 1 + bin/spark-class | 3 +- dev/run-tests | 7 + project/MimaBuild.scala | 83 +++++++++++ project/SparkBuild.scala | 27 +++- project/plugins.sbt | 2 + .../spark/tools/GenerateMIMAIgnore.scala | 132 ++++++++++++++++++ 8 files changed, 249 insertions(+), 7 deletions(-) create mode 100644 project/MimaBuild.scala create mode 100644 tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala diff --git a/.gitignore b/.gitignore index e9d8dc3147aa8..3a68abd955b22 100644 --- a/.gitignore +++ b/.gitignore @@ -7,6 +7,7 @@ sbt/*.jar .settings .cache +.mima-excludes /build/ work/ out/ diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 0624117f4964f..5f543914183e0 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -58,6 +58,7 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then 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/tools/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" diff --git a/bin/spark-class b/bin/spark-class index 229ae2cebbab3..a3efa2ff9820c 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -137,8 +137,7 @@ fi # Compute classpath using external script CLASSPATH=`$FWDIR/bin/compute-classpath.sh` - -if [ "$1" == "org.apache.spark.tools.JavaAPICompletenessChecker" ]; then +if [[ "$1" =~ org.apache.spark.tools.* ]]; then CLASSPATH="$CLASSPATH:$SPARK_TOOLS_JAR" fi diff --git a/dev/run-tests b/dev/run-tests index 6e78cad995000..6f115d2abd5b0 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -59,3 +59,10 @@ if [ -z "$PYSPARK_PYTHON" ]; then export PYSPARK_PYTHON=/usr/local/bin/python2.7 fi ./python/run-tests + +echo "=========================================================================" +echo "Detecting binary incompatibilites with MiMa" +echo "=========================================================================" +./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore +sbt/sbt mima-report-binary-issues + diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala new file mode 100644 index 0000000000000..e7c9c47c960fa --- /dev/null +++ b/project/MimaBuild.scala @@ -0,0 +1,83 @@ +/* + * 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 com.typesafe.tools.mima.plugin.MimaKeys.{binaryIssueFilters, previousArtifact} +import com.typesafe.tools.mima.plugin.MimaPlugin.mimaDefaultSettings +import sbt._ + +object MimaBuild { + + def ignoredABIProblems(base: File) = { + import com.typesafe.tools.mima.core._ + import com.typesafe.tools.mima.core.ProblemFilters._ + + // Excludes placed here will be used for all Spark versions + val defaultExcludes = Seq() + + // Read package-private excludes from file + val excludeFilePath = (base.getAbsolutePath + "/.mima-excludes") + val excludeFile = file(excludeFilePath) + val packagePrivateList: Seq[String] = + if (!excludeFile.exists()) { + Seq() + } else { + IO.read(excludeFile).split("\n") + } + + def excludeClass(className: String) = { + Seq( + excludePackage(className), + ProblemFilters.exclude[MissingClassProblem](className), + ProblemFilters.exclude[MissingTypesProblem](className), + excludePackage(className + "$"), + ProblemFilters.exclude[MissingClassProblem](className + "$"), + ProblemFilters.exclude[MissingTypesProblem](className + "$") + ) + } + def excludeSparkClass(className: String) = excludeClass("org.apache.spark." + className) + + val packagePrivateExcludes = packagePrivateList.flatMap(excludeClass) + + /* Excludes specific to a given version of Spark. When comparing the given version against + its immediate predecessor, the excludes listed here will be applied. */ + val versionExcludes = + SparkBuild.SPARK_VERSION match { + case v if v.startsWith("1.0") => + Seq( + excludePackage("org.apache.spark.api.java"), + excludePackage("org.apache.spark.streaming.api.java"), + excludePackage("org.apache.spark.mllib") + ) ++ + excludeSparkClass("rdd.ClassTags") ++ + excludeSparkClass("util.XORShiftRandom") ++ + excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ + excludeSparkClass("mllib.optimization.SquaredGradient") ++ + excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ + excludeSparkClass("mllib.regression.LassoWithSGD") ++ + excludeSparkClass("mllib.regression.LinearRegressionWithSGD") + case _ => Seq() + } + + defaultExcludes ++ packagePrivateExcludes ++ versionExcludes + } + + def mimaSettings(sparkHome: File) = mimaDefaultSettings ++ Seq( + previousArtifact := None, + binaryIssueFilters ++= ignoredABIProblems(sparkHome) + ) + +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1969486f79df1..21d2779d85b74 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -22,6 +22,7 @@ import sbtassembly.Plugin._ import AssemblyKeys._ import scala.util.Properties import org.scalastyle.sbt.ScalastylePlugin.{Settings => ScalaStyleSettings} +import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact import scala.collection.JavaConversions._ @@ -29,6 +30,8 @@ import scala.collection.JavaConversions._ //import com.jsuereth.pgp.sbtplugin.PgpKeys._ object SparkBuild extends Build { + val SPARK_VERSION = "1.0.0-SNAPSHOT" + // Hadoop version to build against. For example, "1.0.4" for Apache releases, or // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set // through the environment variables SPARK_HADOOP_VERSION and SPARK_YARN. @@ -146,9 +149,9 @@ object SparkBuild extends Build { lazy val allProjects = packageProjects ++ allExternalRefs ++ Seq[ProjectReference](examples, tools, assemblyProj, hive) ++ maybeJava8Tests - def sharedSettings = Defaults.defaultSettings ++ Seq( + def sharedSettings = Defaults.defaultSettings ++ MimaBuild.mimaSettings(file(sparkHome)) ++ Seq( organization := "org.apache.spark", - version := "1.0.0-SNAPSHOT", + version := SPARK_VERSION, scalaVersion := "2.10.3", scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-target:" + SCALAC_JVM_VERSION), @@ -284,9 +287,14 @@ object SparkBuild extends Build { val excludeSLF4J = ExclusionRule(organization = "org.slf4j") val excludeScalap = ExclusionRule(organization = "org.scala-lang", artifact = "scalap") + def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", + version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { + val fullId = if (crossVersion.isEmpty) id else id + "_" + crossVersion + Some(organization % fullId % version) // the artifact to compare binary compatibility with + } + def coreSettings = sharedSettings ++ Seq( name := "spark-core", - libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", "com.google.code.findbugs" % "jsr305" % "1.3.9", @@ -325,7 +333,7 @@ object SparkBuild extends Build { publish := {} ) - def replSettings = sharedSettings ++ Seq( + def replSettings = sharedSettings ++ Seq( name := "spark-repl", libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ), libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v ), @@ -354,17 +362,20 @@ object SparkBuild extends Build { def graphxSettings = sharedSettings ++ Seq( name := "spark-graphx", + previousArtifact := sparkPreviousArtifact("spark-graphx"), libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3" ) ) def bagelSettings = sharedSettings ++ Seq( - name := "spark-bagel" + name := "spark-bagel", + previousArtifact := sparkPreviousArtifact("spark-bagel") ) def mllibSettings = sharedSettings ++ Seq( name := "spark-mllib", + previousArtifact := sparkPreviousArtifact("spark-mllib"), libraryDependencies ++= Seq( "org.jblas" % "jblas" % "1.2.3", "org.scalanlp" %% "breeze" % "0.7" @@ -428,6 +439,7 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", + previousArtifact := sparkPreviousArtifact("spark-streaming"), libraryDependencies ++= Seq( "commons-io" % "commons-io" % "2.4" ) @@ -503,6 +515,7 @@ object SparkBuild extends Build { def twitterSettings() = sharedSettings ++ Seq( name := "spark-streaming-twitter", + previousArtifact := sparkPreviousArtifact("spark-streaming-twitter"), libraryDependencies ++= Seq( "org.twitter4j" % "twitter4j-stream" % "3.0.3" excludeAll(excludeNetty) ) @@ -510,6 +523,7 @@ object SparkBuild extends Build { def kafkaSettings() = sharedSettings ++ Seq( name := "spark-streaming-kafka", + previousArtifact := sparkPreviousArtifact("spark-streaming-kafka"), libraryDependencies ++= Seq( "com.github.sgroschupf" % "zkclient" % "0.1" excludeAll(excludeNetty), "org.apache.kafka" %% "kafka" % "0.8.0" @@ -522,6 +536,7 @@ object SparkBuild extends Build { def flumeSettings() = sharedSettings ++ Seq( name := "spark-streaming-flume", + previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), libraryDependencies ++= Seq( "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty) ) @@ -529,6 +544,7 @@ object SparkBuild extends Build { def zeromqSettings() = sharedSettings ++ Seq( name := "spark-streaming-zeromq", + previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), libraryDependencies ++= Seq( "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty) ) @@ -536,6 +552,7 @@ object SparkBuild extends Build { def mqttSettings() = streamingSettings ++ Seq( name := "spark-streaming-mqtt", + previousArtifact := sparkPreviousArtifact("spark-streaming-mqtt"), libraryDependencies ++= Seq("org.eclipse.paho" % "mqtt-client" % "0.4.0") ) } diff --git a/project/plugins.sbt b/project/plugins.sbt index 32bc044a93221..4ff6f67af45c0 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -19,4 +19,6 @@ addSbtPlugin("net.virtual-void" % "sbt-dependency-graph" % "0.7.4") addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.4.0") +addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") + addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.0") diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala new file mode 100644 index 0000000000000..5547e9fe58fc7 --- /dev/null +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -0,0 +1,132 @@ +/* + * 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.tools + +import java.io.File +import java.util.jar.JarFile + +import scala.collection.mutable +import scala.collection.JavaConversions._ +import scala.reflect.runtime.universe.runtimeMirror + +/** + * A tool for generating classes to be excluded during binary checking with MIMA. It is expected + * that this tool is run with ./spark-class. + * + * MIMA itself only supports JVM-level visibility and doesn't account for package-private classes. + * This tool looks at all currently package-private classes and generates exclusions for them. Note + * that this approach is not sound. It can lead to false positives if we move or rename a previously + * package-private class. It can lead to false negatives if someone explicitly makes a class + * package-private that wasn't before. This exists only to help catch certain classes of changes + * which might be difficult to catch during review. + */ +object GenerateMIMAIgnore { + private val classLoader = Thread.currentThread().getContextClassLoader + private val mirror = runtimeMirror(classLoader) + + private def classesPrivateWithin(packageName: String): Set[String] = { + + val classes = getClasses(packageName, classLoader) + val privateClasses = mutable.HashSet[String]() + + def isPackagePrivate(className: String) = { + try { + /* Couldn't figure out if it's possible to determine a-priori whether a given symbol + is a module or class. */ + + val privateAsClass = mirror + .staticClass(className) + .privateWithin + .fullName + .startsWith(packageName) + + val privateAsModule = mirror + .staticModule(className) + .privateWithin + .fullName + .startsWith(packageName) + + privateAsClass || privateAsModule + } catch { + case _: Throwable => { + println("Error determining visibility: " + className) + false + } + } + } + + for (className <- classes) { + val directlyPrivateSpark = isPackagePrivate(className) + + /* Inner classes defined within a private[spark] class or object are effectively + invisible, so we account for them as package private. */ + val indirectlyPrivateSpark = { + val maybeOuter = className.toString.takeWhile(_ != '$') + if (maybeOuter != className) { + isPackagePrivate(maybeOuter) + } else { + false + } + } + if (directlyPrivateSpark || indirectlyPrivateSpark) privateClasses += className + } + privateClasses.flatMap(c => Seq(c, c.replace("$", "#"))).toSet + } + + def main(args: Array[String]) { + scala.tools.nsc.io.File(".mima-excludes"). + writeAll(classesPrivateWithin("org.apache.spark").mkString("\n")) + println("Created : .mima-excludes in current directory.") + } + + + private def shouldExclude(name: String) = { + // Heuristic to remove JVM classes that do not correspond to user-facing classes in Scala + name.contains("anon") || + name.endsWith("$class") || + name.contains("$sp") + } + + /** + * Scans all classes accessible from the context class loader which belong to the given package + * and subpackages both from directories and jars present on the classpath. + */ + private def getClasses(packageName: String, + classLoader: ClassLoader = Thread.currentThread().getContextClassLoader): Set[String] = { + val path = packageName.replace('.', '/') + val resources = classLoader.getResources(path) + + val jars = resources.filter(x => x.getProtocol == "jar") + .map(_.getFile.split(":")(1).split("!")(0)).toSeq + + jars.flatMap(getClassesFromJar(_, path)) + .map(_.getName) + .filterNot(shouldExclude).toSet + } + + /** + * Get all classes in a package from a jar file. + */ + private def getClassesFromJar(jarPath: String, packageName: String) = { + val jar = new JarFile(new File(jarPath)) + val enums = jar.entries().map(_.getName).filter(_.startsWith(packageName)) + val classes = for (entry <- enums if entry.endsWith(".class")) + yield Class.forName(entry.replace('/', '.').stripSuffix(".class")) + classes + } +} From 5140598df889f7227c9d6a7953031eeef524badd Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 24 Mar 2014 21:55:03 -0700 Subject: [PATCH 130/397] SPARK-1128: set hadoop task properties when constructing HadoopRDD https://spark-project.atlassian.net/browse/SPARK-1128 The task properties are not set when constructing HadoopRDD in current implementation, this may limit the implementation based on ``` mapred.tip.id mapred.task.id mapred.task.is.map mapred.task.partition mapred.job.id ``` This patch also contains a small fix in createJobID (SparkHadoopWriter.scala), where the current implementation actually is not using time parameter Author: CodingCat Author: Nan Zhu Closes #101 from CodingCat/SPARK-1128 and squashes the following commits: ed0980f [CodingCat] make SparkHiveHadoopWriter belongs to spark package 5b1ad7d [CodingCat] move SparkHiveHadoopWriter to org.apache.spark package 258f92c [CodingCat] code cleanup af88939 [CodingCat] update the comments and permission of SparkHadoopWriter 9bd1fe3 [CodingCat] move configuration for jobConf to HadoopRDD b7bdfa5 [Nan Zhu] style fix a3153a8 [Nan Zhu] style fix c3258d2 [CodingCat] set hadoop task properties while using InputFormat --- .../org/apache/spark/SparkHadoopWriter.scala | 29 +++++++------------ .../org/apache/spark/rdd/HadoopRDD.scala | 25 +++++++++++++++- .../apache/spark/rdd/PairRDDFunctions.scala | 7 ++--- .../mapred => spark}/SparkHadoopWriter.scala | 8 ++--- .../apache/spark/sql/hive/hiveOperators.scala | 2 +- 5 files changed, 41 insertions(+), 30 deletions(-) rename sql/hive/src/main/scala/org/apache/{hadoop/mapred => spark}/SparkHadoopWriter.scala (97%) diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index d404459a8eb7e..b92ea01a877f7 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -15,28 +15,26 @@ * limitations under the License. */ -package org.apache.hadoop.mapred +package org.apache.spark import java.io.IOException import java.text.NumberFormat import java.text.SimpleDateFormat import java.util.Date +import org.apache.hadoop.mapred._ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.fs.Path -import org.apache.spark.Logging -import org.apache.spark.SerializableWritable +import org.apache.spark.rdd.HadoopRDD /** - * Internal helper class that saves an RDD using a Hadoop OutputFormat. This is only public - * because we need to access this class from the `spark` package to use some package-private Hadoop - * functions, but this class should not be used directly by users. + * Internal helper class that saves an RDD using a Hadoop OutputFormat. * * Saves the RDD using a JobConf, which should contain an output key class, an output value class, * a filename to write to, etc, exactly like in a Hadoop MapReduce job. */ -private[apache] +private[spark] class SparkHadoopWriter(@transient jobConf: JobConf) extends Logging with SparkHadoopMapRedUtil @@ -59,7 +57,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def preSetup() { setIDs(0, 0, 0) - setConfParams() + HadoopRDD.addLocalConfiguration("", 0, 0, 0, conf.value) val jCtxt = getJobContext() getOutputCommitter().setupJob(jCtxt) @@ -68,7 +66,8 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def setup(jobid: Int, splitid: Int, attemptid: Int) { setIDs(jobid, splitid, attemptid) - setConfParams() + HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(now), + jobid, splitID, attemptID, conf.value) } def open() { @@ -167,21 +166,13 @@ class SparkHadoopWriter(@transient jobConf: JobConf) 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) - } } -private[apache] +private[spark] object SparkHadoopWriter { def createJobID(time: Date, id: Int): JobID = { val formatter = new SimpleDateFormat("yyyyMMddHHmm") - val jobtrackerID = formatter.format(new Date()) + val jobtrackerID = formatter.format(time) new JobID(jobtrackerID, id) } diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 100ddb360732a..932ff5bf369c7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -17,6 +17,8 @@ package org.apache.spark.rdd +import java.text.SimpleDateFormat +import java.util.Date import java.io.EOFException import scala.collection.immutable.Map @@ -27,6 +29,9 @@ import org.apache.hadoop.mapred.InputSplit import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.mapred.RecordReader import org.apache.hadoop.mapred.Reporter +import org.apache.hadoop.mapred.JobID +import org.apache.hadoop.mapred.TaskAttemptID +import org.apache.hadoop.mapred.TaskID import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ @@ -111,6 +116,9 @@ class HadoopRDD[K, V]( protected val inputFormatCacheKey = "rdd_%d_input_format".format(id) + // used to build JobTracker ID + private val createTime = new Date() + // Returns a JobConf that will be used on slaves to obtain input splits for Hadoop reads. protected def getJobConf(): JobConf = { val conf: Configuration = broadcastedConf.value.value @@ -165,12 +173,14 @@ class HadoopRDD[K, V]( override def compute(theSplit: Partition, context: TaskContext) = { val iter = new NextIterator[(K, V)] { + val split = theSplit.asInstanceOf[HadoopPartition] logInfo("Input split: " + split.inputSplit) var reader: RecordReader[K, V] = null - val jobConf = getJobConf() val inputFormat = getInputFormat(jobConf) + HadoopRDD.addLocalConfiguration(new SimpleDateFormat("yyyyMMddHHmm").format(createTime), + context.stageId, theSplit.index, context.attemptId.toInt, jobConf) reader = inputFormat.getRecordReader(split.inputSplit.value, jobConf, Reporter.NULL) // Register an on-task-completion callback to close the input stream. @@ -222,4 +232,17 @@ private[spark] object HadoopRDD { def putCachedMetadata(key: String, value: Any) = SparkEnv.get.hadoopJobMetadata.put(key, value) + + /** Add Hadoop configuration specific to a single partition and attempt. */ + def addLocalConfiguration(jobTrackerId: String, jobId: Int, splitId: Int, attemptId: Int, + conf: JobConf) { + val jobID = new JobID(jobTrackerId, jobId) + val taId = new TaskAttemptID(new TaskID(jobID, true, splitId), attemptId) + + conf.set("mapred.tip.id", taId.getTaskID.toString) + conf.set("mapred.task.id", taId.toString) + conf.setBoolean("mapred.task.is.map", true) + conf.setInt("mapred.task.partition", splitId) + conf.set("mapred.job.id", jobID.toString) + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 75fc02acd1bce..14386ff5b9127 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -34,14 +34,13 @@ import org.apache.hadoop.fs.FileSystem import org.apache.hadoop.io.SequenceFile.CompressionType import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{FileOutputCommitter, FileOutputFormat, JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job => NewAPIHadoopJob, RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job => NewAPIHadoopJob, +RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} -// SparkHadoopWriter and SparkHadoopMapReduceUtil are actually source files defined in Spark. -import org.apache.hadoop.mapred.SparkHadoopWriter - import org.apache.spark._ import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner import org.apache.spark.SparkContext._ import org.apache.spark.partial.{BoundedDouble, PartialResult} diff --git a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala similarity index 97% rename from sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala rename to sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 0b3873191985b..d96c2f70e0c74 100644 --- a/sql/hive/src/main/scala/org/apache/hadoop/mapred/SparkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.hadoop.mapred +package org.apache.spark import java.io.IOException import java.text.NumberFormat @@ -25,16 +25,14 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.ql.exec.{FileSinkOperator, Utilities} import org.apache.hadoop.hive.ql.io.{HiveFileFormatUtils, HiveOutputFormat} import org.apache.hadoop.hive.ql.plan.FileSinkDesc +import org.apache.hadoop.mapred._ import org.apache.hadoop.io.Writable -import org.apache.spark.Logging -import org.apache.spark.SerializableWritable - /** * Internal helper class that saves an RDD using a Hive OutputFormat. * It is based on [[SparkHadoopWriter]]. */ -protected[apache] +protected[spark] class SparkHiveHadoopWriter( @transient jobConf: JobConf, fileSinkConf: FileSinkDesc) 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 9aa9e173a8367..78f69e7ff5731 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 @@ -35,7 +35,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ -import org.apache.spark.{TaskContext, SparkException} +import org.apache.spark.{SparkHiveHadoopWriter, TaskContext, SparkException} /* Implicits */ import scala.collection.JavaConversions._ From b637f2d91ab4d3d5bf13e8d959c919ebd776f6af Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 24 Mar 2014 22:15:51 -0700 Subject: [PATCH 131/397] Unify the logic for column pruning, projection, and filtering of table scans. This removes duplicated logic, dead code and casting when planning parquet table scans and hive table scans. Other changes: - Fix tests now that we are doing a better job of column pruning (i.e., since pruning predicates are applied before we even start scanning tuples, columns required by these predicates do not need to be included in the output of the scan unless they are also included in the final output of this logical plan fragment). - Add rule to simplify trivial filters. This was required to avoid `WHERE false` from getting pushed into table scans, since `HiveTableScan` (reasonably) refuses to apply partition pruning predicates to non-partitioned tables. Author: Michael Armbrust Closes #213 from marmbrus/strategyCleanup and squashes the following commits: 48ce403 [Michael Armbrust] Move one more bit of parquet stuff into the core SQLContext. 834ce08 [Michael Armbrust] Address comments. 0f2c6f5 [Michael Armbrust] Unify the logic for column pruning, projection, and filtering of table scans for both Hive and Parquet relations. Fix tests now that we are doing a better job of column pruning. --- .../sql/catalyst/optimizer/Optimizer.scala | 17 +++ .../org/apache/spark/sql/SQLContext.scala | 38 ++++++ .../spark/sql/execution/SparkStrategies.scala | 41 +++---- .../apache/spark/sql/hive/HiveContext.scala | 6 +- .../spark/sql/hive/HiveStrategies.scala | 113 ++---------------- .../sql/hive/execution/PruningSuite.scala | 12 +- 6 files changed, 94 insertions(+), 133 deletions(-) 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 07ebbc90fce71..f28076999ddbf 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 @@ -30,6 +30,7 @@ object Optimizer extends RuleExecutor[LogicalPlan] { Batch("ConstantFolding", Once, ConstantFolding, BooleanSimplification, + SimplifyFilters, SimplifyCasts) :: Batch("Filter Pushdown", Once, CombineFilters, @@ -90,6 +91,22 @@ object CombineFilters extends Rule[LogicalPlan] { } } +/** + * Removes filters that can be evaluated trivially. This is done either by eliding the filter for + * cases where it will always evaluate to `true`, or substituting a dummy empty relation when the + * filter will always evaluate to `false`. + */ +object SimplifyFilters extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case Filter(Literal(true, BooleanType), child) => + child + case Filter(Literal(null, _), child) => + LocalRelation(child.output) + case Filter(Literal(false, BooleanType), child) => + LocalRelation(child.output) + } +} + /** * Pushes [[catalyst.plans.logical.Filter Filter]] operators through * [[catalyst.plans.logical.Project Project]] operators, in-lining any diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 3e98bd3ca627d..cf3c06acce5b0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -118,9 +118,47 @@ class SQLContext(@transient val sparkContext: SparkContext) TopK :: PartialAggregation :: SparkEquiInnerJoin :: + ParquetOperations :: BasicOperators :: CartesianProduct :: BroadcastNestedLoopJoin :: Nil + + /** + * Used to build table scan operators where complex projection and filtering are done using + * separate physical operators. This function returns the given scan operator with Project and + * Filter nodes added only when needed. For example, a Project operator is only used when the + * final desired output requires complex expressions to be evaluated or when columns can be + * further eliminated out after filtering has been done. + * + * The required attributes for both filtering and expression evaluation are passed to the + * provided `scanBuilder` function so that it can avoid unnecessary column materialization. + */ + def pruneFilterProject( + projectList: Seq[NamedExpression], + filterPredicates: Seq[Expression], + scanBuilder: Seq[Attribute] => SparkPlan): SparkPlan = { + + val projectSet = projectList.flatMap(_.references).toSet + val filterSet = filterPredicates.flatMap(_.references).toSet + val filterCondition = filterPredicates.reduceLeftOption(And) + + // Right now we still use a projection even if the only evaluation is applying an alias + // to a column. Since this is a no-op, it could be avoided. However, using this + // optimization with the current implementation would change the output schema. + // TODO: Decouple final output schema from expression evaluation so this copy can be + // avoided safely. + + if (projectList.toSet == projectSet && filterSet.subsetOf(projectSet)) { + // When it is possible to just use column pruning to get the right projection and + // when the columns of this projection are enough to evaluate all filter conditions, + // just do a scan followed by a filter, with no extra project. + val scan = scanBuilder(projectList.asInstanceOf[Seq[Attribute]]) + filterCondition.map(Filter(_, scan)).getOrElse(scan) + } else { + val scan = scanBuilder((projectSet ++ filterSet).toSeq) + Project(projectList, filterCondition.map(Filter(_, scan)).getOrElse(scan)) + } + } } @transient 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 9eb1032113cd9..8a39ded0a9ec4 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 @@ -18,19 +18,15 @@ package org.apache.spark.sql package execution -import org.apache.spark.SparkContext - import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.parquet.ParquetRelation -import org.apache.spark.sql.parquet.InsertIntoParquetTable +import org.apache.spark.sql.parquet._ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { - - val sparkContext: SparkContext + self: SQLContext#SparkPlanner => object SparkEquiInnerJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { @@ -170,6 +166,25 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { } } + object ParquetOperations extends Strategy { + def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { + // TODO: need to support writing to other types of files. Unify the below code paths. + case logical.WriteToFile(path, child) => + val relation = + ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, None) + InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil + case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => + InsertIntoParquetTable(table, planLater(child))(sparkContext) :: Nil + case PhysicalOperation(projectList, filters, relation: parquet.ParquetRelation) => + // TODO: Should be pushing down filters as well. + pruneFilterProject( + projectList, + filters, + ParquetTableScan(_, relation, None)(sparkContext)) :: Nil + case _ => Nil + } + } + // Can we automate these 'pass through' operations? object BasicOperators extends Strategy { // TODO: Set @@ -185,14 +200,6 @@ 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) => @@ -216,12 +223,6 @@ 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) => - 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/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 af35c919df308..3bcf586662f2c 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 @@ -133,8 +133,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { results } - // TODO: Move this. - SessionState.start(sessionState) /** @@ -191,8 +189,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override val strategies: Seq[Strategy] = Seq( TopK, - ColumnPrunings, - PartitionPrunings, + ParquetOperations, HiveTableScans, DataSinks, Scripts, @@ -217,7 +214,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { 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) => 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 3dd0530225be3..141067247d736 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,9 +21,8 @@ package hive import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ -import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ -import org.apache.spark.sql.parquet.{InsertIntoParquetTable, ParquetRelation, ParquetTableScan} trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. @@ -43,121 +42,31 @@ 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.sparkContext) :: 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)(hiveContext) :: Nil - case m: MetastoreRelation => - HiveTableScan(m.output, m, None)(hiveContext) :: Nil case _ => Nil } } /** - * 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. + * Retrieves data using a HiveTableScan. Partition pruning predicates are also detected and + * applied. */ - object PartitionPrunings extends Strategy { + object HiveTableScans extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case p @ FilteredOperation(predicates, relation: MetastoreRelation) - if relation.isPartitioned => - + case PhysicalOperation(projectList, predicates, relation: MetastoreRelation) => + // Filter out all predicates that only deal with partition keys, these are given to the + // hive table scan operator to be used for partition pruning. val partitionKeyIds = relation.partitionKeys.map(_.exprId).toSet - - // Filter out all predicates that only deal with partition keys val (pruningPredicates, otherPredicates) = predicates.partition { _.references.map(_.exprId).subsetOf(partitionKeyIds) - } - - val scan = HiveTableScan( - relation.output, relation, pruningPredicates.reduceLeftOption(And))(hiveContext) - otherPredicates - .reduceLeftOption(And) - .map(Filter(_, scan)) - .getOrElse(scan) :: Nil - - case _ => - Nil - } - } - - /** - * 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[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) - 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.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 { - case scan: HiveTableScan => - scan.copy(attributes = prunedCols)(hiveContext) - }) - } else { - val scan = relation match { - case MetastoreRelation(_, _, _) => { - HiveTableScan( - prunedCols, - relation.asInstanceOf[MetastoreRelation], - None)(hiveContext) - } - case ParquetRelation(_, _) => { - ParquetTableScan( - relation.output, - relation.asInstanceOf[ParquetRelation], - None)(hiveContext.sparkContext) - .pruneColumns(prunedCols) - } - } - predicateOpt.map(execution.Filter(_, scan)).getOrElse(scan) :: Nil - } - - if (isSimpleProject(projectList) && prunedCols == projectRefs) { - filteredScans - } else { - filteredScans.view.map(execution.Project(projectList, _)) } + pruneFilterProject( + projectList, + otherPredicates, + HiveTableScan(_, relation, pruningPredicates.reduceLeftOption(And))(hiveContext)) :: 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]) - } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index bb65c91e2a651..d2f8e5df5b29e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -33,7 +33,7 @@ class PruningSuite extends HiveComparisonTest { createPruningTest("Column pruning: with partitioned table", "SELECT key FROM srcpart WHERE ds = '2008-04-08' LIMIT 3", Seq("key"), - Seq("key", "ds"), + Seq("key"), Seq( Seq("2008-04-08", "11"), Seq("2008-04-08", "12"))) @@ -97,7 +97,7 @@ class PruningSuite extends HiveComparisonTest { 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"), + Seq("value", "hr"), Seq( Seq("2008-04-08", "11"), Seq("2008-04-08", "12"))) @@ -113,14 +113,14 @@ class PruningSuite extends HiveComparisonTest { 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("value", "hr"), Seq( Seq("2008-04-08", "11"))) 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("value", "hr"), Seq.empty) createPruningTest("Partition pruning: pruning with both column key and partition key", @@ -147,8 +147,8 @@ class PruningSuite extends HiveComparisonTest { (columnNames, partValues) }.head - assert(actualOutputColumns sameElements expectedOutputColumns, "Output columns mismatch") - assert(actualScannedColumns sameElements expectedScannedColumns, "Scanned columns mismatch") + assert(actualOutputColumns === expectedOutputColumns, "Output columns mismatch") + assert(actualScannedColumns === expectedScannedColumns, "Scanned columns mismatch") assert( actualPartValues.length === expectedPartValues.length, From 007a733434aa39cdb137ab9795434ae2af70fe0b Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 24 Mar 2014 22:24:21 -0700 Subject: [PATCH 132/397] SPARK-1286: Make usage of spark-env.sh idempotent Various spark scripts load spark-env.sh. This can cause growth of any variables that may be appended to (SPARK_CLASSPATH, SPARK_REPL_OPTS) and it makes the precedence order for options specified in spark-env.sh less clear. One use-case for the latter is that we want to set options from the command-line of spark-shell, but these options will be overridden by subsequent loading of spark-env.sh. If we were to load the spark-env.sh first and then set our command-line options, we could guarantee correct precedence order. Note that we use SPARK_CONF_DIR if available to support the sbin/ scripts, which always set this variable from sbin/spark-config.sh. Otherwise, we default to the ../conf/ as usual. Author: Aaron Davidson Closes #184 from aarondav/idem and squashes the following commits: e291f91 [Aaron Davidson] Use "private" variables in load-spark-env.sh 8da8360 [Aaron Davidson] Add .sh extension to load-spark-env.sh 93a2471 [Aaron Davidson] SPARK-1286: Make usage of spark-env.sh idempotent --- bin/compute-classpath.sh | 5 +---- bin/load-spark-env.sh | 35 +++++++++++++++++++++++++++++++++++ bin/pyspark | 5 +---- bin/run-example | 5 +---- bin/spark-class | 5 +---- bin/spark-shell | 4 +--- sbin/slaves.sh | 4 +--- sbin/spark-daemon.sh | 4 +--- sbin/start-master.sh | 4 +--- sbin/start-slaves.sh | 4 +--- sbin/stop-slaves.sh | 4 +--- 11 files changed, 45 insertions(+), 34 deletions(-) create mode 100644 bin/load-spark-env.sh diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 5f543914183e0..d6f1ff9084c6f 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -25,10 +25,7 @@ SCALA_VERSION=2.10 # Figure out where Spark is installed FWDIR="$(cd `dirname $0`/..; pwd)" -# Load environment variables from conf/spark-env.sh, if it exists -if [ -e "$FWDIR/conf/spark-env.sh" ] ; then - . $FWDIR/conf/spark-env.sh -fi +. $FWDIR/bin/load-spark-env.sh # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh new file mode 100644 index 0000000000000..476dd826551fd --- /dev/null +++ b/bin/load-spark-env.sh @@ -0,0 +1,35 @@ +#!/usr/bin/env 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 loads spark-env.sh if it exists, and ensures it is only loaded once. +# spark-env.sh is loaded from SPARK_CONF_DIR if set, or within the current directory's +# conf/ subdirectory. + +if [ -z "$SPARK_ENV_LOADED" ]; then + export SPARK_ENV_LOADED=1 + + # Returns the parent of the directory this script lives in. + parent_dir="$(cd `dirname $0`/..; pwd)" + + use_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} + + if [ -f "${use_conf_dir}/spark-env.sh" ]; then + . "${use_conf_dir}/spark-env.sh" + fi +fi diff --git a/bin/pyspark b/bin/pyspark index ed6f8da73035a..67e1f61eeb1e5 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -36,10 +36,7 @@ if [ ! -f "$FWDIR/RELEASE" ]; then fi fi -# Load environment variables from conf/spark-env.sh, if it exists -if [ -e "$FWDIR/conf/spark-env.sh" ] ; then - . $FWDIR/conf/spark-env.sh -fi +. $FWDIR/bin/load-spark-env.sh # Figure out which Python executable to use if [ -z "$PYSPARK_PYTHON" ] ; then diff --git a/bin/run-example b/bin/run-example index adba7dd97aaf8..5af95a08c6c41 100755 --- a/bin/run-example +++ b/bin/run-example @@ -30,10 +30,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" -# Load environment variables from conf/spark-env.sh, if it exists -if [ -e "$FWDIR/conf/spark-env.sh" ] ; then - . $FWDIR/conf/spark-env.sh -fi +. $FWDIR/bin/load-spark-env.sh if [ -z "$1" ]; then echo "Usage: run-example []" >&2 diff --git a/bin/spark-class b/bin/spark-class index a3efa2ff9820c..0dcf0e156cb52 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -30,10 +30,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" # Export this as SPARK_HOME export SPARK_HOME="$FWDIR" -# Load environment variables from conf/spark-env.sh, if it exists -if [ -e "$FWDIR/conf/spark-env.sh" ] ; then - . $FWDIR/conf/spark-env.sh -fi +. $FWDIR/bin/load-spark-env.sh if [ -z "$1" ]; then echo "Usage: spark-class []" >&2 diff --git a/bin/spark-shell b/bin/spark-shell index 7d3fe3aca7f1d..861ab606540cd 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -81,9 +81,7 @@ done # Set MASTER from spark-env if possible DEFAULT_SPARK_MASTER_PORT=7077 if [ -z "$MASTER" ]; then - if [ -e "$FWDIR/conf/spark-env.sh" ]; then - . "$FWDIR/conf/spark-env.sh" - fi + . $FWDIR/bin/load-spark-env.sh if [ "x" != "x$SPARK_MASTER_IP" ]; then if [ "y" != "y$SPARK_MASTER_PORT" ]; then SPARK_MASTER_PORT="${SPARK_MASTER_PORT}" diff --git a/sbin/slaves.sh b/sbin/slaves.sh index a5bc2183d87f3..f89547fef9e46 100755 --- a/sbin/slaves.sh +++ b/sbin/slaves.sh @@ -63,9 +63,7 @@ then shift fi -if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then - . "${SPARK_CONF_DIR}/spark-env.sh" -fi +. "$SPARK_PREFIX/bin/load-spark-env.sh" if [ "$HOSTLIST" = "" ]; then if [ "$SPARK_SLAVES" = "" ]; then diff --git a/sbin/spark-daemon.sh b/sbin/spark-daemon.sh index 2be2b3d7c0933..323f675b17848 100755 --- a/sbin/spark-daemon.sh +++ b/sbin/spark-daemon.sh @@ -86,9 +86,7 @@ spark_rotate_log () fi } -if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then - . "${SPARK_CONF_DIR}/spark-env.sh" -fi +. "$SPARK_PREFIX/bin/load-spark-env.sh" if [ "$SPARK_IDENT_STRING" = "" ]; then export SPARK_IDENT_STRING="$USER" diff --git a/sbin/start-master.sh b/sbin/start-master.sh index 03a3428aea9f1..c5c02491f78e1 100755 --- a/sbin/start-master.sh +++ b/sbin/start-master.sh @@ -39,9 +39,7 @@ done . "$sbin/spark-config.sh" -if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then - . "${SPARK_CONF_DIR}/spark-env.sh" -fi +. "$SPARK_PREFIX/bin/load-spark-env.sh" if [ "$SPARK_MASTER_PORT" = "" ]; then SPARK_MASTER_PORT=7077 diff --git a/sbin/start-slaves.sh b/sbin/start-slaves.sh index da641cfe3c6fa..4912d0c0c7dfd 100755 --- a/sbin/start-slaves.sh +++ b/sbin/start-slaves.sh @@ -38,9 +38,7 @@ done . "$sbin/spark-config.sh" -if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then - . "${SPARK_CONF_DIR}/spark-env.sh" -fi +. "$SPARK_PREFIX/bin/load-spark-env.sh" # Find the port number for the master if [ "$SPARK_MASTER_PORT" = "" ]; then diff --git a/sbin/stop-slaves.sh b/sbin/stop-slaves.sh index 6bf393ccd4b09..7c2201100ef97 100755 --- a/sbin/stop-slaves.sh +++ b/sbin/stop-slaves.sh @@ -22,9 +22,7 @@ sbin=`cd "$sbin"; pwd` . "$sbin/spark-config.sh" -if [ -f "${SPARK_CONF_DIR}/spark-env.sh" ]; then - . "${SPARK_CONF_DIR}/spark-env.sh" -fi +. "$SPARK_PREFIX/bin/load-spark-env.sh" # do before the below calls as they exec if [ -e "$sbin"/../tachyon/bin/tachyon ]; then From 134ace7fea7f772f5bafa9d11b8677cb7d311266 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 25 Mar 2014 09:57:26 -0700 Subject: [PATCH 133/397] Add more hive compatability tests to whitelist Author: Michael Armbrust Closes #220 from marmbrus/moreTests and squashes the following commits: 223ec35 [Michael Armbrust] Blacklist machine specific test 9c966cc [Michael Armbrust] add more hive compatability tests to whitelist --- .../execution/HiveCompatibilitySuite.scala | 21 +++++++++++++++++++ 1 file changed, 21 insertions(+) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index 16bcded8a4c65..d5b54fa90fed3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -65,6 +65,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "describe_comment_nonascii", "udf5", "udf_java_method", + "create_merge_compressed", // Weird DDL differences result in failures on jenkins. "create_like2", @@ -163,6 +164,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "add_partition_with_whitelist", "alias_casted_column", "alter2", + "alter3", "alter4", "alter5", "alter_index", @@ -170,7 +172,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "alter_partition_format_loc", "alter_partition_protect_mode", "alter_partition_with_whitelist", + "alter_rename_partition", "alter_table_serde", + "alter_varchar1", "alter_varchar2", "alter_view_as_select", "ambiguous_col", @@ -209,10 +213,13 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "auto_join_filters", "auto_join_nulls", "auto_join_reordering_values", + "auto_smb_mapjoin_14", "auto_sortmerge_join_1", "auto_sortmerge_join_10", "auto_sortmerge_join_11", "auto_sortmerge_join_12", + "auto_sortmerge_join_13", + "auto_sortmerge_join_14", "auto_sortmerge_join_15", "auto_sortmerge_join_2", "auto_sortmerge_join_3", @@ -224,6 +231,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "auto_sortmerge_join_9", "binary_constant", "binarysortable_1", + "cast1", + "cluster", "combine1", "compute_stats_binary", "compute_stats_boolean", @@ -241,6 +250,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "correlationoptimizer7", "correlationoptimizer8", "count", + "cp_mj_rc", + "create_insert_outputformat", "create_like_view", "create_nested_type", "create_skewed_table1", @@ -248,6 +259,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "ct_case_insensitive", "database_location", "database_properties", + "decimal_4", "decimal_join", "default_partition_name", "delimiter", @@ -298,16 +310,25 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "groupby7_map_multi_single_reducer", "groupby7_map_skew", "groupby7_noskew", + "groupby8", "groupby8_map", "groupby8_map_skew", "groupby8_noskew", + "groupby9", "groupby_distinct_samekey", + "groupby_multi_insert_common_distinct", "groupby_multi_single_reducer2", "groupby_mutli_insert_common_distinct", "groupby_neg_float", "groupby_sort_10", + "groupby_sort_2", + "groupby_sort_3", + "groupby_sort_4", + "groupby_sort_5", "groupby_sort_6", + "groupby_sort_7", "groupby_sort_8", + "groupby_sort_9", "groupby_sort_test_1", "implicit_cast1", "innerjoin", From 71d4ed271bcbddb154643bd44297ed77190e75cf Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Tue, 25 Mar 2014 10:21:25 -0700 Subject: [PATCH 134/397] SPARK-1316. Remove use of Commons IO (This follows from a side point on SPARK-1133, in discussion of the PR: https://github.com/apache/spark/pull/164 ) Commons IO is barely used in the project, and can easily be replaced with equivalent calls to Guava or the existing Spark `Utils.scala` class. Removing a dependency feels good, and this one in particular can get a little problematic since Hadoop uses it too. Author: Sean Owen Closes #226 from srowen/SPARK-1316 and squashes the following commits: 21efef3 [Sean Owen] Remove use of Commons IO --- core/pom.xml | 5 ---- .../scala/org/apache/spark/util/Utils.scala | 5 +++- .../org/apache/spark/util/UtilsSuite.scala | 3 +- pom.xml | 5 ---- project/SparkBuild.scala | 8 ++---- streaming/pom.xml | 4 --- .../streaming/util/MasterFailureTest.scala | 6 ++-- .../spark/streaming/CheckpointSuite.scala | 13 ++++----- .../apache/spark/streaming/FailureSuite.scala | 11 ++------ .../spark/streaming/InputStreamsSuite.scala | 28 ++++++++++--------- 10 files changed, 34 insertions(+), 54 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index a6f478b09bda0..eb6cc4d3105e9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -200,11 +200,6 @@ derby test
    - - commons-io - commons-io - test - org.scalatest scalatest_${scala.binary.version} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 13d9dbdd9af2d..ad87fda140476 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -529,7 +529,10 @@ private[spark] object Utils extends Logging { } } if (!file.delete()) { - throw new IOException("Failed to delete: " + file) + // Delete can also fail if the file simply did not exist + if (file.exists()) { + throw new IOException("Failed to delete: " + file.getAbsolutePath) + } } } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 8f55b2372c9f1..eb8f5915605de 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -24,7 +24,6 @@ import java.nio.{ByteBuffer, ByteOrder} import com.google.common.base.Charsets import com.google.common.io.Files -import org.apache.commons.io.FileUtils import org.scalatest.FunSuite class UtilsSuite extends FunSuite { @@ -136,7 +135,7 @@ class UtilsSuite extends FunSuite { // Read some nonexistent bytes on both ends assert(Utils.offsetBytes(f1Path, -3, 25) === "1\n2\n3\n4\n5\n6\n7\n8\n9\n") - FileUtils.deleteDirectory(tmpDir2) + Utils.deleteRecursively(tmpDir2) } test("deserialize long value") { diff --git a/pom.xml b/pom.xml index fa72d5f263e97..deb89b18ada73 100644 --- a/pom.xml +++ b/pom.xml @@ -435,11 +435,6 @@ 1.9.1 test - - commons-io - commons-io - 2.4 - org.easymock easymock diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 21d2779d85b74..60f14ba37e35c 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -259,8 +259,7 @@ object SparkBuild extends Build { "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", "com.novocode" % "junit-interface" % "0.10" % "test", "org.easymock" % "easymock" % "3.1" % "test", - "org.mockito" % "mockito-all" % "1.8.5" % "test", - "commons-io" % "commons-io" % "2.4" % "test" + "org.mockito" % "mockito-all" % "1.8.5" % "test" ), testOptions += Tests.Argument(TestFrameworks.JUnit, "-v", "-a"), @@ -439,10 +438,7 @@ object SparkBuild extends Build { def streamingSettings = sharedSettings ++ Seq( name := "spark-streaming", - previousArtifact := sparkPreviousArtifact("spark-streaming"), - libraryDependencies ++= Seq( - "commons-io" % "commons-io" % "2.4" - ) + previousArtifact := sparkPreviousArtifact("spark-streaming") ) def yarnCommonSettings = sharedSettings ++ Seq( diff --git a/streaming/pom.xml b/streaming/pom.xml index 2cb8bde6642bc..1953cc6883378 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -74,10 +74,6 @@ junit-interface test - - commons-io - commons-io - target/scala-${scala.binary.version}/classes diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala index 2bb616cfb8b08..c48a38590e060 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/MasterFailureTest.scala @@ -28,12 +28,12 @@ import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.reflect.ClassTag import java.io.{File, ObjectInputStream, IOException} +import java.nio.charset.Charset import java.util.UUID import com.google.common.io.Files -import org.apache.commons.io.FileUtils -import org.apache.hadoop.fs.{FileUtil, FileSystem, Path} +import org.apache.hadoop.fs.Path import org.apache.hadoop.conf.Configuration @@ -389,7 +389,7 @@ class FileGeneratingThread(input: Seq[String], testDir: Path, interval: Long) val localFile = new File(localTestDir, (i + 1).toString) val hadoopFile = new Path(testDir, (i + 1).toString) val tempHadoopFile = new Path(testDir, ".tmp_" + (i + 1).toString) - FileUtils.writeStringToFile(localFile, input(i).toString + "\n") + Files.write(input(i) + "\n", localFile, Charset.forName("UTF-8")) var tries = 0 var done = false while (!done && tries < maxTries) { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 831e7c1471a09..0784e562ac719 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -18,10 +18,10 @@ package org.apache.spark.streaming import java.io.File +import java.nio.charset.Charset import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag -import org.apache.commons.io.FileUtils import com.google.common.io.Files import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.conf.Configuration @@ -29,7 +29,6 @@ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.dstream.{DStream, FileInputDStream} import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils -import org.apache.spark.SparkConf /** * This test suites tests the checkpointing functionality of DStreams - @@ -46,13 +45,13 @@ class CheckpointSuite extends TestSuiteBase { override def beforeFunction() { super.beforeFunction() - FileUtils.deleteDirectory(new File(checkpointDir)) + Utils.deleteRecursively(new File(checkpointDir)) } override def afterFunction() { super.afterFunction() if (ssc != null) ssc.stop() - FileUtils.deleteDirectory(new File(checkpointDir)) + Utils.deleteRecursively(new File(checkpointDir)) } test("basic rdd checkpoints + dstream graph checkpoint recovery") { @@ -256,7 +255,7 @@ class CheckpointSuite extends TestSuiteBase { //var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] Thread.sleep(1000) for (i <- Seq(1, 2, 3)) { - FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") + Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) // wait to make sure that the file is written such that it gets shown in the file listings Thread.sleep(1000) } @@ -273,7 +272,7 @@ class CheckpointSuite extends TestSuiteBase { // Create files while the master is down for (i <- Seq(4, 5, 6)) { - FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") + Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) Thread.sleep(1000) } @@ -289,7 +288,7 @@ class CheckpointSuite extends TestSuiteBase { // Restart stream computation ssc.start() for (i <- Seq(7, 8, 9)) { - FileUtils.writeStringToFile(new File(testDir, i.toString), i.toString + "\n") + Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) Thread.sleep(1000) } Thread.sleep(1000) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala index da9b04de1ac44..92e1b76d28301 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/FailureSuite.scala @@ -19,14 +19,9 @@ package org.apache.spark.streaming import org.apache.spark.Logging import org.apache.spark.streaming.util.MasterFailureTest -import StreamingContext._ +import org.apache.spark.util.Utils -import org.scalatest.{FunSuite, BeforeAndAfter} -import com.google.common.io.Files import java.io.File -import org.apache.commons.io.FileUtils -import collection.mutable.ArrayBuffer - /** * This testsuite tests master failures at random times while the stream is running using @@ -43,12 +38,12 @@ class FailureSuite extends TestSuiteBase with Logging { override def beforeFunction() { super.beforeFunction() - FileUtils.deleteDirectory(new File(directory)) + Utils.deleteRecursively(new File(directory)) } override def afterFunction() { super.afterFunction() - FileUtils.deleteDirectory(new File(directory)) + Utils.deleteRecursively(new File(directory)) } test("multiple failures with map") { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 95bf40ba75956..74e73ebb342fe 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -23,21 +23,23 @@ import akka.actor.IOManager import akka.actor.Props import akka.util.ByteString -import org.apache.spark.streaming.dstream.{NetworkReceiver} -import java.net.{InetSocketAddress, SocketException, Socket, ServerSocket} import java.io.{File, BufferedWriter, OutputStreamWriter} +import java.net.{InetSocketAddress, SocketException, ServerSocket} +import java.nio.charset.Charset import java.util.concurrent.{Executors, TimeUnit, ArrayBlockingQueue} -import collection.mutable.{SynchronizedBuffer, ArrayBuffer} -import util.ManualClock +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} + +import com.google.common.io.Files +import org.scalatest.BeforeAndAfter + +import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.NetworkReceiver import org.apache.spark.streaming.receivers.Receiver -import org.apache.spark.Logging -import scala.util.Random -import org.apache.commons.io.FileUtils -import org.scalatest.BeforeAndAfter -import collection.JavaConversions._ -import com.google.common.io.Files -import java.util.concurrent.atomic.AtomicInteger +import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.util.Utils class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { @@ -112,7 +114,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { Thread.sleep(1000) for (i <- 0 until input.size) { val file = new File(testDir, i.toString) - FileUtils.writeStringToFile(file, input(i).toString + "\n") + Files.write(input(i) + "\n", file, Charset.forName("UTF-8")) logInfo("Created file " + file) Thread.sleep(batchDuration.milliseconds) Thread.sleep(1000) @@ -136,7 +138,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // (whether the elements were received one in each interval is not verified) assert(output.toList === expectedOutput.toList) - FileUtils.deleteDirectory(testDir) + Utils.deleteRecursively(testDir) // Enable manual clock back again for other tests conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") From f8111eaeb0e35f6aa9b1e3ec1173fff207174155 Mon Sep 17 00:00:00 2001 From: Shivaram Venkataraman Date: Tue, 25 Mar 2014 13:05:30 -0700 Subject: [PATCH 135/397] SPARK-1319: Fix scheduler to account for tasks using > 1 CPUs. Move CPUS_PER_TASK to TaskSchedulerImpl as the value is a constant and use it in both Mesos and CoarseGrained scheduler backends. Thanks @kayousterhout for the design discussion Author: Shivaram Venkataraman Closes #219 from shivaram/multi-cpus and squashes the following commits: 5c7d685 [Shivaram Venkataraman] Don't pass availableCpus to TaskSetManager 260e4d5 [Shivaram Venkataraman] Add a check for non-zero CPUs in TaskSetManager 73fcf6f [Shivaram Venkataraman] Add documentation for spark.task.cpus 647bc45 [Shivaram Venkataraman] Fix scheduler to account for tasks using > 1 CPUs. Move CPUS_PER_TASK to TaskSchedulerImpl as the value is a constant and use it in both Mesos and CoarseGrained scheduler backends. --- .../spark/scheduler/TaskSchedulerImpl.scala | 25 +++--- .../spark/scheduler/TaskSetManager.scala | 6 +- .../CoarseGrainedSchedulerBackend.scala | 4 +- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../scheduler/TaskSchedulerImplSuite.scala | 42 +++++++++- .../spark/scheduler/TaskSetManagerSuite.scala | 83 +++++++++---------- docs/configuration.md | 7 ++ 7 files changed, 106 insertions(+), 63 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index 30bceb47b9e7d..a92922166f595 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -62,6 +62,9 @@ private[spark] class TaskSchedulerImpl( // Threshold above which we warn user initial TaskSet may be starved val STARVATION_TIMEOUT = conf.getLong("spark.starvation.timeout", 15000) + // CPUs to request per task + val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) + // TaskSetManagers are not thread safe, so any access to one should be synchronized // on this class. val activeTaskSets = new HashMap[String, TaskSetManager] @@ -228,16 +231,18 @@ private[spark] class TaskSchedulerImpl( for (i <- 0 until shuffledOffers.size) { val execId = shuffledOffers(i).executorId val host = shuffledOffers(i).host - for (task <- taskSet.resourceOffer(execId, host, availableCpus(i), maxLocality)) { - tasks(i) += task - val tid = task.taskId - taskIdToTaskSetId(tid) = taskSet.taskSet.id - taskIdToExecutorId(tid) = execId - activeExecutorIds += execId - executorsByHost(host) += execId - availableCpus(i) -= taskSet.CPUS_PER_TASK - assert (availableCpus(i) >= 0) - launchedTask = true + if (availableCpus(i) >= CPUS_PER_TASK) { + for (task <- taskSet.resourceOffer(execId, host, maxLocality)) { + tasks(i) += task + val tid = task.taskId + taskIdToTaskSetId(tid) = taskSet.taskSet.id + taskIdToExecutorId(tid) = execId + activeExecutorIds += execId + executorsByHost(host) += execId + availableCpus(i) -= CPUS_PER_TASK + assert (availableCpus(i) >= 0) + launchedTask = true + } } } } while (launchedTask) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a73343c1c0826..86d2050a03f18 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -56,9 +56,6 @@ private[spark] class TaskSetManager( { val conf = sched.sc.conf - // CPUs to request per task - val CPUS_PER_TASK = conf.getInt("spark.task.cpus", 1) - /* * Sometimes if an executor is dead or in an otherwise invalid state, the driver * does not realize right away leading to repeated task failures. If enabled, @@ -384,11 +381,10 @@ private[spark] class TaskSetManager( def resourceOffer( execId: String, host: String, - availableCpus: Int, maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { - if (!isZombie && availableCpus >= CPUS_PER_TASK) { + if (!isZombie) { val curTime = clock.getTime() var allowedLocality = getAllowedLocalityLevel(curTime) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index fad03731572e7..990e01a3e7959 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -89,7 +89,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A scheduler.statusUpdate(taskId, state, data.value) if (TaskState.isFinished(state)) { if (executorActor.contains(executorId)) { - freeCores(executorId) += 1 + freeCores(executorId) += scheduler.CPUS_PER_TASK makeOffers(executorId) } else { // Ignoring the update since we don't know about the executor. @@ -140,7 +140,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A // Launch tasks returned by a set of resource offers def launchTasks(tasks: Seq[Seq[TaskDescription]]) { for (task <- tasks.flatten) { - freeCores(task.executorId) -= 1 + freeCores(task.executorId) -= scheduler.CPUS_PER_TASK executorActor(task.executorId) ! LaunchTask(task) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index 4092dd04b112b..dfdcafe19fb93 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -246,7 +246,7 @@ private[spark] class MesosSchedulerBackend( val cpuResource = Resource.newBuilder() .setName("cpus") .setType(Value.Type.SCALAR) - .setScalar(Value.Scalar.newBuilder().setValue(1).build()) + .setScalar(Value.Scalar.newBuilder().setValue(scheduler.CPUS_PER_TASK).build()) .build() MesosTaskInfo.newBuilder() .setTaskId(taskId) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 9274e01632d58..356e28dd19bc5 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -80,7 +80,6 @@ class FakeTaskSetManager( override def resourceOffer( execId: String, host: String, - availableCpus: Int, maxLocality: TaskLocality.TaskLocality) : Option[TaskDescription] = { @@ -125,7 +124,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin manager.parent.name, manager.parent.runningTasks, manager.name, manager.runningTasks)) } for (taskSet <- taskSetQueue) { - taskSet.resourceOffer("execId_1", "hostname_1", 1, TaskLocality.ANY) match { + taskSet.resourceOffer("execId_1", "hostname_1", TaskLocality.ANY) match { case Some(task) => return taskSet.stageId case None => {} @@ -293,4 +292,43 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin assert(count > 0) assert(count < numTrials) } + + test("Scheduler correctly accounts for multiple CPUs per task") { + sc = new SparkContext("local", "TaskSchedulerImplSuite") + val taskCpus = 2 + + sc.conf.set("spark.task.cpus", taskCpus.toString) + val taskScheduler = new TaskSchedulerImpl(sc) + taskScheduler.initialize(new FakeSchedulerBackend) + // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. + val dagScheduler = new DAGScheduler(sc, taskScheduler) { + override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} + override def executorAdded(execId: String, host: String) {} + } + + // Give zero core offers. Should not generate any tasks + val zeroCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", 0), + new WorkerOffer("executor1", "host1", 0)) + val taskSet = FakeTask.createTaskSet(1) + taskScheduler.submitTasks(taskSet) + var taskDescriptions = taskScheduler.resourceOffers(zeroCoreWorkerOffers).flatten + assert(0 === taskDescriptions.length) + + // No tasks should run as we only have 1 core free. + val numFreeCores = 1 + val singleCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", numFreeCores), + new WorkerOffer("executor1", "host1", numFreeCores)) + taskScheduler.submitTasks(taskSet) + taskDescriptions = taskScheduler.resourceOffers(singleCoreWorkerOffers).flatten + assert(0 === taskDescriptions.length) + + // Now change the offers to have 2 cores in one executor and verify if it + // is chosen. + val multiCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", taskCpus), + new WorkerOffer("executor1", "host1", numFreeCores)) + taskScheduler.submitTasks(taskSet) + taskDescriptions = taskScheduler.resourceOffers(multiCoreWorkerOffers).flatten + assert(1 === taskDescriptions.length) + assert("executor0" === taskDescriptions(0).executorId) + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala index 9af5d3a303ae2..c92b6dc96c8eb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSetManagerSuite.scala @@ -93,19 +93,16 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val taskSet = FakeTask.createTaskSet(1) val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES) - // Offer a host with no CPUs - assert(manager.resourceOffer("exec1", "host1", 0, ANY) === None) - // Offer a host with process-local as the constraint; this should work because the TaskSet // above won't have any locality preferences - val taskOption = manager.resourceOffer("exec1", "host1", 2, TaskLocality.PROCESS_LOCAL) + val taskOption = manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") assert(sched.startedTasks.contains(0)) // Re-offer the host -- now we should get no more tasks - assert(manager.resourceOffer("exec1", "host1", 2, PROCESS_LOCAL) === None) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None) // Tell it the task has finished manager.handleSuccessfulTask(0, createTaskResult(0)) @@ -121,7 +118,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // First three offers should all find tasks for (i <- 0 until 3) { - val taskOption = manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) + val taskOption = manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) assert(taskOption.isDefined) val task = taskOption.get assert(task.executorId === "exec1") @@ -129,7 +126,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(sched.startedTasks.toSet === Set(0, 1, 2)) // Re-offer the host -- now we should get no more tasks - assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None) // Finish the first two tasks manager.handleSuccessfulTask(0, createTaskResult(0)) @@ -157,35 +154,35 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1, exec1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) // Offer host1, exec1 again: the last task, which has no prefs, should be chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 3) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 3) // Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None) clock.advance(LOCALITY_WAIT) // Offer host1, exec1 again, at PROCESS_LOCAL level: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", 1, PROCESS_LOCAL) === None) + assert(manager.resourceOffer("exec1", "host1", PROCESS_LOCAL) === None) // Offer host1, exec1 again, at NODE_LOCAL level: we should choose task 2 - assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL).get.index == 2) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL).get.index == 2) // Offer host1, exec1 again, at NODE_LOCAL level: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", 1, NODE_LOCAL) === None) + assert(manager.resourceOffer("exec1", "host1", NODE_LOCAL) === None) // Offer host1, exec1 again, at ANY level: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY) === None) clock.advance(LOCALITY_WAIT) // Offer host1, exec1 again, at ANY level: task 1 should get chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) // Offer host1, exec1 again, at ANY level: nothing should be chosen as we've launched all tasks - assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY) === None) } test("delay scheduling with fallback") { @@ -203,29 +200,29 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) // Offer host1 again: nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY) === None) clock.advance(LOCALITY_WAIT) // Offer host1 again: second task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) // Offer host1 again: third task (on host2) should get chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) // Offer host2: fifth task (also on host2) should get chosen - assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 4) + assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 4) // Now that we've launched a local task, we should no longer launch the task for host3 - assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None) + assert(manager.resourceOffer("exec2", "host2", ANY) === None) clock.advance(LOCALITY_WAIT) // After another delay, we can go ahead and launch that task non-locally - assert(manager.resourceOffer("exec2", "host2", 1, ANY).get.index === 3) + assert(manager.resourceOffer("exec2", "host2", ANY).get.index === 3) } test("delay scheduling with failed hosts") { @@ -240,24 +237,24 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) // First offer host1: first task should be chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) // Offer host1 again: third task should be chosen immediately because host3 is not up - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 2) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 2) // After this, nothing should get chosen - assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY) === None) // Now mark host2 as dead sched.removeExecutor("exec2") manager.executorLost("exec2", "host2") // Task 1 should immediately be launched on host1 because its original host is gone - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 1) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 1) // Now that all tasks have launched, nothing new should be launched anywhere else - assert(manager.resourceOffer("exec1", "host1", 1, ANY) === None) - assert(manager.resourceOffer("exec2", "host2", 1, ANY) === None) + assert(manager.resourceOffer("exec1", "host1", ANY) === None) + assert(manager.resourceOffer("exec2", "host2", ANY) === None) } test("task result lost") { @@ -267,14 +264,14 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val clock = new FakeClock val manager = new TaskSetManager(sched, taskSet, MAX_TASK_FAILURES, clock) - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) // Tell it the task has finished but the result was lost. manager.handleFailedTask(0, TaskState.FINISHED, TaskResultLost) assert(sched.endedTasks(0) === TaskResultLost) // Re-offer the host -- now we should get task 0 again. - assert(manager.resourceOffer("exec1", "host1", 1, ANY).get.index === 0) + assert(manager.resourceOffer("exec1", "host1", ANY).get.index === 0) } test("repeated failures lead to task set abortion") { @@ -287,7 +284,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { // Fail the task MAX_TASK_FAILURES times, and check that the task set is aborted // after the last failure. (1 to manager.maxTaskFailures).foreach { index => - val offerResult = manager.resourceOffer("exec1", "host1", 1, ANY) + val offerResult = manager.resourceOffer("exec1", "host1", ANY) assert(offerResult.isDefined, "Expect resource offer on iteration %s to return a task".format(index)) assert(offerResult.get.index === 0) @@ -317,7 +314,7 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { val manager = new TaskSetManager(sched, taskSet, 4, clock) { - val offerResult = manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -328,15 +325,15 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1 fails after failure 1 due to blacklist - assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.NODE_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.RACK_LOCAL).isEmpty) - assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.ANY).isEmpty) + assert(manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", TaskLocality.NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", TaskLocality.RACK_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", TaskLocality.ANY).isEmpty) } // Run the task on exec1.1 - should work, and then fail it on exec1.1 { - val offerResult = manager.resourceOffer("exec1.1", "host1", 1, TaskLocality.NODE_LOCAL) + val offerResult = manager.resourceOffer("exec1.1", "host1", TaskLocality.NODE_LOCAL) assert(offerResult.isDefined, "Expect resource offer to return a task for exec1.1, offerResult = " + offerResult) @@ -348,12 +345,12 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec1.1 fails after failure 2 due to blacklist - assert(manager.resourceOffer("exec1.1", "host1", 1, TaskLocality.NODE_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1.1", "host1", TaskLocality.NODE_LOCAL).isEmpty) } // Run the task on exec2 - should work, and then fail it on exec2 { - val offerResult = manager.resourceOffer("exec2", "host2", 1, TaskLocality.ANY) + val offerResult = manager.resourceOffer("exec2", "host2", TaskLocality.ANY) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) @@ -364,20 +361,20 @@ class TaskSetManagerSuite extends FunSuite with LocalSparkContext with Logging { assert(!sched.taskSetsFailed.contains(taskSet.id)) // Ensure scheduling on exec2 fails after failure 3 due to blacklist - assert(manager.resourceOffer("exec2", "host2", 1, TaskLocality.ANY).isEmpty) + assert(manager.resourceOffer("exec2", "host2", TaskLocality.ANY).isEmpty) } // After reschedule delay, scheduling on exec1 should be possible. clock.advance(rescheduleDelay) { - val offerResult = manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL) + val offerResult = manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL) assert(offerResult.isDefined, "Expect resource offer to return a task") assert(offerResult.get.index === 0) assert(offerResult.get.executorId === "exec1") - assert(manager.resourceOffer("exec1", "host1", 1, TaskLocality.PROCESS_LOCAL).isEmpty) + assert(manager.resourceOffer("exec1", "host1", TaskLocality.PROCESS_LOCAL).isEmpty) // Cause exec1 to fail : failure 4 manager.handleFailedTask(offerResult.get.taskId, TaskState.FINISHED, TaskResultLost) diff --git a/docs/configuration.md b/docs/configuration.md index 16ee5ec0f230f..1ff0150567255 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -579,6 +579,13 @@ Apart from these, the following properties are also available, and may be useful out and giving up.
    + + + + +
    {s.stageId} - {poolName.get} + + {poolName.get} + + {description} {submissionTime} - {duration.map(d => parent.formatDuration(d)).getOrElse("Unknown")} - {formattedDuration} {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)}
    - + {rdd.name}
    {status.blockManagerId.host + ":" + status.blockManagerId.port} + {Utils.bytesToString(status.memUsedByRDD(rddId))} + ({Utils.bytesToString(status.memRemaining)} Remaining) + {Utils.bytesToString(status.diskUsedByRDD(rddId))}
    {id}
    {status.blockManagerId.host + ":" + status.blockManagerId.port} - {Utils.bytesToString(status.memUsedByRDD(rddId))} - ({Utils.bytesToString(status.memRemaining)} Remaining) - {Utils.bytesToString(status.diskUsedByRDD(rddId))}
    spark.logConf false - Log the supplied SparkConf as INFO at start of spark context. + Whether to log the supplied SparkConf as INFO at start of spark context. +
    spark.eventLog.enabledfalse + Whether to log spark events, useful for reconstructing the Web UI after the application has finished. +
    spark.eventLog.compressfalse + Whether to compress logged events, if spark.eventLog.enabled is true. +
    spark.eventLog.dirfile:///tmp/spark-events + Base directory in which spark events are logged, if spark.eventLog.enabled is true. + Within this base directory, Spark creates a sub-directory for each application, and logs the events + specific to the application in this directory.
    {Utils.megabytesToString(app.desc.memoryPerSlave)} {DeployWebUI.formatDate(app.submitDate)}{WebUI.formatDate(app.submitDate)} {app.desc.user} {app.state.toString}{DeployWebUI.formatDuration(app.duration)}{WebUI.formatDuration(app.duration)}
    {info.status} {info.taskLocality} {info.host}{dateFmt.format(new Date(info.launchTime))}{WebUI.formatDate(new Date(info.launchTime))} {formatDuration}
    spark.task.cpus1 + Number of cores to allocate for each task. +
    ## Viewing Spark Properties From 8237df8060039af59eb387f5ea5d6611e8f3e526 Mon Sep 17 00:00:00 2001 From: witgo Date: Tue, 25 Mar 2014 13:28:13 -0700 Subject: [PATCH 136/397] Avoid Option while generating call site This is an update on https://github.com/apache/spark/pull/180, which changes the solution from blacklisting "Option.scala" to avoiding the Option code path while generating the call path. Also includes a unit test to prevent this issue in the future, and some minor refactoring. Thanks @witgo for reporting this issue and working on the initial solution! Author: witgo Author: Aaron Davidson Closes #222 from aarondav/180 and squashes the following commits: f74aad1 [Aaron Davidson] Avoid Option while generating call site & add unit tests d2b4980 [witgo] Modify the position of the filter 1bc22d7 [witgo] Fix Stage.name return "apply at Option.scala:120" --- .../scala/org/apache/spark/SparkContext.scala | 3 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../scala/org/apache/spark/util/Utils.scala | 18 +++++----- .../apache/spark/SparkContextInfoSuite.scala | 36 ++++++++++++++++++- 4 files changed, 47 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a1003b7925715..4dd298177f07d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -877,7 +877,8 @@ class SparkContext( * has overridden the call site, this will return the user's version. */ private[spark] def getCallSite(): String = { - Option(getLocalProperty("externalCallSite")).getOrElse(Utils.formatCallSiteInfo()) + val defaultCallSite = Utils.getCallSiteInfo + Option(getLocalProperty("externalCallSite")).getOrElse(defaultCallSite.toString) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 1b43040c6d918..4f9d39f86546c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1041,7 +1041,7 @@ abstract class RDD[T: ClassTag]( /** User code that created this RDD (e.g. `textFile`, `parallelize`). */ @transient private[spark] val creationSiteInfo = Utils.getCallSiteInfo - private[spark] def getCreationSite = Utils.formatCallSiteInfo(creationSiteInfo) + private[spark] def getCreationSite: String = creationSiteInfo.toString private[spark] def elementClassTag: ClassTag[T] = classTag[T] diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index ad87fda140476..62ee704d580c2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -679,7 +679,13 @@ private[spark] object Utils extends Logging { private val SPARK_CLASS_REGEX = """^org\.apache\.spark(\.api\.java)?(\.util)?(\.rdd)?\.[A-Z]""".r private[spark] class CallSiteInfo(val lastSparkMethod: String, val firstUserFile: String, - val firstUserLine: Int, val firstUserClass: String) + val firstUserLine: Int, val firstUserClass: String) { + + /** Returns a printable version of the call site info suitable for logs. */ + override def toString = { + "%s at %s:%s".format(lastSparkMethod, firstUserFile, firstUserLine) + } + } /** * When called inside a class in the spark package, returns the name of the user code class @@ -687,8 +693,8 @@ private[spark] object Utils extends Logging { * This is used, for example, to tell users where in their code each RDD got created. */ def getCallSiteInfo: CallSiteInfo = { - val trace = Thread.currentThread.getStackTrace().filter( el => - (!el.getMethodName.contains("getStackTrace"))) + val trace = Thread.currentThread.getStackTrace() + .filterNot(_.getMethodName.contains("getStackTrace")) // Keep crawling up the stack trace until we find the first function not inside of the spark // package. We track the last (shallowest) contiguous Spark method. This might be an RDD @@ -721,12 +727,6 @@ private[spark] object Utils extends Logging { new CallSiteInfo(lastSparkMethod, firstUserFile, firstUserLine, firstUserClass) } - /** Returns a printable version of the call site info suitable for logs. */ - def formatCallSiteInfo(callSiteInfo: CallSiteInfo = Utils.getCallSiteInfo) = { - "%s at %s:%s".format(callSiteInfo.lastSparkMethod, callSiteInfo.firstUserFile, - callSiteInfo.firstUserLine) - } - /** Return a string containing part of a file from byte 'start' to 'end'. */ def offsetBytes(path: String, start: Long, end: Long): String = { val file = new File(path) diff --git a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala index 5cb49d9a7fce1..cd3887dcc7371 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextInfoSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark -import org.scalatest.FunSuite +import org.scalatest.{Assertions, FunSuite} class SparkContextInfoSuite extends FunSuite with LocalSparkContext { test("getPersistentRDDs only returns RDDs that are marked as cached") { @@ -56,4 +56,38 @@ class SparkContextInfoSuite extends FunSuite with LocalSparkContext { rdd.collect() assert(sc.getRDDStorageInfo.size === 1) } + + test("call sites report correct locations") { + sc = new SparkContext("local", "test") + testPackage.runCallSiteTest(sc) + } +} + +/** Call site must be outside of usual org.apache.spark packages (see Utils#SPARK_CLASS_REGEX). */ +package object testPackage extends Assertions { + private val CALL_SITE_REGEX = "(.+) at (.+):([0-9]+)".r + + def runCallSiteTest(sc: SparkContext) { + val rdd = sc.makeRDD(Array(1, 2, 3, 4), 2) + val rddCreationSite = rdd.getCreationSite + val curCallSite = sc.getCallSite() // note: 2 lines after definition of "rdd" + + val rddCreationLine = rddCreationSite match { + case CALL_SITE_REGEX(func, file, line) => { + assert(func === "makeRDD") + assert(file === "SparkContextInfoSuite.scala") + line.toInt + } + case _ => fail("Did not match expected call site format") + } + + curCallSite match { + case CALL_SITE_REGEX(func, file, line) => { + assert(func === "getCallSite") // this is correct because we called it from outside of Spark + assert(file === "SparkContextInfoSuite.scala") + assert(line.toInt === rddCreationLine.toInt + 2) + } + case _ => fail("Did not match expected call site format") + } + } } From 4f7d547b85ed89ba4706e05d7d0984f16749120e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 25 Mar 2014 19:01:18 -0700 Subject: [PATCH 137/397] Initial experimentation with Travis CI configuration This is not intended to replace Jenkins immediately, and Jenkins will remain the CI of reference for merging pull requests in the near term. Long term, it is possible that Travis will give us better integration with github, so we are investigating its use. Author: Michael Armbrust Closes #230 from marmbrus/travis and squashes the following commits: 93f9a32 [Michael Armbrust] Add Apache license to .travis.yml d7c0e78 [Michael Armbrust] Initial experimentation with Travis CI configuration --- .travis.yml | 37 +++++++++++++++++++++++++++++++++++++ 1 file changed, 37 insertions(+) create mode 100644 .travis.yml diff --git a/.travis.yml b/.travis.yml new file mode 100644 index 0000000000000..617da2dc308cc --- /dev/null +++ b/.travis.yml @@ -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. + + language: scala + scala: + - "2.10.3" + jdk: + - oraclejdk7 + env: + matrix: + - TEST=sql/test + - TEST=hive/test + - TEST=catalyst/test + - TEST=streaming/test + - TEST=graphx/test + - TEST=mllib/test + - TEST=graphx/test + - TEST=bagel/test + cache: + directories: + - $HOME/.m2 + - $HOME/.ivy2 + - $HOME/.sbt + script: + - "sbt ++$TRAVIS_SCALA_VERSION scalastyle $TEST" From b859853ba47b6323af0e31a4e2099e943221e1b1 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 26 Mar 2014 00:09:44 -0700 Subject: [PATCH 138/397] SPARK-1321 Use Guava's top k implementation rather than our BoundedPriorityQueue based implementation Also updated the documentation for top and takeOrdered. On my simple test of sorting 100 million (Int, Int) tuples using Spark, Guava's top k implementation (in Ordering) is much faster than the BoundedPriorityQueue implementation for roughly sorted input (10 - 20X faster), and still faster for purely random input (2 - 5X). Author: Reynold Xin Closes #229 from rxin/takeOrdered and squashes the following commits: 0d11844 [Reynold Xin] Use Guava's top k implementation rather than our BoundedPriorityQueue based implementation. Also updated the documentation for top and takeOrdered. --- .../main/scala/org/apache/spark/rdd/RDD.scala | 49 +++++++++++++------ .../apache/spark/util/collection/Utils.scala | 39 +++++++++++++++ 2 files changed, 72 insertions(+), 16 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/util/collection/Utils.scala diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 4f9d39f86546c..6af42248a5c3c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -927,32 +927,49 @@ abstract class RDD[T: ClassTag]( } /** - * Returns the top K elements from this RDD as defined by - * the specified implicit Ordering[T]. + * Returns the top K (largest) elements from this RDD as defined by the specified + * implicit Ordering[T]. This does the opposite of [[takeOrdered]]. For example: + * {{{ + * sc.parallelize([10, 4, 2, 12, 3]).top(1) + * // returns [12] + * + * sc.parallelize([2, 3, 4, 5, 6]).top(2) + * // returns [6, 5] + * }}} + * * @param num the number of top elements to return * @param ord the implicit ordering for T * @return an array of top elements */ - def top(num: Int)(implicit ord: Ordering[T]): Array[T] = { - mapPartitions { items => - val queue = new BoundedPriorityQueue[T](num) - queue ++= items - Iterator.single(queue) - }.reduce { (queue1, queue2) => - queue1 ++= queue2 - queue1 - }.toArray.sorted(ord.reverse) - } + def top(num: Int)(implicit ord: Ordering[T]): Array[T] = takeOrdered(num)(ord.reverse) /** - * Returns the first K elements from this RDD as defined by - * the specified implicit Ordering[T] and maintains the - * ordering. + * Returns the first K (smallest) elements from this RDD as defined by the specified + * implicit Ordering[T] and maintains the ordering. This does the opposite of [[top]]. + * For example: + * {{{ + * sc.parallelize([10, 4, 2, 12, 3]).takeOrdered(1) + * // returns [12] + * + * sc.parallelize([2, 3, 4, 5, 6]).takeOrdered(2) + * // returns [2, 3] + * }}} + * * @param num the number of top elements to return * @param ord the implicit ordering for T * @return an array of top elements */ - def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = top(num)(ord.reverse) + def takeOrdered(num: Int)(implicit ord: Ordering[T]): Array[T] = { + mapPartitions { items => + // Priority keeps the largest elements, so let's reverse the ordering. + val queue = new BoundedPriorityQueue[T](num)(ord.reverse) + queue ++= util.collection.Utils.takeOrdered(items, num)(ord) + Iterator.single(queue) + }.reduce { (queue1, queue2) => + queue1 ++= queue2 + queue1 + }.toArray.sorted(ord) + } /** * Returns the max of this RDD as defined by the implicit Ordering[T]. diff --git a/core/src/main/scala/org/apache/spark/util/collection/Utils.scala b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala new file mode 100644 index 0000000000000..c5268c0fae0ef --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/collection/Utils.scala @@ -0,0 +1,39 @@ +/* + * 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.util.collection + +import scala.collection.JavaConversions.{collectionAsScalaIterable, asJavaIterator} + +import com.google.common.collect.{Ordering => GuavaOrdering} + +/** + * Utility functions for collections. + */ +private[spark] object Utils { + + /** + * Returns the first K elements from the input as defined by the specified implicit Ordering[T] + * and maintains the ordering. + */ + def takeOrdered[T](input: Iterator[T], num: Int)(implicit ord: Ordering[T]): Iterator[T] = { + val ordering = new GuavaOrdering[T] { + override def compare(l: T, r: T) = ord.compare(l, r) + } + collectionAsScalaIterable(ordering.leastOf(asJavaIterator(input), num)).iterator + } +} From a0853a39e1907700737ec924367cd2610da8395a Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Wed, 26 Mar 2014 09:16:37 -0700 Subject: [PATCH 139/397] SPARK-1322, top in pyspark should sort result in descending order. Author: Prashant Sharma Closes #235 from ScrapCodes/SPARK-1322/top-rev-sort and squashes the following commits: f316266 [Prashant Sharma] Minor change in comment. 58e58c6 [Prashant Sharma] SPARK-1322, top in pyspark should sort result in descending order. --- python/pyspark/rdd.py | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index ca2dc119a461a..019c249699c2d 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -693,11 +693,11 @@ def top(self, num): """ Get the top N elements from a RDD. - Note: It returns the list sorted in ascending order. + Note: It returns the list sorted in descending order. >>> sc.parallelize([10, 4, 2, 12, 3]).top(1) [12] >>> sc.parallelize([2, 3, 4, 5, 6]).cache().top(2) - [5, 6] + [6, 5] """ def topIterator(iterator): q = [] @@ -711,7 +711,7 @@ def topIterator(iterator): def merge(a, b): return next(topIterator(a + b)) - return sorted(self.mapPartitions(topIterator).reduce(merge)) + return sorted(self.mapPartitions(topIterator).reduce(merge), reverse=True) def take(self, num): """ From 345825d97987b9eeb2afcf002f815a05ff51fc2e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 26 Mar 2014 15:36:18 -0700 Subject: [PATCH 140/397] Unified package definition format in Spark SQL According to discussions in comments of PR #208, this PR unifies package definition format in Spark SQL. Some broken links in ScalaDoc and typos detected along the way are also fixed. Author: Cheng Lian Closes #225 from liancheng/packageDefinition and squashes the following commits: 75c47b3 [Cheng Lian] Fixed file line length 4f87968 [Cheng Lian] Unified package definition format in Spark SQL --- .../scala/org/apache/spark/Dependency.scala | 5 ++- .../apache/spark/api/java/JavaRDDLike.scala | 4 +- .../spark/sql/catalyst/ScalaReflection.scala | 3 +- .../sql/catalyst/analysis/Analyzer.scala | 9 ++--- .../spark/sql/catalyst/analysis/Catalog.scala | 4 +- .../catalyst/analysis/FunctionRegistry.scala | 4 +- .../catalyst/analysis/HiveTypeCoercion.scala | 8 ++-- .../analysis/MultiInstanceRelation.scala | 3 +- .../spark/sql/catalyst/analysis/package.scala | 3 +- .../sql/catalyst/analysis/unresolved.scala | 5 +-- .../spark/sql/catalyst/dsl/package.scala | 3 +- .../spark/sql/catalyst/errors/package.scala | 3 +- .../catalyst/expressions/BoundAttribute.scala | 6 +-- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/Expression.scala | 9 ++--- .../sql/catalyst/expressions/Projection.scala | 3 +- .../spark/sql/catalyst/expressions/Rand.scala | 4 +- .../spark/sql/catalyst/expressions/Row.scala | 4 +- .../sql/catalyst/expressions/ScalaUdf.scala | 4 +- .../sql/catalyst/expressions/SortOrder.scala | 4 +- .../catalyst/expressions/WrapDynamic.scala | 4 +- .../sql/catalyst/expressions/aggregates.scala | 5 +-- .../sql/catalyst/expressions/arithmetic.scala | 4 +- .../catalyst/expressions/complexTypes.scala | 4 +- .../sql/catalyst/expressions/generators.scala | 5 +-- .../sql/catalyst/expressions/literals.scala | 4 +- .../expressions/namedExpressions.scala | 5 +-- .../catalyst/expressions/nullFunctions.scala | 5 +-- .../sql/catalyst/expressions/package.scala | 5 +-- .../sql/catalyst/expressions/predicates.scala | 7 ++-- .../expressions/stringOperations.scala | 4 +- .../sql/catalyst/optimizer/Optimizer.scala | 5 +-- .../sql/catalyst/planning/QueryPlanner.scala | 5 +-- .../spark/sql/catalyst/planning/package.scala | 3 +- .../sql/catalyst/planning/patterns.scala | 8 ++-- .../spark/sql/catalyst/plans/QueryPlan.scala | 4 +- .../spark/sql/catalyst/plans/joinTypes.scala | 4 +- .../catalyst/plans/logical/BaseRelation.scala | 5 +-- .../catalyst/plans/logical/LogicalPlan.scala | 7 ++-- .../plans/logical/ScriptTransformation.scala | 5 +-- .../catalyst/plans/logical/TestRelation.scala | 6 +-- .../plans/logical/basicOperators.scala | 6 +-- .../catalyst/plans/logical/partitioning.scala | 5 +-- .../spark/sql/catalyst/plans/package.scala | 3 +- .../plans/physical/partitioning.scala | 5 +-- .../spark/sql/catalyst/rules/Rule.scala | 5 +-- .../spark/sql/catalyst/rules/package.scala | 3 +- .../spark/sql/catalyst/trees/TreeNode.scala | 4 +- .../spark/sql/catalyst/trees/package.scala | 5 ++- .../spark/sql/catalyst/types/dataTypes.scala | 4 +- .../spark/sql/catalyst/types/package.scala | 3 +- .../spark/sql/catalyst/util/package.scala | 3 +- .../sql/catalyst/DistributionSuite.scala | 5 +-- .../sql/catalyst/analysis/AnalysisSuite.scala | 4 +- .../analysis/HiveTypeCoercionSuite.scala | 4 +- .../ExpressionEvaluationSuite.scala | 4 +- .../optimizer/ConstantFoldingSuite.scala | 6 +-- .../optimizer/FilterPushdownSuite.scala | 8 ++-- .../catalyst/optimizer/OptimizerTest.scala | 4 +- .../catalyst/trees/RuleExecutorSuite.scala | 4 +- .../spark/sql/columnar/ColumnAccessor.scala | 3 +- .../spark/sql/columnar/ColumnBuilder.scala | 8 ++-- .../spark/sql/columnar/ColumnType.scala | 4 +- .../sql/columnar/NullableColumnBuilder.scala | 7 ++-- .../columnar/inMemoryColumnarOperators.scala | 10 ++--- .../apache/spark/sql/execution/Exchange.scala | 6 +-- .../apache/spark/sql/execution/Generate.scala | 5 +-- .../execution/QueryExecutionException.scala | 3 +- .../spark/sql/execution/SparkPlan.scala | 13 +++---- .../sql/execution/SparkSqlSerializer.scala | 3 +- .../spark/sql/execution/SparkStrategies.scala | 6 +-- .../spark/sql/execution/aggregates.scala | 3 +- .../spark/sql/execution/basicOperators.scala | 3 +- .../apache/spark/sql/execution/debug.scala | 3 +- .../apache/spark/sql/execution/joins.scala | 3 +- .../spark/sql/parquet/ParquetRelation.scala | 9 ++--- .../sql/parquet/ParquetTableSupport.scala | 3 +- .../spark/sql/test/TestSQLContext.scala | 7 ++-- .../org/apache/spark/sql/QueryTest.scala | 2 - .../spark/sql/columnar/ColumnTypeSuite.scala | 3 +- .../NullableColumnAccessorSuite.scala | 3 +- .../columnar/NullableColumnBuilderSuite.scala | 3 +- .../apache/spark/sql/execution/TgfSuite.scala | 3 +- .../org/apache/spark/SparkHadoopWriter.scala | 13 +++---- .../apache/spark/sql/hive/HiveContext.scala | 38 +++++++++---------- .../spark/sql/hive/HiveMetastoreCatalog.scala | 8 ++-- .../org/apache/spark/sql/hive/HiveQl.scala | 3 +- .../spark/sql/hive/HiveStrategies.scala | 4 +- .../spark/sql/hive/ScriptTransformation.scala | 3 +- .../apache/spark/sql/hive/TableReader.scala | 3 +- .../org/apache/spark/sql/hive/TestHive.scala | 3 +- .../apache/spark/sql/hive/hiveOperators.scala | 3 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 4 +- .../execution/BigDataBenchmarkSuite.scala | 10 ++--- .../hive/execution/ConcurrentHiveSuite.scala | 9 ++--- .../hive/execution/HiveComparisonTest.scala | 9 ++--- .../execution/HiveCompatibilitySuite.scala | 6 +-- .../hive/execution/HiveQueryFileTest.scala | 4 +- .../sql/hive/execution/HiveQuerySuite.scala | 8 ++-- .../hive/execution/HiveResolutionSuite.scala | 7 ++-- .../sql/hive/execution/HiveSerDeSuite.scala | 4 +- .../execution/HiveTypeCoercionSuite.scala | 3 +- .../sql/hive/execution/PruningSuite.scala | 6 +-- 103 files changed, 209 insertions(+), 333 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 448f87b81ef4a..3132dcf745e19 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -44,8 +44,9 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output - * @param serializer [[Serializer]] to use. If set to null, the default serializer, as specified - * by `spark.serializer` config option, will be used. + * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to null, + * the default serializer, as specified by `spark.serializer` config option, will + * be used. */ class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 05b89b985736d..ddac553304233 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -481,7 +481,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Returns the maximum element from this RDD as defined by the specified * Comparator[T]. - * @params comp the comparator that defines ordering + * @param comp the comparator that defines ordering * @return the maximum of the RDD * */ def max(comp: Comparator[T]): T = { @@ -491,7 +491,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Returns the minimum element from this RDD as defined by the specified * Comparator[T]. - * @params comp the comparator that defines ordering + * @param comp the comparator that defines ordering * @return the minimum of the RDD * */ def min(comp: Comparator[T]): T = { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index bf7318d2e078b..976dda8d7e59a 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.AttributeReference diff --git a/sql/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 index 161d28eba070e..4ebc0e70d946b 100644 --- a/sql/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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ @@ -89,7 +87,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool /** * Replaces [[UnresolvedAttribute]]s with concrete - * [[expressions.AttributeReference AttributeReferences]] from a logical plan node's children. + * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's + * children. */ object ResolveReferences extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transformUp { @@ -106,7 +105,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool } /** - * Replaces [[UnresolvedFunction]]s with concrete [[expressions.Expression Expressions]]. + * Replaces [[UnresolvedFunction]]s with concrete [[catalyst.expressions.Expression Expressions]]. */ object ResolveFunctions extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { 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 b77f0bbb2f21b..ff66177a03b8c 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import scala.collection.mutable 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 index eed058d3c7482..c0255701b7ba5 100644 --- 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Expression diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala index a6ecf6e2ebec5..4557d77160fad 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union} @@ -37,8 +35,8 @@ trait HiveTypeCoercion { StringToIntegralCasts, FunctionArgumentConversion) /** - * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] dataTypes - * that are made by other rules to instances higher in the query tree. + * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] data + * types 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 { diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala index 3cad3a5d4db4e..a6ce90854dcb4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 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 index 30c55bacc7427..9f37ca904ffeb 100644 --- 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis. diff --git a/sql/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 index 04ae481102c74..41e9bcef3cd7f 100644 --- a/sql/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 @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.{errors, trees} import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression} import org.apache.spark.sql.catalyst.plans.logical.BaseRelation import org.apache.spark.sql.catalyst.trees.TreeNode 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 e6255bcafa8aa..67cddb351c185 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import scala.language.implicitConversions diff --git a/sql/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 index d8b69946fab10..bdeb660b1ecb7 100644 --- a/sql/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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import org.apache.spark.sql.catalyst.trees.TreeNode 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 3fa4148f0dff2..f70e80b7f27f2 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 @@ -15,13 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.rules.Rule +import org.apache.spark.sql.Logging /** * A bound reference points to a specific slot in the input tuple, allowing the actual value 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 71f64ef95064e..c26fc3d0f305f 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ 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 2454a3355b11f..81fd160e00ca1 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 @@ -15,13 +15,12 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType} -import org.apache.spark.sql.catalyst.errors.TreeNodeException abstract class Expression extends TreeNode[Expression] { self: Product => @@ -69,7 +68,7 @@ abstract class Expression extends TreeNode[Expression] { def childrenResolved = !children.exists(!_.resolved) /** - * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type + * A set of helper functions that return the correct descendant of `scala.math.Numeric[T]` type * and do any casting necessary of child evaluation. */ @inline 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 8c407d2fddb42..38542d3fc7290 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions /** * Converts a [[Row]] to another Row given a sequence of expression that define each column of the 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 index 0d173afec8901..0bde621602944 100644 --- 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.DoubleType 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 79c91ebaa4746..31d42b9ee71a0 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.NativeType 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 cc339480550a7..f53d8504b083f 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.DataType 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 index 171997b90ecf4..d5d93778f4b8d 100644 --- 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions abstract sealed class SortDirection case object Ascending extends SortDirection 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 01b7a14d4aca6..9828d0b9bd8b2 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import scala.language.dynamics 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 a16bb80df37af..7303b155cae3d 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 @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.catalyst.trees abstract class AggregateExpression extends Expression { self: Product => 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 81e4a487bdbf9..fba056e7c07e3 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.types._ 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 9ec0f6ade7467..ab96618d73df7 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ 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 9097c635ee96f..e9b491b10a5f2 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 @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.types._ /** 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 0d01312c71c70..b82a12e0f754e 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ 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 47b1241e714af..69c8bed309c18 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 @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.types._ 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 38e38371b1dec..5a47768dcb4a1 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 @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedException case class Coalesce(children: Seq[Expression]) extends Expression { diff --git a/sql/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 index 76554e160b1d1..573ec052f4266 100644 --- a/sql/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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * A set of classes that can be used to represent trees of relational expressions. A key goal of @@ -46,6 +45,6 @@ package catalyst * ability to reason about which subquery produced a given attribute. * * ==Evaluation== - * The result of expressions can be evaluated using the [[Evaluate]] object. + * The result of expressions can be evaluated using the `Expression.apply(Row)` method. */ package object expressions 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 e7f3e8ca60a25..722ff517d250e 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 @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.types.{BooleanType, StringType} +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.types.{BooleanType, StringType} trait Predicate extends Expression { self: Product => 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 index 7584fe03cf745..e195f2ac7efd1 100644 --- 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.BooleanType 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 f28076999ddbf..3dd6818029bcf 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package optimizer +package org.apache.spark.sql.catalyst.optimizer import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.Inner @@ -125,7 +123,6 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] { grandChild)) } - // def replaceAlias(condition: Expression, sourceAliases: Map[Attribute, Expression]): Expression = { condition transform { case a: AttributeReference => sourceAliases.getOrElse(a, a) diff --git a/sql/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 index d50b963dfc25b..67833664b35ae 100644 --- a/sql/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 @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package planning +package org.apache.spark.sql.catalyst.planning +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.trees.TreeNode 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 index 64370ec7c0cc4..0a030b6c21f1f 100644 --- 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * Contains classes for enumerating possible physical plans for a given logical query plan. diff --git a/sql/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 index ff0ea90e54842..6dd816aa91dd1 100644 --- a/sql/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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package planning +package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec @@ -48,7 +46,9 @@ 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. + * together with the top project operator. + * [[org.apache.spark.sql.catalyst.expressions.Alias Aliases]] are in-lined/substituted if + * necessary. */ object PhysicalOperation extends PredicateHelper { type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan) diff --git a/sql/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 index 848db2452ae9c..8199a80f5d6bd 100644 --- a/sql/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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans +package org.apache.spark.sql.catalyst.plans import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} import org.apache.spark.sql.catalyst.trees.TreeNode diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala index 9f2283ad4346e..ae8d7d3e4257f 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans +package org.apache.spark.sql.catalyst.plans sealed abstract class JoinType case object Inner extends JoinType 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 48ff45c3d3ebb..7c616788a3830 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 @@ -15,10 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical abstract class BaseRelation extends LeafNode { self: Product => diff --git a/sql/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 index 225dd260fb9de..5eb52d5350f55 100644 --- a/sql/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 @@ -15,14 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.QueryPlan import org.apache.spark.sql.catalyst.types.StructType +import org.apache.spark.sql.catalyst.trees abstract class LogicalPlan extends QueryPlan[LogicalPlan] { self: Product => diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala index 5a3ea9f0a0464..d3f9d0fb93237 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala @@ -15,10 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} diff --git a/sql/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 index ac7d2d6001779..f8fe558511bfd 100644 --- a/sql/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 @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions.Attribute object LocalRelation { 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 61481de65e76e..9d16189deedfe 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 @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans.JoinType import org.apache.spark.sql.catalyst.types._ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode { 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 index 775e50bbd5128..7146fbd540f29 100644 --- 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 @@ -15,10 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package logical +package org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} 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 index a40ab4bbb154b..42bdab42b79ff 100644 --- 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * A a collection of common abstractions for query plans as well as diff --git a/sql/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 index 20e2a45678c89..8893744eb2e7a 100644 --- a/sql/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 @@ -15,10 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package plans -package physical +package org.apache.spark.sql.catalyst.plans.physical import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} import org.apache.spark.sql.catalyst.types.IntegerType 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 index c7632a62a00d2..1076537bc7602 100644 --- 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 @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package rules +package org.apache.spark.sql.catalyst.rules +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.trees.TreeNode abstract class Rule[TreeType <: TreeNode[_]] extends Logging { 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 index 26ab5430821f4..ca82c5d93d4cb 100644 --- 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * A framework for applying batches rewrite rules to trees, possibly to fixed point. diff --git a/sql/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 index 89e27d81dad0b..0369129393a08 100644 --- a/sql/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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package trees +package org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors._ diff --git a/sql/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 index e2da1d2439bbf..d159ecdd5d781 100644 --- a/sql/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 @@ -15,8 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst + +import org.apache.spark.sql.Logger /** * A library for easily manipulating trees of operators. Operators that extend TreeNode are diff --git a/sql/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 index 90a9f9f7e548d..7a45d1a1b8195 100644 --- a/sql/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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package types +package org.apache.spark.sql.catalyst.types import scala.reflect.runtime.universe.{typeTag, TypeTag} 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 index b65a5617d9a7f..de24449590f9a 100644 --- 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst /** * Contains a type system for attributes produced by relations, including complex types like * structs, arrays and maps. 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 52adea2661363..a001d953592db 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File} 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 fb25e1c246117..46b2250aab231 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst +package org.apache.spark.sql.catalyst import org.scalatest.FunSuite @@ -172,4 +171,4 @@ class DistributionSuite extends FunSuite { AllTuples, false) } -} \ No newline at end of file +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala index 78ec48ba77628..4c313585c6386 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.scalatest.FunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala index b85b72a2840f9..b9e0f8e9dcc5f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package analysis +package org.apache.spark.sql.catalyst.analysis import org.scalatest.FunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index c8fd581aa7b47..94894adf81202 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package expressions +package org.apache.spark.sql.catalyst.expressions import org.scalatest.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 53f760fb4ceb2..2ab14f48ccc8a 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 @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package optimizer +package org.apache.spark.sql.catalyst.optimizer -import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor 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 ae1b2b13dd8f1..ef47850455a37 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 @@ -15,12 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package optimizer +package org.apache.spark.sql.catalyst.optimizer - -import org.apache.spark.sql.catalyst.analysis._ +import org.apache.spark.sql.catalyst.analysis +import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ diff --git a/sql/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 index 0c6e4d5acee83..89982d5cd8d74 100644 --- a/sql/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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package optimizer +package org.apache.spark.sql.catalyst.optimizer import org.scalatest.FunSuite diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala index 738cfa85fbc6a..4b2d45584045f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package catalyst -package trees +package org.apache.spark.sql.catalyst.trees import org.scalatest.FunSuite diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index ddbeba6203aa4..e0c98ecdf8f22 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import java.nio.{ByteOrder, ByteBuffer} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 6bd1841821875..3e622adfd3d6a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -15,12 +15,13 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.columnar.ColumnBuilder._ import org.apache.spark.sql.execution.SparkSqlSerializer private[sql] trait ColumnBuilder { @@ -35,7 +36,6 @@ private[sql] trait ColumnBuilder { } private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends ColumnBuilder { - import ColumnBuilder._ private var columnName: String = _ protected var buffer: ByteBuffer = _ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 3b759a51cc695..a452b86f0cda3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -14,8 +14,8 @@ * See the License for the specific language governing permissions and * limitations under the License. */ -package org.apache.spark.sql -package columnar + +package org.apache.spark.sql.columnar import java.nio.ByteBuffer diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index 1661c3f3ff4a9..048d1f05c7df2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -15,10 +15,11 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar -import java.nio.{ByteOrder, ByteBuffer} +import java.nio.{ByteBuffer, ByteOrder} + +import org.apache.spark.sql.Row /** * Builds a nullable column. The byte buffer of a nullable column contains: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala index c7efd30e87da4..f853759e5a306 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala @@ -15,18 +15,18 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} import org.apache.spark.sql.execution.{SparkPlan, LeafNode} +import org.apache.spark.sql.Row + +/* Implicit conversions */ +import org.apache.spark.sql.columnar.ColumnType._ private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan) extends LeafNode { - // For implicit conversion from `DataType` to `ColumnType` - import ColumnType._ - override def output: Seq[Attribute] = attributes lazy val cachedColumnBuffers = { 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 65d77e3a40cf4..869673b1fe978 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 @@ -15,16 +15,16 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution +import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.ShuffledRDD +import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.errors.attachTree import org.apache.spark.sql.catalyst.expressions.{MutableProjection, RowOrdering} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair -import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { 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 7e50fda4ceac5..e902e6ced521d 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 @@ -15,10 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution -import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, Projection} /** * Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the 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 index 7ce8608d2061d..16806c620635f 100644 --- 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 @@ -15,7 +15,6 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution class QueryExecutionException(message: String) extends Exception(message) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 5626181d184e5..acb1ee83a72f6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -15,16 +15,15 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import org.apache.spark.rdd.RDD - +import org.apache.spark.sql.{Logging, Row} +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation -import org.apache.spark.sql.catalyst.plans.QueryPlan -import org.apache.spark.sql.catalyst.plans.logical +import org.apache.spark.sql.catalyst.expressions.GenericRow +import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} import org.apache.spark.sql.catalyst.plans.physical._ -import org.apache.spark.sql.catalyst.trees abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { self: Product => @@ -47,7 +46,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.toArray) + new GenericRow(values.toArray) } /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index ad7cd58b6aaaf..1c3196ae2e7b6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import java.nio.ByteBuffer 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 8a39ded0a9ec4..86f9d3e0fa954 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 @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution +import org.apache.spark.sql.{SQLContext, execution} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ @@ -175,7 +175,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => InsertIntoParquetTable(table, planLater(child))(sparkContext) :: Nil - case PhysicalOperation(projectList, filters, relation: parquet.ParquetRelation) => + case PhysicalOperation(projectList, filters, relation: ParquetRelation) => // TODO: Should be pushing down filters as well. pruneFilterProject( projectList, 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 14e5ab628f23d..8515a18f18c55 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.errors._ 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 e4f918b6787fe..65cb8f8becefa 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import scala.reflect.runtime.universe.TypeTag 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 index db259b4c4b95b..40982f1fffbbf 100644 --- 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution object DebugQuery { def apply(plan: SparkPlan): SparkPlan = { 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 a6e3892e88703..f0d21143ba5d1 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import scala.collection.mutable 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 011aaf7440cb1..2b825f84ee910 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 @@ -82,11 +82,10 @@ object ParquetRelation { type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow /** - * Creates a new ParquetRelation and underlying Parquetfile for the given - * 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. + * Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that + * this is used inside [[org.apache.spark.sql.execution.SparkStrategies 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. 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 91b4848fe403f..c21e400282004 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 @@ -135,8 +135,7 @@ 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. + * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record to a `Row` object. * * @param schema The corresponding Catalyst schema in the form of a list of attributes. */ diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala index ca56c4476bcc2..f2389f8f0591e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala @@ -15,9 +15,10 @@ * limitations under the License. */ -package org.apache.spark -package sql -package test +package org.apache.spark.sql.test + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.SQLContext /** A SQLContext that can be used for local testing. */ object TestSQLContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala index 5c8cb086ee3b8..d719ceb827691 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala @@ -22,8 +22,6 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.util._ -/* Implicits */ - class QueryTest extends FunSuite { /** * Runs the plan and makes sure the answer matches the expected result. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 1997d8638cb61..2d431affbcfcc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import java.nio.ByteBuffer diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 279607ccfaa5b..d413d483f4e7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types.DataType diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 3354da3fa3e0f..5222a47e1ab87 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package columnar +package org.apache.spark.sql.columnar import org.scalatest.FunSuite diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala index 93b2a308a410a..ca5c8b8eb63dc 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package execution +package org.apache.spark.sql.execution import org.apache.spark.sql.QueryTest import org.apache.spark.sql.catalyst.expressions._ diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index d96c2f70e0c74..7219c030cb0f0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -113,11 +113,10 @@ class SparkHiveHadoopWriter( cmtr.commitTask(taCtxt) logInfo (taID + ": Committed") } catch { - case e: IOException => { + 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) @@ -161,12 +160,12 @@ class SparkHiveHadoopWriter( taskContext } - private def setIDs(jobid: Int, splitid: Int, attemptid: Int) { - jobID = jobid - splitID = splitid - attemptID = attemptid + private def setIDs(jobId: Int, splitId: Int, attemptId: Int) { + jobID = jobId + splitID = splitId + attemptID = attemptId - jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobid)) + jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId)) taID = new SerializableWritable[TaskAttemptID]( new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID)) } 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 3bcf586662f2c..fc5057b73fe24 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 @@ -32,7 +32,7 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog} import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan, LowerCaseSchema} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema} import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand, ExplainCommand} import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution._ @@ -86,7 +86,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val input = new java.io.InputStream { val iterator = (start ++ end).iterator - def read(): Int = if (iterator.hasNext) iterator.next else -1 + def read(): Int = if (iterator.hasNext) iterator.next() else -1 } val reader = new BufferedReader(new InputStreamReader(input)) val stringBuilder = new StringBuilder @@ -148,24 +148,24 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { SessionState.start(sessionState) - if (proc.isInstanceOf[Driver]) { - val driver: Driver = proc.asInstanceOf[Driver] - driver.init() + proc match { + case driver: 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) { + 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) - } - driver.setMaxRows(maxRows) - driver.getResults(results) - driver.destroy() - results - } else { - sessionState.out.println(tokens(0) + " " + cmd_1) - Seq(proc.run(cmd_1).getResponseCode.toString) + results + case _ => + sessionState.out.println(tokens(0) + " " + cmd_1) + Seq(proc.run(cmd_1).getResponseCode.toString) } } catch { case e: Exception => @@ -226,7 +226,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { sparkContext.parallelize(asRows, 1) } case _ => - executedPlan.execute.map(_.copy()) + executedPlan.execute().map(_.copy()) } protected val primitiveTypes = 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 1667a217297b1..4f8353666a12b 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import scala.util.parsing.combinator.RegexParsers @@ -27,14 +26,15 @@ 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.spark.sql.catalyst.analysis.{Catalog, EliminateAnalysisOperators} +import org.apache.spark.sql.Logging +import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ +/* Implicit conversions */ import scala.collection.JavaConversions._ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { 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 8e76a7348e957..f4b61381f9a27 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import org.apache.hadoop.hive.ql.lib.Node import org.apache.hadoop.hive.ql.parse._ 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 141067247d736..3ca1d93c11fa9 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 @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive +import org.apache.spark.sql.SQLContext import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ 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 dc4181ec9905c..2610100043659 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import java.io.{BufferedReader, InputStreamReader} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index 99dc85ec19868..ca5311344615f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{Path, PathFilter} diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index a26b0ff2319d4..bc3447b9d802d 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import java.io.File import java.util.{Set => JavaSet} 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 78f69e7ff5731..e2d9d8de2572a 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.metastore.MetaStoreUtils 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 72ccd4f4a4302..44901db3f963b 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 @@ -15,8 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive +package org.apache.spark.sql.hive import scala.collection.mutable.ArrayBuffer @@ -29,6 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive._ import org.apache.hadoop.hive.serde2.{io => hiveIo} import org.apache.hadoop.{io => hadoopIo} +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 4b45e698601eb..8488f23abd0fd 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -15,19 +15,17 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution import java.io.File +import org.apache.spark.sql.hive.TestHive._ + /** * A set of test cases based on the big-data-benchmark. * https://amplab.cs.berkeley.edu/benchmark/ */ class BigDataBenchmarkSuite extends HiveComparisonTest { - import TestHive._ - val testDataDirectory = new File("target/big-data-benchmark-testdata") val testTables = Seq( @@ -123,4 +121,4 @@ class BigDataBenchmarkSuite extends HiveComparisonTest { |-- SELECT * FROM url_counts_total """.stripMargin) } -} \ No newline at end of file +} 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 index 02ee2a0ebc354..ac87f2cb10d12 100644 --- 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 @@ -15,12 +15,11 @@ * limitations under the License. */ -package org.apache.spark -package sql -package hive -package execution +package org.apache.spark.sql.hive.execution -import org.scalatest.{FunSuite, BeforeAndAfterAll} +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.hive.TestHiveContext +import org.scalatest.{BeforeAndAfterAll, FunSuite} class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { ignore("multiple instances not supported") { 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 e8fcc272354fa..c7a350ef94edd 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 @@ -15,17 +15,16 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution import java.io._ -import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} - +import org.apache.spark.sql.Logging import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeCommand} import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.Sort +import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} +import org.apache.spark.sql.hive.TestHive /** * Allows the creations of tests that execute the same query against both hive diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index d5b54fa90fed3..f74b0fbb97c83 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -15,9 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.hive.TestHive /** * Runs the test cases that are included in the hive distribution. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala index 2d2f13333a65c..50ab71a9003d3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution import java.io.File diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index b804634db1f5f..4b92d167a1263 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -15,16 +15,14 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution + +import org.apache.spark.sql.hive.TestHive._ /** * 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 TestHive._ - createQueryTest("Simple Average", "SELECT AVG(key) FROM src") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 4bdea214677ad..d77900ddc950c 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -15,11 +15,10 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution -import TestHive._ +import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.TestHive._ case class Data(a: Int, B: Int, n: Nested) case class Nested(a: Int, B: Int) 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 index c2264926f4356..df9bae96494d5 100644 --- 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 @@ -15,9 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution /** * A set of tests that validates support for Hive SerDe. 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 index bb33583e5f582..e030c8ee3dfc8 100644 --- 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 @@ -21,7 +21,6 @@ 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 => @@ -29,4 +28,4 @@ class HiveTypeCoercionSuite extends HiveComparisonTest { createQueryTest(s"$i + $j", s"SELECT $i + $j FROM src LIMIT 1") } } -} \ No newline at end of file +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index d2f8e5df5b29e..1318ac1968dad 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -15,11 +15,9 @@ * limitations under the License. */ -package org.apache.spark.sql -package hive -package execution +package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.{HiveTableScan, TestHive} /* Implicit conversions */ import scala.collection.JavaConversions._ From 32cbdfd2887f7a792f360ac3224f8c38cc97d21f Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 26 Mar 2014 18:19:15 -0700 Subject: [PATCH 141/397] [SQL] Un-ignore a test that is now passing. Add golden answer for aforementioned test. Also, fix golden test generation from sbt/sbt by setting the classpath correctly. Author: Michael Armbrust Closes #244 from marmbrus/partTest and squashes the following commits: 37a33c9 [Michael Armbrust] Un-ignore a test that is now passing, add golden answer for aforementioned test. Fix golden test generation from sbt/sbt. --- sbt/sbt | 8 + ...le scan-0-3e8898a13ccef627603f340d1f8bdd80 | 2000 +++++++++++++++++ .../sql/hive/execution/HiveQuerySuite.scala | 6 +- 3 files changed, 2010 insertions(+), 4 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80 diff --git a/sbt/sbt b/sbt/sbt index 3ffa4ed9ab5a7..9de265bd07dcb 100755 --- a/sbt/sbt +++ b/sbt/sbt @@ -1,5 +1,13 @@ #!/usr/bin/env bash +# When creating new tests for Spark SQL Hive, the HADOOP_CLASSPATH must contain the hive jars so +# that we can run Hive to generate the golden answer. This is not required for normal development +# or testing. +for i in $HIVE_HOME/lib/* +do HADOOP_CLASSPATH=$HADOOP_CLASSPATH:$i +done +export HADOOP_CLASSPATH + realpath () { ( TARGET_FILE=$1 diff --git a/sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80 b/sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80 new file mode 100644 index 0000000000000..a3cb00feaca62 --- /dev/null +++ b/sql/hive/src/test/resources/golden/partitioned table scan-0-3e8898a13ccef627603f340d1f8bdd80 @@ -0,0 +1,2000 @@ +2008-04-08 11 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 12 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-09 11 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 12 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 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 4b92d167a1263..c184ebe288af4 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -53,10 +53,8 @@ class HiveQuerySuite 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("partitioned table scan", + "SELECT ds, hr, key, value FROM srcpart") createQueryTest("hash", "SELECT hash('test') FROM src LIMIT 1") From e15e57413e07e5d4787514702f735bba0c30cae5 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 26 Mar 2014 18:19:49 -0700 Subject: [PATCH 142/397] [SQL] Add a custom serializer for maps since they do not have a no-arg constructor. Author: Michael Armbrust Closes #243 from marmbrus/mapSer and squashes the following commits: 54045f7 [Michael Armbrust] Add a custom serializer for maps since they do not have a no-arg constructor. --- .../sql/execution/SparkSqlSerializer.scala | 18 ++++++++++++++++++ 1 file changed, 18 insertions(+) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 1c3196ae2e7b6..915f551fb2f01 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -32,6 +32,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { kryo.setRegistrationRequired(false) kryo.register(classOf[MutablePair[_, _]]) kryo.register(classOf[Array[Any]]) + kryo.register(classOf[scala.collection.immutable.Map$Map1], new MapSerializer) 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[_]]) @@ -70,3 +71,20 @@ class BigDecimalSerializer extends Serializer[BigDecimal] { BigDecimal(input.readString()) } } + +/** + * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize + * them as `Array[(k,v)]`. + */ +class MapSerializer extends Serializer[Map[_,_]] { + def write(kryo: Kryo, output: Output, map: Map[_,_]) { + kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray) + } + + def read(kryo: Kryo, input: Input, tpe: Class[Map[_,_]]): Map[_,_] = { + kryo.readObject(input, classOf[Array[Any]]) + .sliding(2,2) + .map { case Array(k,v) => (k,v) } + .toMap + } +} From be6d96c15b3c31cd27bdd79fb259072479151ae6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 26 Mar 2014 18:22:15 -0700 Subject: [PATCH 143/397] SPARK-1324: SparkUI Should Not Bind to SPARK_PUBLIC_DNS /cc @aarondav and @andrewor14 Author: Patrick Wendell Closes #231 from pwendell/ui-binding and squashes the following commits: e8025f8 [Patrick Wendell] SPARK-1324: SparkUI Should Not Bind to SPARK_PUBLIC_DNS --- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 9 +++++---- 1 file changed, 5 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index fd638c83aac6e..ef1ad872c8ef7 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -47,7 +47,8 @@ private[spark] class SparkUI( val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - private val host = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(Utils.localHostName()) + private val bindHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt private var serverInfo: Option[ServerInfo] = None @@ -79,8 +80,8 @@ private[spark] class SparkUI( /** Bind the HTTP server which backs this web interface */ def bind() { try { - serverInfo = Some(startJettyServer(host, port, handlers, sc.conf)) - logInfo("Started Spark Web UI at http://%s:%d".format(host, boundPort)) + serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) + logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) } catch { case e: Exception => logError("Failed to create Spark JettyUtils", e) @@ -111,7 +112,7 @@ private[spark] class SparkUI( logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = "http://" + host + ":" + boundPort + private[spark] def appUIAddress = "http://" + publicHost + ":" + boundPort } From 3e63d98f09065386901d78c141b0da93cdce0f76 Mon Sep 17 00:00:00 2001 From: NirmalReddy Date: Wed, 26 Mar 2014 18:24:55 -0700 Subject: [PATCH 144/397] Spark 1095 : Adding explicit return types to all public methods Excluded those that are self-evident and the cases that are discussed in the mailing list. Author: NirmalReddy Author: NirmalReddy Closes #168 from NirmalReddy/Spark-1095 and squashes the following commits: ac54b29 [NirmalReddy] import misplaced 8c5ff3e [NirmalReddy] Changed syntax of unit returning methods 02d0778 [NirmalReddy] fixed explicit types in all the other packages 1c17773 [NirmalReddy] fixed explicit types in core package --- .../scala/org/apache/spark/SparkContext.scala | 31 ++++++++++++------- .../apache/spark/api/java/JavaRDDLike.scala | 15 ++++++--- .../spark/api/java/JavaSparkContext.scala | 2 +- .../apache/spark/deploy/ClientArguments.scala | 2 +- .../apache/spark/deploy/SparkHadoopUtil.scala | 2 +- .../master/ZooKeeperPersistenceEngine.scala | 3 +- .../spark/metrics/sink/ConsoleSink.scala | 2 +- .../apache/spark/metrics/sink/CsvSink.scala | 2 +- .../spark/metrics/sink/GraphiteSink.scala | 4 +-- .../org/apache/spark/rdd/CoGroupedRDD.scala | 2 +- .../org/apache/spark/rdd/HadoopRDD.scala | 2 +- .../scala/org/apache/spark/rdd/JdbcRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 2 +- .../apache/spark/storage/StorageLevel.scala | 7 +++-- .../org/apache/spark/util/Distribution.scala | 7 +++-- .../spark/metrics/sink/GangliaSink.scala | 12 ++++--- .../scala/org/apache/spark/graphx/Graph.scala | 3 +- .../apache/spark/graphx/impl/GraphImpl.scala | 2 +- .../apache/spark/graphx/lib/Analytics.scala | 2 +- .../spark/streaming/StreamingContext.scala | 4 +-- .../streaming/api/java/JavaDStreamLike.scala | 6 ++-- .../api/java/JavaStreamingContext.scala | 22 +++++++++---- .../spark/streaming/dstream/DStream.scala | 8 +++-- .../spark/streaming/scheduler/BatchInfo.scala | 8 +++-- 25 files changed, 97 insertions(+), 57 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 4dd298177f07d..b23accbbb9410 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary +import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ @@ -230,7 +231,7 @@ class SparkContext( postEnvironmentUpdate() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ - val hadoopConfiguration = { + val hadoopConfiguration: Configuration = { val env = SparkEnv.get val hadoopConf = SparkHadoopUtil.get.newConfiguration() // Explicitly check for S3 environment variables @@ -630,7 +631,7 @@ class SparkContext( * standard mutable collections. So you can use this with mutable Map, Set, etc. */ def accumulableCollection[R <% Growable[T] with TraversableOnce[T] with Serializable, T] - (initialValue: R) = { + (initialValue: R): Accumulable[R, T] = { val param = new GrowableAccumulableParam[R,T] new Accumulable(initialValue, param) } @@ -640,7 +641,7 @@ class SparkContext( * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T) = env.broadcastManager.newBroadcast[T](value, isLocal) + def broadcast[T](value: T): Broadcast[T] = env.broadcastManager.newBroadcast[T](value, isLocal) /** * Add a file to be downloaded with this Spark job on every node. @@ -1126,7 +1127,7 @@ object SparkContext extends Logging { implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) implicit def rddToSequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable: ClassTag]( - rdd: RDD[(K, V)]) = + rdd: RDD[(K, V)]) = new SequenceFileRDDFunctions(rdd) implicit def rddToOrderedRDDFunctions[K <% Ordered[K]: ClassTag, V: ClassTag]( @@ -1163,27 +1164,33 @@ object SparkContext extends Logging { } // Helper objects for converting common types to Writable - private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) = { + private def simpleWritableConverter[T, W <: Writable: ClassTag](convert: W => T) + : WritableConverter[T] = { val wClass = classTag[W].runtimeClass.asInstanceOf[Class[W]] new WritableConverter[T](_ => wClass, x => convert(x.asInstanceOf[W])) } - implicit def intWritableConverter() = simpleWritableConverter[Int, IntWritable](_.get) + implicit def intWritableConverter(): WritableConverter[Int] = + simpleWritableConverter[Int, IntWritable](_.get) - implicit def longWritableConverter() = simpleWritableConverter[Long, LongWritable](_.get) + implicit def longWritableConverter(): WritableConverter[Long] = + simpleWritableConverter[Long, LongWritable](_.get) - implicit def doubleWritableConverter() = simpleWritableConverter[Double, DoubleWritable](_.get) + implicit def doubleWritableConverter(): WritableConverter[Double] = + simpleWritableConverter[Double, DoubleWritable](_.get) - implicit def floatWritableConverter() = simpleWritableConverter[Float, FloatWritable](_.get) + implicit def floatWritableConverter(): WritableConverter[Float] = + simpleWritableConverter[Float, FloatWritable](_.get) - implicit def booleanWritableConverter() = + implicit def booleanWritableConverter(): WritableConverter[Boolean] = simpleWritableConverter[Boolean, BooleanWritable](_.get) - implicit def bytesWritableConverter() = { + implicit def bytesWritableConverter(): WritableConverter[Array[Byte]] = { simpleWritableConverter[Array[Byte], BytesWritable](_.getBytes) } - implicit def stringWritableConverter() = simpleWritableConverter[String, Text](_.toString) + implicit def stringWritableConverter(): WritableConverter[String] = + simpleWritableConverter[String, Text](_.toString) implicit def writableWritableConverter[T <: Writable]() = new WritableConverter[T](_.runtimeClass.asInstanceOf[Class[T]], _.asInstanceOf[T]) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index ddac553304233..e03b8e78d5f52 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -391,19 +391,24 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { /** * Save this RDD as a text file, using string representations of elements. */ - def saveAsTextFile(path: String) = rdd.saveAsTextFile(path) + def saveAsTextFile(path: String): Unit = { + rdd.saveAsTextFile(path) + } /** * Save this RDD as a compressed text file, using string representations of elements. */ - def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]) = + def saveAsTextFile(path: String, codec: Class[_ <: CompressionCodec]): Unit = { rdd.saveAsTextFile(path, codec) + } /** * Save this RDD as a SequenceFile of serialized objects. */ - def saveAsObjectFile(path: String) = rdd.saveAsObjectFile(path) + def saveAsObjectFile(path: String): Unit = { + rdd.saveAsObjectFile(path) + } /** * Creates tuples of the elements in this RDD by applying `f`. @@ -420,7 +425,9 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * executed on this RDD. It is strongly recommended that this RDD is persisted in * memory, otherwise saving it on a file will require recomputation. */ - def checkpoint() = rdd.checkpoint() + def checkpoint(): Unit = { + rdd.checkpoint() + } /** * Return whether this RDD has been checkpointed or not diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 35508b6e5acba..e531a57aced31 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -463,7 +463,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork sc.setCheckpointDir(dir) } - def getCheckpointDir = JavaUtils.optionToOptional(sc.getCheckpointDir) + def getCheckpointDir: Optional[String] = JavaUtils.optionToOptional(sc.getCheckpointDir) protected def checkpointFile[T](path: String): JavaRDD[T] = { implicit val ctag: ClassTag[T] = fakeClassTag diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index 00f5cd54ad650..c07838f798799 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -112,5 +112,5 @@ private[spark] class ClientArguments(args: Array[String]) { } object ClientArguments { - def isValidJarUrl(s: String) = s.matches("(.+):(.+)jar") + def isValidJarUrl(s: String): Boolean = s.matches("(.+):(.+)jar") } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index d2d8d6d662d55..9bdbfb33bf54f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -32,7 +32,7 @@ import scala.collection.JavaConversions._ * Contains util methods to interact with Hadoop from Spark. */ class SparkHadoopUtil { - val conf = newConfiguration() + val conf: Configuration = newConfiguration() UserGroupInformation.setConfiguration(conf) def runAsUser(user: String)(func: () => Unit) { diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala index 5413ff671ad8d..834dfedee52ce 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ZooKeeperPersistenceEngine.scala @@ -20,6 +20,7 @@ package org.apache.spark.deploy.master import scala.collection.JavaConversions._ import akka.serialization.Serialization +import org.apache.curator.framework.CuratorFramework import org.apache.zookeeper.CreateMode import org.apache.spark.{Logging, SparkConf} @@ -29,7 +30,7 @@ class ZooKeeperPersistenceEngine(serialization: Serialization, conf: SparkConf) with Logging { val WORKING_DIR = conf.get("spark.deploy.zookeeper.dir", "/spark") + "/master_status" - val zk = SparkCuratorUtil.newClient(conf) + val zk: CuratorFramework = SparkCuratorUtil.newClient(conf) SparkCuratorUtil.mkdir(zk, WORKING_DIR) diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 4d2ffc54d8983..64eac73605388 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -38,7 +38,7 @@ class ConsoleSink(val property: Properties, val registry: MetricRegistry, case None => CONSOLE_DEFAULT_PERIOD } - val pollUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { + val pollUnit: TimeUnit = Option(property.getProperty(CONSOLE_KEY_UNIT)) match { case Some(s) => TimeUnit.valueOf(s.toUpperCase()) case None => TimeUnit.valueOf(CONSOLE_DEFAULT_UNIT) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 319f40815d65f..544848d4150b6 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -41,7 +41,7 @@ class CsvSink(val property: Properties, val registry: MetricRegistry, case None => CSV_DEFAULT_PERIOD } - val pollUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { + val pollUnit: TimeUnit = Option(property.getProperty(CSV_KEY_UNIT)) match { case Some(s) => TimeUnit.valueOf(s.toUpperCase()) case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index 0ffdf3846dc4a..7f0a2fd16fa99 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -39,7 +39,7 @@ class GraphiteSink(val property: Properties, val registry: MetricRegistry, val GRAPHITE_KEY_UNIT = "unit" val GRAPHITE_KEY_PREFIX = "prefix" - def propertyToOption(prop: String) = Option(property.getProperty(prop)) + def propertyToOption(prop: String): Option[String] = Option(property.getProperty(prop)) if (!propertyToOption(GRAPHITE_KEY_HOST).isDefined) { throw new Exception("Graphite sink requires 'host' property.") @@ -57,7 +57,7 @@ class GraphiteSink(val property: Properties, val registry: MetricRegistry, case None => GRAPHITE_DEFAULT_PERIOD } - val pollUnit = propertyToOption(GRAPHITE_KEY_UNIT) match { + val pollUnit: TimeUnit = propertyToOption(GRAPHITE_KEY_UNIT) match { case Some(s) => TimeUnit.valueOf(s.toUpperCase()) case None => TimeUnit.valueOf(GRAPHITE_DEFAULT_UNIT) } diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 8561711931047..9aa454a5c8b88 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -103,7 +103,7 @@ class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: array } - override val partitioner = Some(part) + override val partitioner: Some[Partitioner] = Some(part) override def compute(s: Partition, context: TaskContext): Iterator[(K, CoGroupCombiner)] = { val sparkConf = SparkEnv.get.conf diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 932ff5bf369c7..3af008bd72378 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -171,7 +171,7 @@ class HadoopRDD[K, V]( array } - override def compute(theSplit: Partition, context: TaskContext) = { + override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { val iter = new NextIterator[(K, V)] { val split = theSplit.asInstanceOf[HadoopPartition] diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 8df8718f3b65b..1b503743ac117 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -116,7 +116,7 @@ class JdbcRDD[T: ClassTag]( } object JdbcRDD { - def resultSetToObjectArray(rs: ResultSet) = { + def resultSetToObjectArray(rs: ResultSet): Array[Object] = { Array.tabulate[Object](rs.getMetaData.getColumnCount)(i => rs.getObject(i + 1)) } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index d1fff296878c3..461a749eac48b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -80,7 +80,7 @@ class NewHadoopRDD[K, V]( result } - override def compute(theSplit: Partition, context: TaskContext) = { + override def compute(theSplit: Partition, context: TaskContext): InterruptibleIterator[(K, V)] = { val iter = new Iterator[(K, V)] { val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6af42248a5c3c..ce2b8ac27206b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -121,7 +121,7 @@ abstract class RDD[T: ClassTag]( @transient var name: String = null /** Assign a name to this RDD */ - def setName(_name: String) = { + def setName(_name: String): RDD[T] = { name = _name this } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 1b7934d59fa1d..4212a539dab4b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -126,15 +126,16 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2) /** Create a new StorageLevel object */ - def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) = + def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, + replication: Int = 1): StorageLevel = getCachedStorageLevel(new StorageLevel(useDisk, useMemory, deserialized, replication)) /** Create a new StorageLevel object from its integer representation */ - def apply(flags: Int, replication: Int) = + def apply(flags: Int, replication: Int): StorageLevel = getCachedStorageLevel(new StorageLevel(flags, replication)) /** Read StorageLevel object from ObjectInput stream */ - def apply(in: ObjectInput) = { + def apply(in: ObjectInput): StorageLevel = { val obj = new StorageLevel() obj.readExternal(in) getCachedStorageLevel(obj) diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index ab738c4b868fa..5b347555fe708 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -19,6 +19,8 @@ package org.apache.spark.util import java.io.PrintStream +import scala.collection.immutable.IndexedSeq + /** * Util for getting some stats from a small sample of numeric values, with some handy * summary functions. @@ -40,7 +42,8 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) * given from 0 to 1 * @param probabilities */ - def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) = { + def getQuantiles(probabilities: Traversable[Double] = defaultProbabilities) + : IndexedSeq[Double] = { probabilities.toIndexedSeq.map{p:Double => data(closestIndex(p))} } @@ -48,7 +51,7 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) math.min((p * length).toInt + startIdx, endIdx - 1) } - def showQuantiles(out: PrintStream = System.out) = { + def showQuantiles(out: PrintStream = System.out): Unit = { out.println("min\t25%\t50%\t75%\tmax") getQuantiles(defaultProbabilities).foreach{q => out.print(q + "\t")} out.println diff --git a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala index cd37317da77de..d03d7774e8c80 100644 --- a/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala +++ b/extras/spark-ganglia-lgpl/src/main/scala/org/apache/spark/metrics/sink/GangliaSink.scala @@ -23,6 +23,7 @@ import java.util.concurrent.TimeUnit import com.codahale.metrics.MetricRegistry import com.codahale.metrics.ganglia.GangliaReporter import info.ganglia.gmetric4j.gmetric.GMetric +import info.ganglia.gmetric4j.gmetric.GMetric.UDPAddressingMode import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem @@ -33,10 +34,10 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val GANGLIA_DEFAULT_PERIOD = 10 val GANGLIA_KEY_UNIT = "unit" - val GANGLIA_DEFAULT_UNIT = TimeUnit.SECONDS + val GANGLIA_DEFAULT_UNIT: TimeUnit = TimeUnit.SECONDS val GANGLIA_KEY_MODE = "mode" - val GANGLIA_DEFAULT_MODE = GMetric.UDPAddressingMode.MULTICAST + val GANGLIA_DEFAULT_MODE: UDPAddressingMode = GMetric.UDPAddressingMode.MULTICAST // TTL for multicast messages. If listeners are X hops away in network, must be at least X. val GANGLIA_KEY_TTL = "ttl" @@ -45,7 +46,7 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val GANGLIA_KEY_HOST = "host" val GANGLIA_KEY_PORT = "port" - def propertyToOption(prop: String) = Option(property.getProperty(prop)) + def propertyToOption(prop: String): Option[String] = Option(property.getProperty(prop)) if (!propertyToOption(GANGLIA_KEY_HOST).isDefined) { throw new Exception("Ganglia sink requires 'host' property.") @@ -58,11 +59,12 @@ class GangliaSink(val property: Properties, val registry: MetricRegistry, val host = propertyToOption(GANGLIA_KEY_HOST).get val port = propertyToOption(GANGLIA_KEY_PORT).get.toInt val ttl = propertyToOption(GANGLIA_KEY_TTL).map(_.toInt).getOrElse(GANGLIA_DEFAULT_TTL) - val mode = propertyToOption(GANGLIA_KEY_MODE) + val mode: UDPAddressingMode = propertyToOption(GANGLIA_KEY_MODE) .map(u => GMetric.UDPAddressingMode.valueOf(u.toUpperCase)).getOrElse(GANGLIA_DEFAULT_MODE) val pollPeriod = propertyToOption(GANGLIA_KEY_PERIOD).map(_.toInt) .getOrElse(GANGLIA_DEFAULT_PERIOD) - val pollUnit = propertyToOption(GANGLIA_KEY_UNIT).map(u => TimeUnit.valueOf(u.toUpperCase)) + val pollUnit: TimeUnit = propertyToOption(GANGLIA_KEY_UNIT) + .map(u => TimeUnit.valueOf(u.toUpperCase)) .getOrElse(GANGLIA_DEFAULT_UNIT) MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 65a1a8c68f6d2..ef05623d7a0a1 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -419,5 +419,6 @@ object Graph { * All the convenience operations are defined in the [[GraphOps]] class which may be * shared across multiple graph implementations. */ - implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag](g: Graph[VD, ED]) = g.ops + implicit def graphToGraphOps[VD: ClassTag, ED: ClassTag] + (g: Graph[VD, ED]): GraphOps[VD, ED] = g.ops } // end of Graph object diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 5e9be18990ba3..43ac11d8957f6 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -197,7 +197,7 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( override def mapReduceTriplets[A: ClassTag]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], reduceFunc: (A, A) => A, - activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None) = { + activeSetOpt: Option[(VertexRDD[_], EdgeDirection)] = None): VertexRDD[A] = { ClosureCleaner.clean(mapFunc) ClosureCleaner.clean(reduceFunc) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala index 24699dfdd38b0..fa533a512d53b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/lib/Analytics.scala @@ -26,7 +26,7 @@ import org.apache.spark.graphx.PartitionStrategy._ */ object Analytics extends Logging { - def main(args: Array[String]) = { + def main(args: Array[String]): Unit = { val host = args(0) val taskType = args(1) val fname = args(2) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 062b888e80b48..e198c69470c1f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -431,7 +431,7 @@ class StreamingContext private[streaming] ( * Stop the execution of the streams. * @param stopSparkContext Stop the associated SparkContext or not */ - def stop(stopSparkContext: Boolean = true) = synchronized { + def stop(stopSparkContext: Boolean = true): Unit = synchronized { scheduler.stop() logInfo("StreamingContext stopped successfully") waiter.notifyStop() @@ -489,7 +489,7 @@ object StreamingContext extends Logging { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. */ - def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls) + def jarOfClass(cls: Class[_]): Seq[String] = SparkContext.jarOfClass(cls) private[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { // Set the default cleaner delay to an hour if not already set. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index a85cd04c9319c..bb2f492d06a00 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -49,7 +49,9 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Print the first ten elements of each RDD generated in this DStream. This is an output * operator, so this DStream will be registered as an output stream and there materialized. */ - def print() = dstream.print() + def print(): Unit = { + dstream.print() + } /** * Return a new DStream in which each RDD has a single element generated by counting each RDD @@ -401,7 +403,7 @@ trait JavaDStreamLike[T, This <: JavaDStreamLike[T, This, R], R <: JavaRDDLike[T * Enable periodic checkpointing of RDDs of this DStream. * @param interval Time interval after which generated RDD will be checkpointed */ - def checkpoint(interval: Duration) = { + def checkpoint(interval: Duration): DStream[T] = { dstream.checkpoint(interval) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index c48d754e439e9..b705d2ec9a58e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -477,31 +477,41 @@ class JavaStreamingContext(val ssc: StreamingContext) { /** * Start the execution of the streams. */ - def start() = ssc.start() + def start(): Unit = { + ssc.start() + } /** * Wait for the execution to stop. Any exceptions that occurs during the execution * will be thrown in this thread. */ - def awaitTermination() = ssc.awaitTermination() + def awaitTermination(): Unit = { + ssc.awaitTermination() + } /** * Wait for the execution to stop. Any exceptions that occurs during the execution * will be thrown in this thread. * @param timeout time to wait in milliseconds */ - def awaitTermination(timeout: Long) = ssc.awaitTermination(timeout) + def awaitTermination(timeout: Long): Unit = { + ssc.awaitTermination(timeout) + } /** * Stop the execution of the streams. Will stop the associated JavaSparkContext as well. */ - def stop() = ssc.stop() + def stop(): Unit = { + ssc.stop() + } /** * Stop the execution of the streams. * @param stopSparkContext Stop the associated SparkContext or not */ - def stop(stopSparkContext: Boolean) = ssc.stop(stopSparkContext) + def stop(stopSparkContext: Boolean): Unit = { + ssc.stop(stopSparkContext) + } } /** @@ -579,7 +589,7 @@ object JavaStreamingContext { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. */ - def jarOfClass(cls: Class[_]) = SparkContext.jarOfClass(cls).toArray + def jarOfClass(cls: Class[_]): Array[String] = SparkContext.jarOfClass(cls).toArray } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 6bff56a9d332a..d48b51aa69565 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -503,14 +503,18 @@ abstract class DStream[T: ClassTag] ( * 'this' DStream will be registered as an output stream and therefore materialized. */ @deprecated("use foreachRDD", "0.9.0") - def foreach(foreachFunc: RDD[T] => Unit) = this.foreachRDD(foreachFunc) + def foreach(foreachFunc: RDD[T] => Unit): Unit = { + this.foreachRDD(foreachFunc) + } /** * Apply a function to each RDD in this DStream. This is an output operator, so * 'this' DStream will be registered as an output stream and therefore materialized. */ @deprecated("use foreachRDD", "0.9.0") - def foreach(foreachFunc: (RDD[T], Time) => Unit) = this.foreachRDD(foreachFunc) + def foreach(foreachFunc: (RDD[T], Time) => Unit): Unit = { + this.foreachRDD(foreachFunc) + } /** * Apply a function to each RDD in this DStream. This is an output operator, so diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 4e8d07fe921fb..7f3cd2f8eb1fd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -39,17 +39,19 @@ case class BatchInfo( * was submitted to the streaming scheduler. Essentially, it is * `processingStartTime` - `submissionTime`. */ - def schedulingDelay = processingStartTime.map(_ - submissionTime) + def schedulingDelay: Option[Long] = processingStartTime.map(_ - submissionTime) /** * Time taken for the all jobs of this batch to finish processing from the time they started * processing. Essentially, it is `processingEndTime` - `processingStartTime`. */ - def processingDelay = processingEndTime.zip(processingStartTime).map(x => x._1 - x._2).headOption + def processingDelay: Option[Long] = processingEndTime.zip(processingStartTime) + .map(x => x._1 - x._2).headOption /** * Time taken for all the jobs of this batch to finish processing from the time they * were submitted. Essentially, it is `processingDelay` + `schedulingDelay`. */ - def totalDelay = schedulingDelay.zip(processingDelay).map(x => x._1 + x._2).headOption + def totalDelay: Option[Long] = schedulingDelay.zip(processingDelay) + .map(x => x._1 + x._2).headOption } From 1fa48d9422d543827011eec0cdf12d060b78a7c7 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Wed, 26 Mar 2014 18:31:52 -0700 Subject: [PATCH 145/397] SPARK-1325. The maven build error for Spark Tools This is just a slight variation on https://github.com/apache/spark/pull/234 and alternative suggestion for SPARK-1325. `scala-actors` is not necessary. `SparkBuild.scala` should be updated to reflect the direct dependency on `scala-reflect` and `scala-compiler`. And the `repl` build, which has the same dependencies, should also be consistent between Maven / SBT. Author: Sean Owen Author: witgo Closes #240 from srowen/SPARK-1325 and squashes the following commits: 25bd7db [Sean Owen] Add necessary dependencies scala-reflect and scala-compiler to tools. Update repl dependencies, which are similar, to be consistent between Maven / SBT in this regard too. --- pom.xml | 5 +++++ project/SparkBuild.scala | 4 +++- repl/pom.xml | 5 +++++ tools/pom.xml | 8 ++++++++ 4 files changed, 21 insertions(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index deb89b18ada73..f0644ed284363 100644 --- a/pom.xml +++ b/pom.xml @@ -419,6 +419,11 @@ scala-compiler ${scala.version} + + org.scala-lang + scala-reflect + ${scala.version} + org.scala-lang jline diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 60f14ba37e35c..9e269e6551341 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -356,7 +356,9 @@ object SparkBuild extends Build { ) ++ assemblySettings ++ extraAssemblySettings def toolsSettings = sharedSettings ++ Seq( - name := "spark-tools" + name := "spark-tools", + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ), + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v ) ) ++ assemblySettings ++ extraAssemblySettings def graphxSettings = sharedSettings ++ Seq( diff --git a/repl/pom.xml b/repl/pom.xml index fc49c8b811316..78d2fe13c27eb 100644 --- a/repl/pom.xml +++ b/repl/pom.xml @@ -77,6 +77,11 @@ scala-compiler ${scala.version} + + org.scala-lang + scala-reflect + ${scala.version} + org.scala-lang jline diff --git a/tools/pom.xml b/tools/pom.xml index 11433e596f5b0..ae2ba64e07c21 100644 --- a/tools/pom.xml +++ b/tools/pom.xml @@ -55,6 +55,14 @@ spark-streaming_${scala.binary.version} ${project.version} + + org.scala-lang + scala-reflect + + + org.scala-lang + scala-compiler + org.scalatest scalatest_${scala.binary.version} From d679843a39bb4918a08a5aebdf113ac8886a5275 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 26 Mar 2014 19:30:20 -0700 Subject: [PATCH 146/397] [SPARK-1327] GLM needs to check addIntercept for intercept and weights GLM needs to check addIntercept for intercept and weights. The current implementation always uses the first weight as intercept. Added a test for training without adding intercept. JIRA: https://spark-project.atlassian.net/browse/SPARK-1327 Author: Xiangrui Meng Closes #236 from mengxr/glm and squashes the following commits: bcac1ac [Xiangrui Meng] add two tests to ensure {Lasso, Ridge}.setIntercept will throw an exceptions a104072 [Xiangrui Meng] remove protected to be compatible with 0.9 0e57aa4 [Xiangrui Meng] update Lasso and RidgeRegression to parse the weights correctly from GLM mark createModel protected mark predictPoint protected d7f629f [Xiangrui Meng] fix a bug in GLM when intercept is not used --- .../GeneralizedLinearAlgorithm.scala | 21 ++++++++------- .../apache/spark/mllib/regression/Lasso.scala | 20 +++++++++----- .../mllib/regression/LinearRegression.scala | 20 +++++++------- .../mllib/regression/RidgeRegression.scala | 18 +++++++++---- .../spark/mllib/regression/LassoSuite.scala | 9 ++++--- .../regression/LinearRegressionSuite.scala | 26 ++++++++++++++++++- .../regression/RidgeRegressionSuite.scala | 9 ++++--- 7 files changed, 86 insertions(+), 37 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index b9621530efa22..3e1ed91bf6729 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -136,25 +136,28 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, labeledPoint.features.+:(1.0))) + input.map(labeledPoint => (labeledPoint.label, 1.0 +: labeledPoint.features)) } else { input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) } val initialWeightsWithIntercept = if (addIntercept) { - initialWeights.+:(1.0) + 0.0 +: initialWeights } else { initialWeights } - val weights = optimizer.optimize(data, initialWeightsWithIntercept) - val intercept = weights(0) - val weightsScaled = weights.tail + val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) - val model = createModel(weightsScaled, intercept) + val (intercept, weights) = if (addIntercept) { + (weightsWithIntercept(0), weightsWithIntercept.tail) + } else { + (0.0, weightsWithIntercept) + } + + logInfo("Final weights " + weights.mkString(",")) + logInfo("Final intercept " + intercept) - logInfo("Final model weights " + model.weights.mkString(",")) - logInfo("Final model intercept " + model.intercept) - model + createModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index fb2bc9b92a51c..be63ce8538fef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -36,8 +36,10 @@ class LassoModel( extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { + override def predictPoint( + dataMatrix: DoubleMatrix, + weightMatrix: DoubleMatrix, + intercept: Double): Double = { dataMatrix.dot(weightMatrix) + intercept } } @@ -66,7 +68,7 @@ class LassoWithSGD private ( .setMiniBatchFraction(miniBatchFraction) // We don't want to penalize the intercept, so set this to false. - setIntercept(false) + super.setIntercept(false) var yMean = 0.0 var xColMean: DoubleMatrix = _ @@ -77,10 +79,16 @@ class LassoWithSGD private ( */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*) + override def setIntercept(addIntercept: Boolean): this.type = { + // TODO: Support adding intercept. + if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") + this + } + + override def createModel(weights: Array[Double], intercept: Double) = { + val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) val weightsScaled = weightsMat.div(xColSd) - val interceptScaled = yMean - (weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0)) + val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) new LassoModel(weightsScaled.data, interceptScaled) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 8ee40addb25d9..f5f15d1a33f4d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -31,13 +31,14 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class LinearRegressionModel( - override val weights: Array[Double], - override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) - with RegressionModel with Serializable { - - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { + override val weights: Array[Double], + override val intercept: Double) + extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { + + override def predictPoint( + dataMatrix: DoubleMatrix, + weightMatrix: DoubleMatrix, + intercept: Double): Double = { dataMatrix.dot(weightMatrix) + intercept } } @@ -55,8 +56,7 @@ class LinearRegressionWithSGD private ( var stepSize: Double, var numIterations: Int, var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[LinearRegressionModel] - with Serializable { + extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable { val gradient = new LeastSquaresGradient() val updater = new SimpleUpdater() @@ -69,7 +69,7 @@ class LinearRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { + override def createModel(weights: Array[Double], intercept: Double) = { new LinearRegressionModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index c504d3d40c773..feb100f21888f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -36,8 +36,10 @@ class RidgeRegressionModel( extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double) = { + override def predictPoint( + dataMatrix: DoubleMatrix, + weightMatrix: DoubleMatrix, + intercept: Double): Double = { dataMatrix.dot(weightMatrix) + intercept } } @@ -67,7 +69,7 @@ class RidgeRegressionWithSGD private ( .setMiniBatchFraction(miniBatchFraction) // We don't want to penalize the intercept in RidgeRegression, so set this to false. - setIntercept(false) + super.setIntercept(false) var yMean = 0.0 var xColMean: DoubleMatrix = _ @@ -78,8 +80,14 @@ class RidgeRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length + 1, 1, (Array(intercept) ++ weights):_*) + override def setIntercept(addIntercept: Boolean): this.type = { + // TODO: Support adding intercept. + if (addIntercept) throw new UnsupportedOperationException("Adding intercept is not supported.") + this + } + + override def createModel(weights: Array[Double], intercept: Double) = { + val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) val weightsScaled = weightsMat.div(xColSd) val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index 64e4cbb860f61..2cebac943e15f 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -17,11 +17,8 @@ package org.apache.spark.mllib.regression - -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite -import org.apache.spark.SparkContext import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} class LassoSuite extends FunSuite with LocalSparkContext { @@ -104,4 +101,10 @@ class LassoSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + test("do not support intercept") { + intercept[UnsupportedOperationException] { + new LassoWithSGD().setIntercept(true) + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 281f9df36ddb3..5d251bcbf35db 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -17,7 +17,6 @@ package org.apache.spark.mllib.regression -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} @@ -57,4 +56,29 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + // Test if we can correctly learn Y = 10*X1 + 10*X2 + test("linear regression without intercept") { + val testRDD = sc.parallelize(LinearDataGenerator.generateLinearInput( + 0.0, Array(10.0, 10.0), 100, 42), 2).cache() + val linReg = new LinearRegressionWithSGD().setIntercept(false) + linReg.optimizer.setNumIterations(1000).setStepSize(1.0) + + val model = linReg.run(testRDD) + + assert(model.intercept === 0.0) + assert(model.weights.length === 2) + assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) + assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + + val validationData = LinearDataGenerator.generateLinearInput( + 0.0, Array(10.0, 10.0), 100, 17) + val validationRDD = sc.parallelize(validationData, 2).cache() + + // Test prediction on RDD. + validatePrediction(model.predict(validationRDD.map(_.features)).collect(), validationData) + + // Test prediction on Array. + validatePrediction(validationData.map(row => model.predict(row.features)), validationData) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index 67dd06cc0f5eb..b2044ed0d8066 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -17,14 +17,11 @@ package org.apache.spark.mllib.regression - import org.jblas.DoubleMatrix -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} - class RidgeRegressionSuite extends FunSuite with LocalSparkContext { def predictionError(predictions: Seq[Double], input: Seq[LabeledPoint]) = { @@ -74,4 +71,10 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { assert(ridgeErr < linearErr, "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } + + test("do not support intercept") { + intercept[UnsupportedOperationException] { + new RidgeRegressionWithSGD().setIntercept(true) + } + } } From 5b2d863e339b3955eafc0588212bd5231a9163a0 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 27 Mar 2014 08:53:42 -0700 Subject: [PATCH 147/397] Cut down the granularity of travis tests. This PR amortizes the cost of downloading all the jars and compiling core across more test cases. In one anecdotal run this change takes the cumulative time down from ~80 minutes to ~40 minutes. Author: Michael Armbrust Closes #255 from marmbrus/travis and squashes the following commits: 506b22d [Michael Armbrust] Cut down the granularity of travis tests so we can amortize the cost of compilation. --- .travis.yml | 11 +++-------- 1 file changed, 3 insertions(+), 8 deletions(-) diff --git a/.travis.yml b/.travis.yml index 617da2dc308cc..8ebd0d68429fc 100644 --- a/.travis.yml +++ b/.travis.yml @@ -20,18 +20,13 @@ - oraclejdk7 env: matrix: - - TEST=sql/test + - TEST="scalastyle assembly/assembly" + - TEST="catalyst/test sql/test streaming/test mllib/test graphx/test bagel/test" - TEST=hive/test - - TEST=catalyst/test - - TEST=streaming/test - - TEST=graphx/test - - TEST=mllib/test - - TEST=graphx/test - - TEST=bagel/test cache: directories: - $HOME/.m2 - $HOME/.ivy2 - $HOME/.sbt script: - - "sbt ++$TRAVIS_SCALA_VERSION scalastyle $TEST" + - "sbt ++$TRAVIS_SCALA_VERSION $TEST" From 426042ad24a54b4b776085cbf4e1896464efc613 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 27 Mar 2014 11:54:43 -0500 Subject: [PATCH 148/397] SPARK-1330 removed extra echo from comput_classpath.sh remove the extra echo which prevents spark-class from working. Note that I did not update the comment above it, which is also wrong because I'm not sure what it should do. Should hive only be included if explicitly built with sbt hive/assembly or should sbt assembly build it? Author: Thomas Graves Closes #241 from tgravescs/SPARK-1330 and squashes the following commits: b10d708 [Thomas Graves] SPARK-1330 removed extra echo from comput_classpath.sh --- bin/compute-classpath.sh | 1 - 1 file changed, 1 deletion(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index d6f1ff9084c6f..bef42df71ce01 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -36,7 +36,6 @@ CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" # Hopefully we will find a way to avoid uber-jars entirely 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[*]}") From 53953d0933c0a7c3bd3bc1003954426363912e4b Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 27 Mar 2014 11:49:11 -0700 Subject: [PATCH 149/397] SPARK-1335. Also increase perm gen / code cache for scalatest when invoked via Maven build I am observing build failures when the Maven build reaches tests in the new SQL components. (I'm on Java 7 / OSX 10.9). The failure is the usual complaint from scala, that it's out of permgen space, or that JIT out of code cache space. I see that various build scripts increase these both for SBT. This change simply adds these settings to scalatest's arguments. Works for me and seems a bit more consistent. (I also snuck in cures for new build warnings from new scaladoc. Felt too trivial for a new PR, although it's separate. Just something I also saw while examining the build output.) Author: Sean Owen Closes #253 from srowen/SPARK-1335 and squashes the following commits: c0f2d31 [Sean Owen] Appease scalastyle with a newline at the end of the file a02679c [Sean Owen] Fix scaladoc errors due to missing links, which are generating build warnings, from some recent doc changes. We apparently can't generate links outside the module. b2c6a09 [Sean Owen] Add perm gen, code cache settings to scalatest, mirroring SBT settings elsewhere, which allows tests to complete in at least one environment where they are failing. (Also removed a duplicate -Xms setting elsewhere.) --- pom.xml | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/pom.xml b/pom.xml index f0644ed284363..72acf2b402703 100644 --- a/pom.xml +++ b/pom.xml @@ -646,7 +646,6 @@ -deprecation - -Xms64m -Xms1024m -Xmx1024m -XX:PermSize=${PermGen} @@ -689,7 +688,7 @@ ${project.build.directory}/surefire-reports . ${project.build.directory}/SparkTestSuite.txt - -Xms64m -Xmx3g + -Xmx3g -XX:MaxPermSize=${MaxPermGen} -XX:ReservedCodeCacheSize=512m From 6f986f0b87bd03f4df2bf6c917e61241e9b14ac2 Mon Sep 17 00:00:00 2001 From: Petko Nikolov Date: Thu, 27 Mar 2014 15:49:07 -0700 Subject: [PATCH 150/397] [SPARK-1268] Adding XOR and AND-NOT operations to spark.util.collection.BitSet Symmetric difference (xor) in particular is useful for computing some distance metrics (e.g. Hamming). Unit tests added. Author: Petko Nikolov Closes #172 from petko-nikolov/bitset-imprv and squashes the following commits: 451f28b [Petko Nikolov] fixed style mistakes 5beba18 [Petko Nikolov] rm outer loop in andNot test 0e61035 [Petko Nikolov] conform to spark style; rm redundant asserts; more unit tests added; use arraycopy instead of loop d53cdb9 [Petko Nikolov] rm incidentally added space 4e1df43 [Petko Nikolov] adding xor and and-not to BitSet; unit tests added --- .../apache/spark/util/collection/BitSet.scala | 39 +++++++++ .../spark/util/collection/BitSetSuite.scala | 83 +++++++++++++++++++ 2 files changed, 122 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala index d3153d2cac4a5..af1f64649f354 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/BitSet.scala @@ -88,6 +88,45 @@ class BitSet(numBits: Int) extends Serializable { newBS } + /** + * Compute the symmetric difference by performing bit-wise XOR of the two sets returning the + * result. + */ + def ^(other: BitSet): BitSet = { + val newBS = new BitSet(math.max(capacity, other.capacity)) + val smaller = math.min(numWords, other.numWords) + var ind = 0 + while (ind < smaller) { + newBS.words(ind) = words(ind) ^ other.words(ind) + ind += 1 + } + if (ind < numWords) { + Array.copy( words, ind, newBS.words, ind, numWords - ind ) + } + if (ind < other.numWords) { + Array.copy( other.words, ind, newBS.words, ind, other.numWords - ind ) + } + newBS + } + + /** + * Compute the difference of the two sets by performing bit-wise AND-NOT returning the + * result. + */ + def andNot(other: BitSet): BitSet = { + val newBS = new BitSet(capacity) + val smaller = math.min(numWords, other.numWords) + var ind = 0 + while (ind < smaller) { + newBS.words(ind) = words(ind) & ~other.words(ind) + ind += 1 + } + if (ind < numWords) { + Array.copy( words, ind, newBS.words, ind, numWords - ind ) + } + newBS + } + /** * Sets the bit at the specified index to true. * @param index the bit index diff --git a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala index c32183c134f9c..b85a409a4b2e9 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/BitSetSuite.scala @@ -69,4 +69,87 @@ class BitSetSuite extends FunSuite { assert(bitset.nextSetBit(96) === 96) assert(bitset.nextSetBit(97) === -1) } + + test( "xor len(bitsetX) < len(bitsetY)" ) { + val setBitsX = Seq( 0, 2, 3, 37, 41 ) + val setBitsY = Seq( 0, 1, 3, 37, 38, 41, 85) + val bitsetX = new BitSet(60) + setBitsX.foreach( i => bitsetX.set(i)) + val bitsetY = new BitSet(100) + setBitsY.foreach( i => bitsetY.set(i)) + + val bitsetXor = bitsetX ^ bitsetY + + assert(bitsetXor.nextSetBit(0) === 1) + assert(bitsetXor.nextSetBit(1) === 1) + assert(bitsetXor.nextSetBit(2) === 2) + assert(bitsetXor.nextSetBit(3) === 38) + assert(bitsetXor.nextSetBit(38) === 38) + assert(bitsetXor.nextSetBit(39) === 85) + assert(bitsetXor.nextSetBit(42) === 85) + assert(bitsetXor.nextSetBit(85) === 85) + assert(bitsetXor.nextSetBit(86) === -1) + + } + + test( "xor len(bitsetX) > len(bitsetY)" ) { + val setBitsX = Seq( 0, 1, 3, 37, 38, 41, 85) + val setBitsY = Seq( 0, 2, 3, 37, 41 ) + val bitsetX = new BitSet(100) + setBitsX.foreach( i => bitsetX.set(i)) + val bitsetY = new BitSet(60) + setBitsY.foreach( i => bitsetY.set(i)) + + val bitsetXor = bitsetX ^ bitsetY + + assert(bitsetXor.nextSetBit(0) === 1) + assert(bitsetXor.nextSetBit(1) === 1) + assert(bitsetXor.nextSetBit(2) === 2) + assert(bitsetXor.nextSetBit(3) === 38) + assert(bitsetXor.nextSetBit(38) === 38) + assert(bitsetXor.nextSetBit(39) === 85) + assert(bitsetXor.nextSetBit(42) === 85) + assert(bitsetXor.nextSetBit(85) === 85) + assert(bitsetXor.nextSetBit(86) === -1) + + } + + test( "andNot len(bitsetX) < len(bitsetY)" ) { + val setBitsX = Seq( 0, 2, 3, 37, 41, 48 ) + val setBitsY = Seq( 0, 1, 3, 37, 38, 41, 85) + val bitsetX = new BitSet(60) + setBitsX.foreach( i => bitsetX.set(i)) + val bitsetY = new BitSet(100) + setBitsY.foreach( i => bitsetY.set(i)) + + val bitsetDiff = bitsetX.andNot( bitsetY ) + + assert(bitsetDiff.nextSetBit(0) === 2) + assert(bitsetDiff.nextSetBit(1) === 2) + assert(bitsetDiff.nextSetBit(2) === 2) + assert(bitsetDiff.nextSetBit(3) === 48) + assert(bitsetDiff.nextSetBit(48) === 48) + assert(bitsetDiff.nextSetBit(49) === -1) + assert(bitsetDiff.nextSetBit(65) === -1) + } + + test( "andNot len(bitsetX) > len(bitsetY)" ) { + val setBitsX = Seq( 0, 1, 3, 37, 38, 41, 85) + val setBitsY = Seq( 0, 2, 3, 37, 41, 48 ) + val bitsetX = new BitSet(100) + setBitsX.foreach( i => bitsetX.set(i)) + val bitsetY = new BitSet(60) + setBitsY.foreach( i => bitsetY.set(i)) + + val bitsetDiff = bitsetX.andNot( bitsetY ) + + assert(bitsetDiff.nextSetBit(0) === 1) + assert(bitsetDiff.nextSetBit(1) === 1) + assert(bitsetDiff.nextSetBit(2) === 38) + assert(bitsetDiff.nextSetBit(3) === 38) + assert(bitsetDiff.nextSetBit(38) === 38) + assert(bitsetDiff.nextSetBit(39) === 85) + assert(bitsetDiff.nextSetBit(85) === 85) + assert(bitsetDiff.nextSetBit(86) === -1) + } } From 3d89043b7ed13bc1bb703f6eb7c00e46b936de1e Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 27 Mar 2014 22:17:15 -0700 Subject: [PATCH 151/397] [SPARK-1210] Prevent ContextClassLoader of Actor from becoming ClassLoader of Executo... ...r. Constructor of `org.apache.spark.executor.Executor` should not set context class loader of current thread, which is backend Actor's thread. Run the following code in local-mode REPL. ``` scala> case class Foo(i: Int) scala> val ret = sc.parallelize((1 to 100).map(Foo), 10).collect ``` This causes errors as follows: ``` ERROR actor.OneForOneStrategy: [L$line5.$read$$iwC$$iwC$$iwC$$iwC$Foo; java.lang.ArrayStoreException: [L$line5.$read$$iwC$$iwC$$iwC$$iwC$Foo; at scala.runtime.ScalaRunTime$.array_update(ScalaRunTime.scala:88) at org.apache.spark.SparkContext$$anonfun$runJob$3.apply(SparkContext.scala:870) at org.apache.spark.SparkContext$$anonfun$runJob$3.apply(SparkContext.scala:870) at org.apache.spark.scheduler.JobWaiter.taskSucceeded(JobWaiter.scala:56) at org.apache.spark.scheduler.DAGScheduler.handleTaskCompletion(DAGScheduler.scala:859) at org.apache.spark.scheduler.DAGScheduler.processEvent(DAGScheduler.scala:616) at org.apache.spark.scheduler.DAGScheduler$$anonfun$start$1$$anon$2$$anonfun$receive$1.applyOrElse(DAGScheduler.scala:207) at akka.actor.ActorCell.receiveMessage(ActorCell.scala:498) at akka.actor.ActorCell.invoke(ActorCell.scala:456) at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:237) at akka.dispatch.Mailbox.run(Mailbox.scala:219) at akka.dispatch.ForkJoinExecutorConfigurator$AkkaForkJoinTask.exec(AbstractDispatcher.scala:386) at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260) at scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339) at scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979) at scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107) ``` This is because the class loaders to deserialize result `Foo` instances might be different from backend Actor's, and the Actor's class loader should be the same as Driver's. Author: Takuya UESHIN Closes #15 from ueshin/wip/wrongcontextclassloader and squashes the following commits: d79e8c0 [Takuya UESHIN] Change a parent class loader of ExecutorURLClassLoader. c6c09b6 [Takuya UESHIN] Add a test to collect objects of class defined in repl. 43e0feb [Takuya UESHIN] Prevent ContextClassLoader of Actor from becoming ClassLoader of Executor. --- .../scala/org/apache/spark/executor/Executor.scala | 5 ++--- .../test/scala/org/apache/spark/repl/ReplSuite.scala | 11 +++++++++++ 2 files changed, 13 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 8fe9b848ba145..13e2e292428b4 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -112,11 +112,10 @@ private[spark] class Executor( } } - // Create our ClassLoader and set it on this thread + // Create our ClassLoader // do this after SparkEnv creation so can access the SecurityManager private val urlClassLoader = createClassLoader() private val replClassLoader = addReplClassLoaderIfNeeded(urlClassLoader) - Thread.currentThread.setContextClassLoader(replClassLoader) // Akka's message frame size. If task result is bigger than this, we use the block manager // to send the result back. @@ -294,7 +293,7 @@ private[spark] class Executor( * created by the interpreter to the search path */ private def createClassLoader(): ExecutorURLClassLoader = { - val loader = this.getClass.getClassLoader + val loader = Thread.currentThread().getContextClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 8203b8f6122e1..4155007c6d337 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -242,4 +242,15 @@ class ReplSuite extends FunSuite { assertContains("res4: Array[Int] = Array(0, 0, 0, 0, 0)", output) } } + + test("collecting objects of class defined in repl") { + val output = runInterpreter("local[2]", + """ + |case class Foo(i: Int) + |val ret = sc.parallelize((1 to 100).map(Foo), 10).collect + """.stripMargin) + assertDoesNotContain("error:", output) + assertDoesNotContain("Exception", output) + assertContains("ret: Array[Foo] = Array(Foo(1),", output) + } } From 632c322036b123c6f72e0c8b87d50e08bec3a1ab Mon Sep 17 00:00:00 2001 From: Nick Lanham Date: Thu, 27 Mar 2014 22:45:00 -0700 Subject: [PATCH 152/397] Make sed do -i '' on OSX I don't have access to an OSX machine, so if someone could test this that would be great. Author: Nick Lanham Closes #258 from nicklan/osx-sed-fix and squashes the following commits: a6f158f [Nick Lanham] Also make mktemp work on OSX 558fd6e [Nick Lanham] Make sed do -i '' on OSX --- make-distribution.sh | 11 +++++++++-- 1 file changed, 9 insertions(+), 2 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index 6bc6819d8da92..b07aef2ef49c2 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -128,7 +128,7 @@ if [ "$SPARK_TACHYON" == "true" ]; then TACHYON_VERSION="0.4.1" TACHYON_URL="https://github.com/amplab/tachyon/releases/download/v${TACHYON_VERSION}/tachyon-${TACHYON_VERSION}-bin.tar.gz" - TMPD=`mktemp -d` + TMPD=`mktemp -d 2>/dev/null || mktemp -d -t 'disttmp'` pushd $TMPD > /dev/null echo "Fetchting tachyon tgz" @@ -139,7 +139,14 @@ if [ "$SPARK_TACHYON" == "true" ]; then mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web" cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon" cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" - sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" + + if [[ `uname -a` == Darwin* ]]; then + # osx sed wants an empty argument to -i option of sed + TACHYON_SED="sed -i ''" + else + TACHYON_SED="sed -i" + fi + $TACHYON_SED -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" popd > /dev/null rm -rf $TMPD From 60abc252545ec7a5d59957a32e764cd18f6c16b4 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 28 Mar 2014 00:21:49 -0700 Subject: [PATCH 153/397] SPARK-1096, a space after comment start style checker. Author: Prashant Sharma Closes #124 from ScrapCodes/SPARK-1096/scalastyle-comment-check and squashes the following commits: 214135a [Prashant Sharma] Review feedback. 5eba88c [Prashant Sharma] Fixed style checks for ///+ comments. e54b2f8 [Prashant Sharma] improved message, work around. 83e7144 [Prashant Sharma] removed dependency on scalastyle in plugin, since scalastyle sbt plugin already depends on the right version. Incase we update the plugin we will have to adjust our spark-style project to depend on right scalastyle version. 810a1d6 [Prashant Sharma] SPARK-1096, a space after comment style checker. ba33193 [Prashant Sharma] scala style as a project --- .../scala/org/apache/spark/SparkEnv.scala | 2 +- .../spark/broadcast/TorrentBroadcast.scala | 2 +- .../spark/deploy/LocalSparkCluster.scala | 4 +- .../deploy/master/LeaderElectionAgent.scala | 2 +- .../org/apache/spark/executor/Executor.scala | 1 - .../apache/spark/metrics/MetricsConfig.scala | 2 +- .../org/apache/spark/network/Connection.scala | 18 +++--- .../spark/network/ConnectionManager.scala | 18 +++--- .../spark/network/ConnectionManagerTest.scala | 4 +- .../apache/spark/network/ReceiverTest.scala | 2 +- .../org/apache/spark/network/SenderTest.scala | 2 +- .../spark/network/netty/FileHeader.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 2 +- .../CoarseGrainedSchedulerBackend.scala | 2 +- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../apache/spark/util/ClosureCleaner.scala | 10 ++-- .../util/IndestructibleActorSystem.scala | 2 +- .../org/apache/spark/util/MutablePair.scala | 4 +- .../org/apache/spark/AccumulatorSuite.scala | 6 +- .../org/apache/spark/CheckpointSuite.scala | 1 - .../org/apache/spark/PartitioningSuite.scala | 2 +- .../spark/scheduler/SparkListenerSuite.scala | 2 +- .../org/apache/spark/util/UtilsSuite.scala | 2 +- .../org/apache/spark/examples/LocalALS.scala | 1 - .../examples/SimpleSkewedGroupByTest.scala | 2 +- .../org/apache/spark/examples/SparkALS.scala | 1 - .../apache/spark/examples/SparkHdfsLR.scala | 2 - .../streaming/examples/ActorWordCount.scala | 2 +- .../streaming/examples/ZeroMQWordCount.scala | 2 +- .../streaming/zeromq/ZeroMQReceiver.scala | 2 +- .../org/apache/spark/graphx/EdgeTriplet.scala | 4 +- .../apache/spark/graphx/impl/GraphImpl.scala | 4 +- .../graphx/impl/MessageToPartition.scala | 2 +- .../spark/graphx/impl/Serializers.scala | 2 - .../spark/graphx/util/BytecodeUtils.scala | 2 +- .../spark/graphx/util/GraphGenerators.scala | 2 +- project/SparkBuild.scala | 6 +- project/plugins.sbt | 1 + project/project/SparkPluginBuild.scala | 44 +++++++++++++++ .../SparkSpaceAfterCommentStyleCheck.scala | 56 +++++++++++++++++++ .../spark/repl/ExecutorClassLoader.scala | 4 +- .../org/apache/spark/repl/SparkIMain.scala | 2 +- scalastyle-config.xml | 1 + .../org/apache/spark/sql/hive/HiveQl.scala | 2 +- .../apache/spark/sql/hive/TableReader.scala | 2 +- .../apache/spark/streaming/DStreamGraph.scala | 2 +- .../dstream/NetworkInputDStream.scala | 3 +- .../dstream/ReducedWindowedDStream.scala | 4 +- .../streaming/dstream/StateDStream.scala | 5 +- .../scheduler/NetworkInputTracker.scala | 2 +- .../spark/streaming/CheckpointSuite.scala | 2 +- .../spark/streaming/InputStreamsSuite.scala | 3 +- .../apache/spark/deploy/yarn/ClientBase.scala | 1 - .../yarn/ClientDistributedCacheManager.scala | 4 +- .../ClientDistributedCacheManagerSuite.scala | 2 +- 55 files changed, 180 insertions(+), 88 deletions(-) create mode 100644 project/project/SparkPluginBuild.scala create mode 100644 project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index a1af63fa4a391..5ceac28fe7afb 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -81,7 +81,7 @@ class SparkEnv private[spark] ( // Unfortunately Akka's awaitTermination doesn't actually wait for the Netty server to shut // down, but let's call it anyway in case it gets fixed in a later release // UPDATE: In Akka 2.1.x, this hangs if there are remote actors, so we can't call it. - //actorSystem.awaitTermination() + // actorSystem.awaitTermination() } private[spark] diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 3cd71213769b7..2595c15104e87 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -167,7 +167,7 @@ extends Logging { private var initialized = false private var conf: SparkConf = null def initialize(_isDriver: Boolean, conf: SparkConf) { - TorrentBroadcast.conf = conf //TODO: we might have to fix it in tests + TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests synchronized { if (!initialized) { initialized = true diff --git a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala index a73b459c3cea1..9a7a113c95715 100644 --- a/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala +++ b/core/src/main/scala/org/apache/spark/deploy/LocalSparkCluster.scala @@ -66,9 +66,9 @@ class LocalSparkCluster(numWorkers: Int, coresPerWorker: Int, memoryPerWorker: I // TODO: In Akka 2.1.x, ActorSystem.awaitTermination hangs when you have remote actors! // This is unfortunate, but for now we just comment it out. workerActorSystems.foreach(_.shutdown()) - //workerActorSystems.foreach(_.awaitTermination()) + // workerActorSystems.foreach(_.awaitTermination()) masterActorSystems.foreach(_.shutdown()) - //masterActorSystems.foreach(_.awaitTermination()) + // masterActorSystems.foreach(_.awaitTermination()) masterActorSystems.clear() workerActorSystems.clear() } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala index a730fe1f599af..4433a2ec29be6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/LeaderElectionAgent.scala @@ -30,7 +30,7 @@ import org.apache.spark.deploy.master.MasterMessages.ElectedLeader * [[org.apache.spark.deploy.master.MasterMessages.RevokedLeadership RevokedLeadership]] */ private[spark] trait LeaderElectionAgent extends Actor { - //TODO: LeaderElectionAgent does not necessary to be an Actor anymore, need refactoring. + // TODO: LeaderElectionAgent does not necessary to be an Actor anymore, need refactoring. val masterActor: ActorRef } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 13e2e292428b4..aecb069e4202b 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -275,7 +275,6 @@ private[spark] class Executor( // have left some weird state around depending on when the exception was thrown, but on // the other hand, maybe we could detect that when future tasks fail and exit then. logError("Exception in task ID " + taskId, t) - //System.exit(1) } } finally { // TODO: Unregister shuffle memory only for ResultTask diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index 6883a54494598..3e3e18c3537d0 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -42,7 +42,7 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi } def initialize() { - //Add default properties in case there's no properties file + // Add default properties in case there's no properties file setDefaultProperties(properties) // If spark.metrics.conf is not set, try to get file in class path diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 8fd9c2b87d256..16bd00fd189ff 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -48,7 +48,7 @@ abstract class Connection(val channel: SocketChannel, val selector: Selector, channel.socket.setTcpNoDelay(true) channel.socket.setReuseAddress(true) channel.socket.setKeepAlive(true) - /*channel.socket.setReceiveBufferSize(32768) */ + /* channel.socket.setReceiveBufferSize(32768) */ @volatile private var closed = false var onCloseCallback: Connection => Unit = null @@ -206,12 +206,12 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, private class Outbox { val messages = new Queue[Message]() - val defaultChunkSize = 65536 //32768 //16384 + val defaultChunkSize = 65536 var nextMessageToBeUsed = 0 def addMessage(message: Message) { messages.synchronized{ - /*messages += message*/ + /* messages += message*/ messages.enqueue(message) logDebug("Added [" + message + "] to outbox for sending to " + "[" + getRemoteConnectionManagerId() + "]") @@ -221,8 +221,8 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, def getChunk(): Option[MessageChunk] = { messages.synchronized { while (!messages.isEmpty) { - /*nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ - /*val message = messages(nextMessageToBeUsed)*/ + /* nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ + /* val message = messages(nextMessageToBeUsed)*/ val message = messages.dequeue val chunk = message.getChunkForSending(defaultChunkSize) if (chunk.isDefined) { @@ -262,7 +262,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, val currentBuffers = new ArrayBuffer[ByteBuffer]() - /*channel.socket.setSendBufferSize(256 * 1024)*/ + /* channel.socket.setSendBufferSize(256 * 1024)*/ override def getRemoteAddress() = address @@ -355,7 +355,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } case None => { // changeConnectionKeyInterest(0) - /*key.interestOps(0)*/ + /* key.interestOps(0)*/ return false } } @@ -540,10 +540,10 @@ private[spark] class ReceivingConnection( return false } - /*logDebug("Read " + bytesRead + " bytes for the buffer")*/ + /* logDebug("Read " + bytesRead + " bytes for the buffer")*/ if (currentChunk.buffer.remaining == 0) { - /*println("Filled buffer at " + System.currentTimeMillis)*/ + /* println("Filled buffer at " + System.currentTimeMillis)*/ val bufferMessage = inbox.getMessageForChunk(currentChunk).get if (bufferMessage.isCompletelyReceived) { bufferMessage.flip diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index a75130cba2a2e..2682f9d0ed7f0 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -505,7 +505,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } handleMessageExecutor.execute(runnable) - /*handleMessage(connection, message)*/ + /* handleMessage(connection, message)*/ } private def handleClientAuthentication( @@ -733,7 +733,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, logTrace("Sending Security [" + message + "] to [" + connManagerId + "]") val connection = connectionsById.getOrElseUpdate(connManagerId, startNewConnection()) - //send security message until going connection has been authenticated + // send security message until going connection has been authenticated connection.send(message) wakeupSelector() @@ -859,14 +859,14 @@ private[spark] object ConnectionManager { None }) - /*testSequentialSending(manager)*/ - /*System.gc()*/ + /* testSequentialSending(manager)*/ + /* System.gc()*/ - /*testParallelSending(manager)*/ - /*System.gc()*/ + /* testParallelSending(manager)*/ + /* System.gc()*/ - /*testParallelDecreasingSending(manager)*/ - /*System.gc()*/ + /* testParallelDecreasingSending(manager)*/ + /* System.gc()*/ testContinuousSending(manager) System.gc() @@ -948,7 +948,7 @@ private[spark] object ConnectionManager { val ms = finishTime - startTime val tput = mb * 1000.0 / ms println("--------------------------") - /*println("Started at " + startTime + ", finished at " + finishTime) */ + /* println("Started at " + startTime + ", finished at " + finishTime) */ println("Sent " + mb + " MB in " + ms + " ms (" + tput + " MB/s)") println("--------------------------") println() diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index 35f64134b073a..e5745d7daa153 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -47,8 +47,8 @@ private[spark] object ConnectionManagerTest extends Logging{ val slaves = slavesFile.mkString.split("\n") slavesFile.close() - /*println("Slaves")*/ - /*slaves.foreach(println)*/ + /* println("Slaves")*/ + /* slaves.foreach(println)*/ val tasknum = if (args.length > 2) args(2).toInt else slaves.length val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 val count = if (args.length > 4) args(4).toInt else 3 diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 3c09a713c6fe0..17fd931c9f075 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -27,7 +27,7 @@ private[spark] object ReceiverTest { println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - /*println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/ + /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/ val buffer = ByteBuffer.wrap("response".getBytes) Some(Message.createBufferMessage(buffer, msg.id)) }) diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index aac2c24a46faa..905eddfbb9450 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -50,7 +50,7 @@ private[spark] object SenderTest { (0 until count).foreach(i => { val dataMessage = Message.createBufferMessage(buffer.duplicate) val startTime = System.currentTimeMillis - /*println("Started timer at " + startTime)*/ + /* println("Started timer at " + startTime)*/ val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index f9082ffb9141a..4164e81d3a8ae 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -32,7 +32,7 @@ private[spark] class FileHeader ( buf.writeInt(fileLen) buf.writeInt(blockId.name.length) blockId.name.foreach((x: Char) => buf.writeByte(x)) - //padding the rest of header + // padding the rest of header if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) } else { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 77c558ac46f6f..4fce47e1ee8de 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -753,7 +753,7 @@ class DAGScheduler( val properties = if (stageIdToActiveJob.contains(jobId)) { stageIdToActiveJob(stage.jobId).properties } else { - //this stage will be assigned to "default" pool + // this stage will be assigned to "default" pool null } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 990e01a3e7959..7bfc30b4208a3 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -172,7 +172,7 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A properties += ((key, value)) } } - //TODO (prashant) send conf instead of properties + // TODO (prashant) send conf instead of properties driverActor = actorSystem.actorOf( Props(new DriverActor(properties)), name = CoarseGrainedSchedulerBackend.ACTOR_NAME) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index bcfc39146a61e..2fbbda5b76c74 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -284,7 +284,7 @@ object BlockFetcherIterator { } } catch { case x: InterruptedException => logInfo("Copier Interrupted") - //case _ => throw new SparkException("Exception Throw in Shuffle Copier") + // case _ => throw new SparkException("Exception Throw in Shuffle Copier") } } } diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index a8d20ee332355..cdbbc65292188 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -112,7 +112,7 @@ private[spark] object ClosureCleaner extends Logging { accessedFields(cls) = Set[String]() for (cls <- func.getClass :: innerClasses) getClassReader(cls).accept(new FieldAccessFinder(accessedFields), 0) - //logInfo("accessedFields: " + accessedFields) + // logInfo("accessedFields: " + accessedFields) val inInterpreter = { try { @@ -139,13 +139,13 @@ private[spark] object ClosureCleaner extends Logging { val field = cls.getDeclaredField(fieldName) field.setAccessible(true) val value = field.get(obj) - //logInfo("1: Setting " + fieldName + " on " + cls + " to " + value); + // logInfo("1: Setting " + fieldName + " on " + cls + " to " + value); field.set(outer, value) } } if (outer != null) { - //logInfo("2: Setting $outer on " + func.getClass + " to " + outer); + // logInfo("2: Setting $outer on " + func.getClass + " to " + outer); val field = func.getClass.getDeclaredField("$outer") field.setAccessible(true) field.set(func, outer) @@ -153,7 +153,7 @@ private[spark] object ClosureCleaner extends Logging { } private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { - //logInfo("Creating a " + cls + " with outer = " + outer) + // logInfo("Creating a " + cls + " with outer = " + outer) if (!inInterpreter) { // This is a bona fide closure class, whose constructor has no effects // other than to set its fields, so use its constructor @@ -170,7 +170,7 @@ private[spark] object ClosureCleaner extends Logging { val newCtor = rf.newConstructorForSerialization(cls, parentCtor) val obj = newCtor.newInstance().asInstanceOf[AnyRef] if (outer != null) { - //logInfo("3: Setting $outer on " + cls + " to " + outer); + // logInfo("3: Setting $outer on " + cls + " to " + outer); val field = cls.getDeclaredField("$outer") field.setAccessible(true) field.set(obj, outer) diff --git a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala index c539d2f708f95..4188a869c13da 100644 --- a/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala +++ b/core/src/main/scala/org/apache/spark/util/IndestructibleActorSystem.scala @@ -49,7 +49,7 @@ private[akka] class IndestructibleActorSystemImpl( if (isFatalError(cause) && !settings.JvmExitOnFatalError) { log.error(cause, "Uncaught fatal error from thread [{}] not shutting down " + "ActorSystem [{}] tolerating and continuing.... ", thread.getName, name) - //shutdown() //TODO make it configurable + // shutdown() //TODO make it configurable } else { fallbackHandler.uncaughtException(thread, cause) } 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 2c1a6f8fd0a44..a898824cff0ca 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -24,8 +24,8 @@ package org.apache.spark.util * @param _1 Element 1 of this MutablePair * @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] +case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T1, + @specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T2] (var _1: T1, var _2: T2) extends Product2[T1, T2] { diff --git a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala index 6c73ea6949dd2..4e7c34e6d1ada 100644 --- a/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala +++ b/core/src/test/scala/org/apache/spark/AccumulatorSuite.scala @@ -66,7 +66,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte test ("add value to collection accumulators") { val maxI = 1000 - for (nThreads <- List(1, 10)) { //test single & multi-threaded + for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) val d = sc.parallelize(1 to maxI) @@ -83,7 +83,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte test ("value not readable in tasks") { val maxI = 1000 - for (nThreads <- List(1, 10)) { //test single & multi-threaded + for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) val d = sc.parallelize(1 to maxI) @@ -124,7 +124,7 @@ class AccumulatorSuite extends FunSuite with ShouldMatchers with LocalSparkConte test ("localValue readable in tasks") { val maxI = 1000 - for (nThreads <- List(1, 10)) { //test single & multi-threaded + for (nThreads <- List(1, 10)) { // test single & multi-threaded sc = new SparkContext("local[" + nThreads + "]", "test") val acc: Accumulable[mutable.Set[Any], Any] = sc.accumulable(new mutable.HashSet[Any]()) val groupedInts = (1 to (maxI/20)).map {x => (20 * (x - 1) to 20 * x).toSet} diff --git a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala index d2e29f20f0b08..d2555b7c052c1 100644 --- a/core/src/test/scala/org/apache/spark/CheckpointSuite.scala +++ b/core/src/test/scala/org/apache/spark/CheckpointSuite.scala @@ -432,7 +432,6 @@ object CheckpointSuite { // This is a custom cogroup function that does not use mapValues like // the PairRDDFunctions.cogroup() def cogroup[K, V](first: RDD[(K, V)], second: RDD[(K, V)], part: Partitioner) = { - //println("First = " + first + ", second = " + second) new CoGroupedRDD[K]( Seq(first.asInstanceOf[RDD[(K, _)]], second.asInstanceOf[RDD[(K, _)]]), part diff --git a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala index 996db70809320..7c30626a0c421 100644 --- a/core/src/test/scala/org/apache/spark/PartitioningSuite.scala +++ b/core/src/test/scala/org/apache/spark/PartitioningSuite.scala @@ -146,7 +146,7 @@ class PartitioningSuite extends FunSuite with SharedSparkContext with PrivateMet assert(intercept[SparkException]{ arrs.distinct() }.getMessage.contains("array")) // We can't catch all usages of arrays, since they might occur inside other collections: - //assert(fails { arrPairs.distinct() }) + // assert(fails { arrPairs.distinct() }) assert(intercept[SparkException]{ arrPairs.partitionBy(new HashPartitioner(2)) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.join(arrPairs) }.getMessage.contains("array")) assert(intercept[SparkException]{ arrPairs.leftOuterJoin(arrPairs) }.getMessage.contains("array")) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index a25ce35736146..7c843772bc2e0 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -111,7 +111,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) - //just to make sure some of the tasks take a noticeable amount of time + // just to make sure some of the tasks take a noticeable amount of time val w = {i:Int => if (i == 0) Thread.sleep(100) diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index eb8f5915605de..616214fb5e3a6 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -39,7 +39,7 @@ class UtilsSuite extends FunSuite { } test("copyStream") { - //input array initialization + // input array initialization val bytes = Array.ofDim[Byte](9000) Random.nextBytes(bytes) diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index c8ecbb8e41a86..0095cb8425456 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -53,7 +53,6 @@ object LocalALS { for (i <- 0 until M; j <- 0 until U) { r.set(i, j, blas.ddot(ms(i), us(j))) } - //println("R: " + r) blas.daxpy(-1, targetR, r) val sumSqs = r.aggregate(Functions.plus, Functions.square) sqrt(sumSqs / (M * U)) diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 73b0e216cac98..1fdb324b89f3a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -61,7 +61,7 @@ object SimpleSkewedGroupByTest { println("RESULT: " + pairs1.groupByKey(numReducers).count) // Print how many keys each reducer got (for debugging) - //println("RESULT: " + pairs1.groupByKey(numReducers) + // println("RESULT: " + pairs1.groupByKey(numReducers) // .map{case (k,v) => (k, v.size)} // .collectAsMap) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index ce4b3c8451e00..f59ab7e7cc24a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -54,7 +54,6 @@ object SparkALS { for (i <- 0 until M; j <- 0 until U) { r.set(i, j, blas.ddot(ms(i), us(j))) } - //println("R: " + r) blas.daxpy(-1, targetR, r) val sumSqs = r.aggregate(Functions.plus, Functions.square) sqrt(sumSqs / (M * U)) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index cf1fc3e808c76..e698b9bf376e1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -34,8 +34,6 @@ object SparkHdfsLR { case class DataPoint(x: Vector, y: Double) def parsePoint(line: String): DataPoint = { - //val nums = line.split(' ').map(_.toDouble) - //return DataPoint(new Vector(nums.slice(1, D+1)), nums(0)) val tok = new java.util.StringTokenizer(line, " ") var y = tok.nextToken.toDouble var x = new Array[Double](D) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index 62d3a52615584..a22e64ca3ce45 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -168,7 +168,7 @@ object ActorWordCount { Props(new SampleActorReceiver[String]("akka.tcp://test@%s:%s/user/FeederActor".format( host, port.toInt))), "SampleReceiver") - //compute wordcount + // compute wordcount lines.flatMap(_.split("\\s+")).map(x => (x, 1)).reduceByKey(_ + _).print() ssc.start() diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 35be7ffa1e872..35f8f885f8f0e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -88,7 +88,7 @@ object ZeroMQWordCount { def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator - //For this stream, a zeroMQ publisher should be running. + // For this stream, a zeroMQ publisher should be running. val lines = ZeroMQUtils.createStream(ssc, url, Subscribe(topic), bytesToStringIterator _) val words = lines.flatMap(_.split(" ")) val wordCounts = words.map(x => (x, 1)).reduceByKey(_ + _) diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala index 6acba25f44c0a..a538c38dc4d6f 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -44,7 +44,7 @@ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, case m: ZMQMessage => logDebug("Received message for:" + m.frame(0)) - //We ignore first frame for processing as it is the topic + // We ignore first frame for processing as it is the topic val bytes = m.frames.tail pushBlock(bytesToObjects(bytes)) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala index fea43c3b2bbf1..dfc6a801587d2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeTriplet.scala @@ -27,12 +27,12 @@ class EdgeTriplet[VD, ED] extends Edge[ED] { /** * The source vertex attribute */ - var srcAttr: VD = _ //nullValue[VD] + var srcAttr: VD = _ // nullValue[VD] /** * The destination vertex attribute */ - var dstAttr: VD = _ //nullValue[VD] + var dstAttr: VD = _ // nullValue[VD] /** * Set the edge properties of this triplet. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala index 43ac11d8957f6..c2b510a31ee3f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/GraphImpl.scala @@ -190,9 +190,9 @@ class GraphImpl[VD: ClassTag, ED: ClassTag] protected ( new GraphImpl(vertices, newETable, routingTable, replicatedVertexView) } - ////////////////////////////////////////////////////////////////////////////////////////////////// + // /////////////////////////////////////////////////////////////////////////////////////////////// // Lower level transformation methods - ////////////////////////////////////////////////////////////////////////////////////////////////// + // /////////////////////////////////////////////////////////////////////////////////////////////// override def mapReduceTriplets[A: ClassTag]( mapFunc: EdgeTriplet[VD, ED] => Iterator[(VertexId, A)], diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index fe6fe76defdc5..bebe3740bc6c0 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -45,7 +45,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( * @param data value to send */ private[graphx] -class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/*, AnyRef*/) T]( +class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T]( @transient var partition: PartitionID, var data: T) extends Product2[PartitionID, T] with Serializable { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 34a145e01818f..2f2c524df6394 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -298,7 +298,6 @@ abstract class ShuffleSerializationStream(s: OutputStream) extends Serialization s.write(v.toInt) } - //def writeDouble(v: Double): Unit = writeUnsignedVarLong(java.lang.Double.doubleToLongBits(v)) def writeDouble(v: Double): Unit = writeLong(java.lang.Double.doubleToLongBits(v)) override def flush(): Unit = s.flush() @@ -391,7 +390,6 @@ abstract class ShuffleDeserializationStream(s: InputStream) extends Deserializat (s.read() & 0xFF) } - //def readDouble(): Double = java.lang.Double.longBitsToDouble(readUnsignedVarLong()) def readDouble(): Double = java.lang.Double.longBitsToDouble(readLong()) override def close(): Unit = s.close() diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index 014a7335f85cc..087b1156f690b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -65,7 +65,7 @@ private[graphx] object BytecodeUtils { val finder = new MethodInvocationFinder(c.getName, m) getClassReader(c).accept(finder, 0) for (classMethod <- finder.methodsInvoked) { - //println(classMethod) + // println(classMethod) if (classMethod._1 == targetClass && classMethod._2 == targetMethod) { return true } else if (!seen.contains(classMethod)) { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala index f841846c0e510..a3c8de3f9068f 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/GraphGenerators.scala @@ -123,7 +123,7 @@ object GraphGenerators { * the dimensions of the adjacency matrix */ private def addEdge(numVertices: Int): Edge[Int] = { - //val (src, dst) = chooseCell(numVertices/2.0, numVertices/2.0, numVertices/2.0) + // val (src, dst) = chooseCell(numVertices/2.0, numVertices/2.0, numVertices/2.0) val v = math.round(numVertices.toFloat/2.0).toInt val (src, dst) = chooseCell(v, v, v) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9e269e6551341..2549bc9710f1f 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -17,7 +17,7 @@ import sbt._ import sbt.Classpaths.publishTask -import Keys._ +import sbt.Keys._ import sbtassembly.Plugin._ import AssemblyKeys._ import scala.util.Properties @@ -27,7 +27,7 @@ import com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact import scala.collection.JavaConversions._ // For Sonatype publishing -//import com.jsuereth.pgp.sbtplugin.PgpKeys._ +// import com.jsuereth.pgp.sbtplugin.PgpKeys._ object SparkBuild extends Build { val SPARK_VERSION = "1.0.0-SNAPSHOT" @@ -200,7 +200,7 @@ object SparkBuild extends Build { publishMavenStyle := true, - //useGpg in Global := true, + // useGpg in Global := true, pomExtra := ( diff --git a/project/plugins.sbt b/project/plugins.sbt index 4ff6f67af45c0..5aa8a1ec2409b 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -22,3 +22,4 @@ addSbtPlugin("org.scalastyle" %% "scalastyle-sbt-plugin" % "0.4.0") addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.0") + diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala new file mode 100644 index 0000000000000..43361aa2b4c41 --- /dev/null +++ b/project/project/SparkPluginBuild.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. + */ + +import sbt._ +import sbt.Keys._ + +/** + * This plugin project is there to define new scala style rules for spark. This is + * a plugin project so that this gets compiled first and is put on the classpath and + * becomes available for scalastyle sbt plugin. + */ +object SparkPluginDef extends Build { + lazy val root = Project("plugins", file(".")) dependsOn(sparkStyle) + lazy val sparkStyle = Project("spark-style", file("spark-style"), settings = styleSettings) + val sparkVersion = "1.0.0-SNAPSHOT" + // There is actually no need to publish this artifact. + def styleSettings = Defaults.defaultSettings ++ Seq ( + name := "spark-style", + organization := "org.apache.spark", + version := sparkVersion, + scalaVersion := "2.10.3", + scalacOptions := Seq("-unchecked", "-deprecation"), + libraryDependencies ++= Dependencies.scalaStyle, + sbtPlugin := true + ) + + object Dependencies { + val scalaStyle = Seq("org.scalastyle" %% "scalastyle" % "0.4.0") + } +} diff --git a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala new file mode 100644 index 0000000000000..2f3c1a182814d --- /dev/null +++ b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala @@ -0,0 +1,56 @@ +/* + * 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.scalastyle + +import java.util.regex.Pattern + +import org.scalastyle.{PositionError, ScalariformChecker, ScalastyleError} +import scalariform.lexer.{MultiLineComment, ScalaDocComment, SingleLineComment, Token} +import scalariform.parser.CompilationUnit + +class SparkSpaceAfterCommentStartChecker extends ScalariformChecker { + val errorKey: String = "insert.a.single.space.after.comment.start" + + private def multiLineCommentRegex(comment: Token) = + Pattern.compile( """/\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() + + private def scalaDocPatternRegex(comment: Token) = + Pattern.compile( """/\*\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() + + private def singleLineCommentRegex(comment: Token): Boolean = + comment.text.trim.matches( """//\S+.*""") && !comment.text.trim.matches( """///+""") + + override def verify(ast: CompilationUnit): List[ScalastyleError] = { + ast.tokens + .filter(hasComment) + .map { + _.associatedWhitespaceAndComments.comments.map { + case x: SingleLineComment if singleLineCommentRegex(x.token) => Some(x.token.offset) + case x: MultiLineComment if multiLineCommentRegex(x.token) => Some(x.token.offset) + case x: ScalaDocComment if scalaDocPatternRegex(x.token) => Some(x.token.offset) + case _ => None + }.flatten + }.flatten.map(PositionError(_)) + } + + + private def hasComment(x: Token) = + x.associatedWhitespaceAndComments != null && !x.associatedWhitespaceAndComments.comments.isEmpty + +} diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index ee972887feda6..bf73800388ebf 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -124,8 +124,8 @@ extends ClassVisitor(ASM4, cv) { mv.visitVarInsn(ALOAD, 0) // load this mv.visitMethodInsn(INVOKESPECIAL, "java/lang/Object", "", "()V") mv.visitVarInsn(ALOAD, 0) // load this - //val classType = className.replace('.', '/') - //mv.visitFieldInsn(PUTSTATIC, classType, "MODULE$", "L" + classType + ";") + // val classType = className.replace('.', '/') + // mv.visitFieldInsn(PUTSTATIC, classType, "MODULE$", "L" + classType + ";") mv.visitInsn(RETURN) mv.visitMaxs(-1, -1) // stack size and local vars will be auto-computed mv.visitEnd() diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala index 90a96ad38381e..fa2f1a88c4eb5 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkIMain.scala @@ -834,7 +834,7 @@ import org.apache.spark.util.Utils } ((pos, msg)) :: loop(filtered) } - //PRASHANT: This leads to a NoSuchMethodError for _.warnings. Yet to figure out its purpose. + // PRASHANT: This leads to a NoSuchMethodError for _.warnings. Yet to figure out its purpose. // val warnings = loop(run.allConditionalWarnings flatMap (_.warnings)) // if (warnings.nonEmpty) // mostRecentWarnings = warnings diff --git a/scalastyle-config.xml b/scalastyle-config.xml index ee968c53b3e4b..76ba1ecca33ab 100644 --- a/scalastyle-config.xml +++ b/scalastyle-config.xml @@ -140,4 +140,5 @@ + 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 f4b61381f9a27..b70ec897e43e7 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 @@ -662,7 +662,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(groups, Star(None, First(_)) :: Nil, joinedResult) + ??? // Aggregate(groups, Star(None, First(_)) :: Nil, joinedResult) case Token(allJoinTokens(joinToken), relation1 :: diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala index ca5311344615f..0da5eb754cb3f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala @@ -94,7 +94,7 @@ class HadoopTableReader(@transient _tableDesc: TableDesc, @transient sc: HiveCon val tablePath = hiveTable.getPath val inputPathStr = applyFilterIfNeeded(tablePath, filterOpt) - //logDebug("Table input: %s".format(tablePath)) + // logDebug("Table input: %s".format(tablePath)) val ifc = hiveTable.getInputFormatClass .asInstanceOf[java.lang.Class[InputFormat[Writable, Writable]]] val hadoopRDD = createHadoopRdd(tableDesc, inputPathStr, ifc) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index fde46705d89fb..d3339063cc079 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -153,7 +153,7 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def validate() { this.synchronized { assert(batchDuration != null, "Batch duration has not been set") - //assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + + // assert(batchDuration >= Milliseconds(100), "Batch duration of " + batchDuration + // " is very low") assert(getOutputStreams().size > 0, "No output streams registered, so nothing to execute") } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 0dc6704603f82..72ad0bae75bfb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -128,7 +128,6 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging } catch { case ie: InterruptedException => logInfo("Receiving thread interrupted") - //println("Receiving thread interrupted") case e: Exception => stopOnError(e) } @@ -142,7 +141,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging def stop() { receivingThread.interrupt() onStop() - //TODO: terminate the actor + // TODO: terminate the actor } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala index ca0a8ae47864d..b334d68bf9910 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReducedWindowedDStream.scala @@ -78,7 +78,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( override def checkpoint(interval: Duration): DStream[(K, V)] = { super.checkpoint(interval) - //reducedStream.checkpoint(interval) + // reducedStream.checkpoint(interval) this } @@ -128,7 +128,7 @@ class ReducedWindowedDStream[K: ClassTag, V: ClassTag]( // Cogroup the reduced RDDs and merge the reduced values val cogroupedRDD = new CoGroupedRDD[K](allRDDs.toSeq.asInstanceOf[Seq[RDD[(K, _)]]], partitioner) - //val mergeValuesFunc = mergeValues(oldRDDs.size, newRDDs.size) _ + // val mergeValuesFunc = mergeValues(oldRDDs.size, newRDDs.size) _ val numOldValues = oldRDDs.size val numNewValues = newRDDs.size diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 9d8889b655356..5f7d3ba26c656 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -64,7 +64,6 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( } val cogroupedRDD = parentRDD.cogroup(prevStateRDD, partitioner) val stateRDD = cogroupedRDD.mapPartitions(finalFunc, preservePartitioning) - //logDebug("Generating state RDD for time " + validTime) Some(stateRDD) } case None => { // If parent RDD does not exist @@ -97,11 +96,11 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( val groupedRDD = parentRDD.groupByKey(partitioner) val sessionRDD = groupedRDD.mapPartitions(finalFunc, preservePartitioning) - //logDebug("Generating state RDD for time " + validTime + " (first)") + // logDebug("Generating state RDD for time " + validTime + " (first)") Some(sessionRDD) } case None => { // If parent RDD does not exist, then nothing to do! - //logDebug("Not generating state RDD (no previous state, no parent)") + // logDebug("Not generating state RDD (no previous state, no parent)") None } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index e4fa163f2e069..cad68e248ab29 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -126,7 +126,7 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { receiverInfo -= streamId logError("De-registered receiver for network stream " + streamId + " with message " + msg) - //TODO: Do something about the corresponding NetworkInputDStream + // TODO: Do something about the corresponding NetworkInputDStream } } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala index 0784e562ac719..25739956cb889 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/CheckpointSuite.scala @@ -252,7 +252,7 @@ class CheckpointSuite extends TestSuiteBase { ssc.start() // Create files and advance manual clock to process them - //var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + // var clock = ssc.scheduler.clock.asInstanceOf[ManualClock] Thread.sleep(1000) for (i <- Seq(1, 2, 3)) { Files.write(i + "\n", new File(testDir, i.toString), Charset.forName("UTF-8")) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 74e73ebb342fe..7df206241beb6 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -154,7 +154,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) val networkStream = ssc.actorStream[String](Props(new TestActor(port)), "TestActor", - StorageLevel.MEMORY_AND_DISK) //Had to pass the local value of port to prevent from closing over entire scope + // Had to pass the local value of port to prevent from closing over entire scope + StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] val outputStream = new TestOutputStream(networkStream, outputBuffer) def output = outputBuffer.flatMap(x => x) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 57e5761cba896..6568003bf1008 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -139,7 +139,6 @@ trait ClientBase extends Logging { } else if (srcHost != null && dstHost == null) { return false } - //check for ports if (srcUri.getPort() != dstUri.getPort()) { false } else { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala index 68cda0f1c9f8b..9b7f1fca96c6d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManager.scala @@ -157,7 +157,7 @@ class ClientDistributedCacheManager() extends Logging { def isPublic(conf: Configuration, uri: URI, statCache: Map[URI, FileStatus]): Boolean = { val fs = FileSystem.get(uri, conf) val current = new Path(uri.getPath()) - //the leaf level file should be readable by others + // the leaf level file should be readable by others if (!checkPermissionOfOther(fs, current, FsAction.READ, statCache)) { return false } @@ -177,7 +177,7 @@ class ClientDistributedCacheManager() extends Logging { statCache: Map[URI, FileStatus]): Boolean = { var current = path while (current != null) { - //the subdirs in the path should have execute permissions for others + // the subdirs in the path should have execute permissions for others if (!checkPermissionOfOther(fs, current, FsAction.EXECUTE, statCache)) { return false } diff --git a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala index 458df4fa3cd99..80b57d1355a3a 100644 --- a/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala +++ b/yarn/common/src/test/scala/org/apache/spark/deploy/yarn/ClientDistributedCacheManagerSuite.scala @@ -99,7 +99,7 @@ class ClientDistributedCacheManagerSuite extends FunSuite with MockitoSugar { assert(env.get("SPARK_YARN_CACHE_ARCHIVES_FILE_SIZES") === None) assert(env.get("SPARK_YARN_CACHE_ARCHIVES_VISIBILITIES") === None) - //add another one and verify both there and order correct + // add another one and verify both there and order correct val realFileStatus = new FileStatus(20, false, 1, 1024, 10, 30, null, "testOwner", null, new Path("/tmp/testing2")) val destPath2 = new Path("file:///foo.invalid.com:8080/tmp/testing2") From 75d46be5d61fb92a6db2efb9e3a690716ef521d3 Mon Sep 17 00:00:00 2001 From: Nick Lanham Date: Fri, 28 Mar 2014 13:33:35 -0700 Subject: [PATCH 154/397] fix path for jar, make sed actually work on OSX Author: Nick Lanham Closes #264 from nicklan/make-distribution-fixes and squashes the following commits: 172b981 [Nick Lanham] fix path for jar, make sed actually work on OSX --- make-distribution.sh | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/make-distribution.sh b/make-distribution.sh index b07aef2ef49c2..5c780fcbda863 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -141,12 +141,11 @@ if [ "$SPARK_TACHYON" == "true" ]; then cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" if [[ `uname -a` == Darwin* ]]; then - # osx sed wants an empty argument to -i option of sed - TACHYON_SED="sed -i ''" + # need to run sed differently on osx + nl=$'\n'; sed -i "" -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\\$nl export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" else - TACHYON_SED="sed -i" + sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" fi - $TACHYON_SED -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" popd > /dev/null rm -rf $TMPD From 3738f24421d6f3bd10e5ef9ebfc10f702a5cb7ac Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 28 Mar 2014 23:09:29 -0700 Subject: [PATCH 155/397] SPARK-1345 adding missing dependency on avro for hadoop 0.23 to the new ... ...sql pom files Author: Thomas Graves Closes #263 from tgravescs/SPARK-1345 and squashes the following commits: b43a2a0 [Thomas Graves] SPARK-1345 adding missing dependency on avro for hadoop 0.23 to the new sql pom files --- sql/catalyst/pom.xml | 12 ++++++++++++ sql/core/pom.xml | 11 +++++++++++ sql/hive/pom.xml | 11 +++++++++++ 3 files changed, 34 insertions(+) diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 740f1fdc83299..0edce55a93338 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -31,6 +31,18 @@ Spark Project Catalyst http://spark.apache.org/ + + + yarn-alpha + + + org.apache.avro + avro + + + + + org.apache.spark diff --git a/sql/core/pom.xml b/sql/core/pom.xml index e367edfb1f562..85580ed6b822f 100644 --- a/sql/core/pom.xml +++ b/sql/core/pom.xml @@ -30,6 +30,17 @@ jar Spark Project SQL http://spark.apache.org/ + + + yarn-alpha + + + org.apache.avro + avro + + + + diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 7b5ea98f27ff5..63f592cb4b441 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -30,6 +30,17 @@ jar Spark Project Hive http://spark.apache.org/ + + + yarn-alpha + + + org.apache.avro + avro + + + + From 1617816090e7b20124a512a43860a21232ebf511 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Sat, 29 Mar 2014 14:41:36 -0700 Subject: [PATCH 156/397] SPARK-1126. spark-app preliminary This is a starting version of the spark-app script for running compiled binaries against Spark. It still needs tests and some polish. The only testing I've done so far has been using it to launch jobs in yarn-standalone mode against a pseudo-distributed cluster. This leaves out the changes required for launching python scripts. I think it might be best to save those for another JIRA/PR (while keeping to the design so that they won't require backwards-incompatible changes). Author: Sandy Ryza Closes #86 from sryza/sandy-spark-1126 and squashes the following commits: d428d85 [Sandy Ryza] Commenting, doc, and import fixes from Patrick's comments e7315c6 [Sandy Ryza] Fix failing tests 34de899 [Sandy Ryza] Change --more-jars to --jars and fix docs 299ddca [Sandy Ryza] Fix scalastyle a94c627 [Sandy Ryza] Add newline at end of SparkSubmit 04bc4e2 [Sandy Ryza] SPARK-1126. spark-submit script --- bin/spark-submit | 38 ++++ .../org/apache/spark/deploy/SparkSubmit.scala | 212 ++++++++++++++++++ .../spark/deploy/SparkSubmitArguments.scala | 176 +++++++++++++++ .../spark/deploy/SparkSubmitSuite.scala | 121 ++++++++++ docs/cluster-overview.md | 50 +++++ docs/running-on-yarn.md | 6 +- docs/spark-standalone.md | 7 +- .../cluster/YarnClientSchedulerBackend.scala | 45 ++-- 8 files changed, 630 insertions(+), 25 deletions(-) create mode 100755 bin/spark-submit create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala create mode 100644 core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala diff --git a/bin/spark-submit b/bin/spark-submit new file mode 100755 index 0000000000000..d92d55a032bd5 --- /dev/null +++ b/bin/spark-submit @@ -0,0 +1,38 @@ +#!/usr/bin/env 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. +# + +export SPARK_HOME="$(cd `dirname $0`/..; pwd)" +ORIG_ARGS=$@ + +while (($#)); do + if [ $1 = "--deploy-mode" ]; then + DEPLOY_MODE=$2 + elif [ $1 = "--driver-memory" ]; then + DRIVER_MEMORY=$2 + fi + + shift +done + +if [ ! -z $DRIVER_MEMORY ] && [ ! -z $DEPLOY_MODE ] && [ $DEPLOY_MODE = "client" ]; then + export SPARK_MEM=$DRIVER_MEMORY +fi + +$SPARK_HOME/bin/spark-class org.apache.spark.deploy.SparkSubmit $ORIG_ARGS + diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala new file mode 100644 index 0000000000000..24a9c98e188f6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -0,0 +1,212 @@ +/* + * 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.deploy + +import java.io.File +import java.net.URL + +import org.apache.spark.executor.ExecutorURLClassLoader + +import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.HashMap +import scala.collection.mutable.Map + +/** + * Scala code behind the spark-submit script. The script handles setting up the classpath with + * relevant Spark dependencies and provides a layer over the different cluster managers and deploy + * modes that Spark supports. + */ +object SparkSubmit { + val YARN = 1 + val STANDALONE = 2 + val MESOS = 4 + val LOCAL = 8 + val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL + + var clusterManager: Int = LOCAL + + def main(args: Array[String]) { + val appArgs = new SparkSubmitArguments(args) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + launch(childArgs, classpath, sysProps, mainClass) + } + + /** + * @return + * a tuple containing the arguments for the child, a list of classpath + * entries for the child, and the main class for the child + */ + def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], + ArrayBuffer[String], Map[String, String], String) = { + if (appArgs.master.startsWith("yarn")) { + clusterManager = YARN + } else if (appArgs.master.startsWith("spark")) { + clusterManager = STANDALONE + } else if (appArgs.master.startsWith("mesos")) { + clusterManager = MESOS + } else if (appArgs.master.startsWith("local")) { + clusterManager = LOCAL + } else { + System.err.println("master must start with yarn, mesos, spark, or local") + System.exit(1) + } + + // Because "yarn-standalone" and "yarn-client" encapsulate both the master + // and deploy mode, we have some logic to infer the master and deploy mode + // from each other if only one is specified, or exit early if they are at odds. + if (appArgs.deployMode == null && appArgs.master == "yarn-standalone") { + appArgs.deployMode = "cluster" + } + if (appArgs.deployMode == "cluster" && appArgs.master == "yarn-client") { + System.err.println("Deploy mode \"cluster\" and master \"yarn-client\" are at odds") + System.exit(1) + } + if (appArgs.deployMode == "client" && appArgs.master == "yarn-standalone") { + System.err.println("Deploy mode \"client\" and master \"yarn-standalone\" are at odds") + System.exit(1) + } + if (appArgs.deployMode == "cluster" && appArgs.master.startsWith("yarn")) { + appArgs.master = "yarn-standalone" + } + if (appArgs.deployMode != "cluster" && appArgs.master.startsWith("yarn")) { + appArgs.master = "yarn-client" + } + + val deployOnCluster = Option(appArgs.deployMode).getOrElse("client") == "cluster" + + val childClasspath = new ArrayBuffer[String]() + val childArgs = new ArrayBuffer[String]() + val sysProps = new HashMap[String, String]() + var childMainClass = "" + + if (clusterManager == MESOS && deployOnCluster) { + System.err.println("Mesos does not support running the driver on the cluster") + System.exit(1) + } + + if (!deployOnCluster) { + childMainClass = appArgs.mainClass + childClasspath += appArgs.primaryResource + } else if (clusterManager == YARN) { + childMainClass = "org.apache.spark.deploy.yarn.Client" + childArgs += ("--jar", appArgs.primaryResource) + childArgs += ("--class", appArgs.mainClass) + } + + val options = List[OptionAssigner]( + new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), + new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), + new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), + new OptionAssigner(appArgs.queue, YARN, false, sysProp = "spark.yarn.queue"), + new OptionAssigner(appArgs.numExecutors, YARN, true, clOption = "--num-executors"), + new OptionAssigner(appArgs.numExecutors, YARN, false, sysProp = "spark.executor.instances"), + new OptionAssigner(appArgs.executorMemory, YARN, true, clOption = "--executor-memory"), + new OptionAssigner(appArgs.executorMemory, STANDALONE | MESOS | YARN, false, + sysProp = "spark.executor.memory"), + new OptionAssigner(appArgs.driverMemory, STANDALONE, true, clOption = "--memory"), + new OptionAssigner(appArgs.driverCores, STANDALONE, true, clOption = "--cores"), + new OptionAssigner(appArgs.executorCores, YARN, true, clOption = "--executor-cores"), + new OptionAssigner(appArgs.executorCores, YARN, false, sysProp = "spark.executor.cores"), + new OptionAssigner(appArgs.totalExecutorCores, STANDALONE | MESOS, false, + sysProp = "spark.cores.max"), + new OptionAssigner(appArgs.files, YARN, false, sysProp = "spark.yarn.dist.files"), + new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), + new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), + new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), + new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars") + ) + + // more jars + if (appArgs.jars != null && !deployOnCluster) { + for (jar <- appArgs.jars.split(",")) { + childClasspath += jar + } + } + + for (opt <- options) { + if (opt.value != null && deployOnCluster == opt.deployOnCluster && + (clusterManager & opt.clusterManager) != 0) { + if (opt.clOption != null) { + childArgs += (opt.clOption, opt.value) + } else if (opt.sysProp != null) { + sysProps.put(opt.sysProp, opt.value) + } + } + } + + if (deployOnCluster && clusterManager == STANDALONE) { + if (appArgs.supervise) { + childArgs += "--supervise" + } + + childMainClass = "org.apache.spark.deploy.Client" + childArgs += "launch" + childArgs += (appArgs.master, appArgs.primaryResource, appArgs.mainClass) + } + + // args + if (appArgs.childArgs != null) { + if (!deployOnCluster || clusterManager == STANDALONE) { + childArgs ++= appArgs.childArgs + } else if (clusterManager == YARN) { + for (arg <- appArgs.childArgs) { + childArgs += ("--args", arg) + } + } + } + + (childArgs, childClasspath, sysProps, childMainClass) + } + + def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], + sysProps: Map[String, String], childMainClass: String) { + val loader = new ExecutorURLClassLoader(new Array[URL](0), + Thread.currentThread.getContextClassLoader) + Thread.currentThread.setContextClassLoader(loader) + + for (jar <- childClasspath) { + addJarToClasspath(jar, loader) + } + + for ((key, value) <- sysProps) { + System.setProperty(key, value) + } + + val mainClass = Class.forName(childMainClass, true, loader) + val mainMethod = mainClass.getMethod("main", new Array[String](0).getClass) + mainMethod.invoke(null, childArgs.toArray) + } + + def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { + val localJarFile = new File(localJar) + if (!localJarFile.exists()) { + System.err.println("Jar does not exist: " + localJar + ". Skipping.") + } + + val url = localJarFile.getAbsoluteFile.toURI.toURL + loader.addURL(url) + } +} + +private[spark] class OptionAssigner(val value: String, + val clusterManager: Int, + val deployOnCluster: Boolean, + val clOption: String = null, + val sysProp: String = null +) { } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala new file mode 100644 index 0000000000000..ff2aa68908e34 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -0,0 +1,176 @@ +/* + * 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.deploy + +import scala.collection.mutable.ArrayBuffer + +/** + * Parses and encapsulates arguments from the spark-submit script. + */ +private[spark] class SparkSubmitArguments(args: Array[String]) { + var master: String = "local" + var deployMode: String = null + var executorMemory: String = null + var executorCores: String = null + var totalExecutorCores: String = null + var driverMemory: String = null + var driverCores: String = null + var supervise: Boolean = false + var queue: String = null + var numExecutors: String = null + var files: String = null + var archives: String = null + var mainClass: String = null + var primaryResource: String = null + var name: String = null + var childArgs: ArrayBuffer[String] = new ArrayBuffer[String]() + var jars: String = null + + loadEnvVars() + parseArgs(args.toList) + + def loadEnvVars() { + master = System.getenv("MASTER") + deployMode = System.getenv("DEPLOY_MODE") + } + + def parseArgs(args: List[String]) { + if (args.size == 0) { + printUsageAndExit(1) + System.exit(1) + } + primaryResource = args(0) + parseOpts(args.tail) + } + + def parseOpts(opts: List[String]): Unit = opts match { + case ("--name") :: value :: tail => + name = value + parseOpts(tail) + + case ("--master") :: value :: tail => + master = value + parseOpts(tail) + + case ("--class") :: value :: tail => + mainClass = value + parseOpts(tail) + + case ("--deploy-mode") :: value :: tail => + if (value != "client" && value != "cluster") { + System.err.println("--deploy-mode must be either \"client\" or \"cluster\"") + System.exit(1) + } + deployMode = value + parseOpts(tail) + + case ("--num-executors") :: value :: tail => + numExecutors = value + parseOpts(tail) + + case ("--total-executor-cores") :: value :: tail => + totalExecutorCores = value + parseOpts(tail) + + case ("--executor-cores") :: value :: tail => + executorCores = value + parseOpts(tail) + + case ("--executor-memory") :: value :: tail => + executorMemory = value + parseOpts(tail) + + case ("--driver-memory") :: value :: tail => + driverMemory = value + parseOpts(tail) + + case ("--driver-cores") :: value :: tail => + driverCores = value + parseOpts(tail) + + case ("--supervise") :: tail => + supervise = true + parseOpts(tail) + + case ("--queue") :: value :: tail => + queue = value + parseOpts(tail) + + case ("--files") :: value :: tail => + files = value + parseOpts(tail) + + case ("--archives") :: value :: tail => + archives = value + parseOpts(tail) + + case ("--arg") :: value :: tail => + childArgs += value + parseOpts(tail) + + case ("--jars") :: value :: tail => + jars = value + parseOpts(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case Nil => + + case _ => + printUsageAndExit(1, opts) + } + + def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + if (unknownParam != null) { + System.err.println("Unknown/unsupported param " + unknownParam) + } + System.err.println( + """Usage: spark-submit [options] + |Options: + | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. + | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. + | --class CLASS_NAME Name of your app's main class (required for Java apps). + | --arg ARG Argument to be passed to your application's main class. This + | option can be specified multiple times for multiple args. + | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). + | --name NAME The name of your application (Default: 'Spark'). + | --jars JARS A comma-separated list of local jars to include on the + | driver classpath and that SparkContext.addJar will work + | with. Doesn't work on standalone with 'cluster' deploy mode. + | + | Spark standalone with cluster deploy mode only: + | --driver-cores NUM Cores for driver (Default: 1). + | --supervise If given, restarts the driver on failure. + | + | Spark standalone and Mesos only: + | --total-executor-cores NUM Total cores for all executors. + | + | YARN-only: + | --executor-cores NUM Number of cores per executor (Default: 1). + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). + | --queue QUEUE_NAME The YARN queue to submit to (Default: 'default'). + | --num-executors NUM Number of executors to (Default: 2). + | --files FILES Comma separated list of files to be placed in the working dir + | of each executor. + | --archives ARCHIVES Comma separated list of archives to be extracted into the + | working dir of each executor.""".stripMargin + ) + System.exit(exitCode) + } +} diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala new file mode 100644 index 0000000000000..29fef2ed8c165 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -0,0 +1,121 @@ +/* + * 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.deploy + +import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers +import org.apache.spark.deploy.SparkSubmit._ + +class SparkSubmitSuite extends FunSuite with ShouldMatchers { + test("prints usage on empty input") { + val clArgs = Array[String]() + // val appArgs = new SparkSubmitArguments(clArgs) + } + + test("handles YARN cluster mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", + "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", + "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", + "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", + "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val childArgsStr = childArgs.mkString(" ") + childArgsStr should include ("--jar thejar.jar") + childArgsStr should include ("--class org.SomeClass") + childArgsStr should include ("--addJars one.jar,two.jar,three.jar") + childArgsStr should include ("--executor-memory 5g") + childArgsStr should include ("--driver-memory 4g") + childArgsStr should include ("--executor-cores 5") + childArgsStr should include ("--args arg1 --args arg2") + childArgsStr should include ("--queue thequeue") + childArgsStr should include ("--files file1.txt,file2.txt") + childArgsStr should include ("--archives archive1.txt,archive2.txt") + childArgsStr should include ("--num-executors 6") + mainClass should be ("org.apache.spark.deploy.yarn.Client") + classpath should have length (0) + sysProps should have size (0) + } + + test("handles YARN client mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "client", + "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", + "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", + "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", + "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + childArgs.mkString(" ") should be ("arg1 arg2") + mainClass should be ("org.SomeClass") + classpath should contain ("thejar.jar") + classpath should contain ("one.jar") + classpath should contain ("two.jar") + classpath should contain ("three.jar") + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.executor.cores") should be ("5") + sysProps("spark.yarn.queue") should be ("thequeue") + sysProps("spark.yarn.dist.files") should be ("file1.txt,file2.txt") + sysProps("spark.yarn.dist.archives") should be ("archive1.txt,archive2.txt") + sysProps("spark.executor.instances") should be ("6") + } + + test("handles standalone cluster mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", + "--master", "spark://h:p", "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", + "--supervise", "--driver-memory", "4g", "--driver-cores", "5") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + val childArgsStr = childArgs.mkString(" ") + print("child args: " + childArgsStr) + childArgsStr.startsWith("--memory 4g --cores 5 --supervise") should be (true) + childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") + mainClass should be ("org.apache.spark.deploy.Client") + classpath should have length (0) + sysProps should have size (0) + } + + test("handles standalone client mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "client", + "--master", "spark://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", + "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", + "--driver-memory", "4g") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + childArgs.mkString(" ") should be ("arg1 arg2") + mainClass should be ("org.SomeClass") + classpath should contain ("thejar.jar") + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.cores.max") should be ("5") + } + + test("handles mesos client mode") { + val clArgs = Array("thejar.jar", "--deploy-mode", "client", + "--master", "mesos://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", + "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", + "--driver-memory", "4g") + val appArgs = new SparkSubmitArguments(clArgs) + val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) + childArgs.mkString(" ") should be ("arg1 arg2") + mainClass should be ("org.SomeClass") + classpath should contain ("thejar.jar") + sysProps("spark.executor.memory") should be ("5g") + sysProps("spark.cores.max") should be ("5") + } +} diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index a555a7b5023e3..b69e3416fb322 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -50,6 +50,50 @@ The system currently supports three cluster managers: In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone cluster on Amazon EC2. +# Launching Applications + +The recommended way to launch a compiled Spark application is through the spark-submit script (located in the +bin directory), which takes care of setting up the classpath with Spark and its dependencies, as well as +provides a layer over the different cluster managers and deploy modes that Spark supports. It's usage is + + spark-submit `` `` + +Where options are any of: + +- **\--class** - The main class to run. +- **\--master** - The URL of the cluster manager master, e.g. spark://host:port, mesos://host:port, yarn, + or local. +- **\--deploy-mode** - "client" to run the driver in the client process or "cluster" to run the driver in + a process on the cluster. For Mesos, only "client" is supported. +- **\--executor-memory** - Memory per executor (e.g. 1000M, 2G). +- **\--executor-cores** - Number of cores per executor. (Default: 2) +- **\--driver-memory** - Memory for driver (e.g. 1000M, 2G) +- **\--name** - Name of the application. +- **\--arg** - Argument to be passed to the application's main class. This option can be specified + multiple times to pass multiple arguments. +- **\--jars** - A comma-separated list of local jars to include on the driver classpath and that + SparkContext.addJar will work with. Doesn't work on standalone with 'cluster' deploy mode. + +The following currently only work for Spark standalone with cluster deploy mode: + +- **\--driver-cores** - Cores for driver (Default: 1). +- **\--supervise** - If given, restarts the driver on failure. + +The following only works for Spark standalone and Mesos only: + +- **\--total-executor-cores** - Total cores for all executors. + +The following currently only work for YARN: + +- **\--queue** - The YARN queue to place the application in. +- **\--files** - Comma separated list of files to be placed in the working dir of each executor. +- **\--archives** - Comma separated list of archives to be extracted into the working dir of each + executor. +- **\--num-executors** - Number of executors (Default: 2). + +The master and deploy mode can also be set with the MASTER and DEPLOY_MODE environment variables. +Values for these options passed via command line will override the environment variables. + # Shipping Code to the Cluster The recommended way to ship your code to the cluster is to pass it through SparkContext's constructor, @@ -102,6 +146,12 @@ The following table summarizes terms you'll see used to refer to cluster concept Cluster manager An external service for acquiring resources on the cluster (e.g. standalone manager, Mesos, YARN) + + Deploy mode + Distinguishes where the driver process runs. In "cluster" mode, the framework launches + the driver inside of the cluster. In "client" mode, the submitter launches the driver + outside of the cluster. + Worker node Any node that can run application code in the cluster diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index 2e9dec4856ee9..d8657c4bc7096 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -48,10 +48,12 @@ System Properties: Ensure that HADOOP_CONF_DIR or YARN_CONF_DIR points to the directory which contains the (client side) configuration files for the Hadoop cluster. These configs are used to connect to the cluster, write to the dfs, and connect to the YARN ResourceManager. -There are two scheduler modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. +There are two deploy modes that can be used to launch Spark applications on YARN. In yarn-cluster mode, the Spark driver runs inside an application master process which is managed by YARN on the cluster, and the client can go away after initiating the application. In yarn-client mode, the driver runs in the client process, and the application master is only used for requesting resources from YARN. Unlike in Spark standalone and Mesos mode, in which the master's address is specified in the "master" parameter, in YARN mode the ResourceManager's address is picked up from the Hadoop configuration. Thus, the master parameter is simply "yarn-client" or "yarn-cluster". +The spark-submit script described in the [cluster mode overview](cluster-overview.html) provides the most straightforward way to submit a compiled Spark application to YARN in either deploy mode. For info on the lower-level invocations it uses, read ahead. For running spark-shell against YARN, skip down to the yarn-client section. + ## Launching a Spark application with yarn-cluster mode. The command to launch the Spark application on the cluster is as follows: @@ -121,7 +123,7 @@ or MASTER=yarn-client ./bin/spark-shell -## Viewing logs +# Viewing logs In YARN terminology, executors and application masters run inside "containers". YARN has two modes for handling container logs after an application has completed. If log aggregation is turned on (with the yarn.log-aggregation-enable config), container logs are copied to HDFS and deleted on the local machine. These logs can be viewed from anywhere on the cluster with the "yarn logs" command. diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 51fb3a4f7f8c5..7e4eea323aa63 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -146,10 +146,13 @@ automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` vari You can also pass an option `-c ` to control the number of cores that spark-shell uses on the cluster. -# Launching Applications Inside the Cluster +# Launching Compiled Spark Applications -You may also run your application entirely inside of the cluster by submitting your application driver using the submission client. The syntax for submitting applications is as follows: +Spark supports two deploy modes. Spark applications may run with the driver inside the client process or entirely inside the cluster. +The spark-submit script described in the [cluster mode overview](cluster-overview.html) provides the most straightforward way to submit a compiled Spark application to the cluster in either deploy mode. For info on the lower-level invocations used to launch an app inside the cluster, read ahead. + +## Launching Applications Inside the Cluster ./bin/spark-class org.apache.spark.deploy.Client launch [client-options] \ diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index d1f13e3c369ed..161918859e7c4 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -33,11 +33,12 @@ private[spark] class YarnClientSchedulerBackend( var client: Client = null var appId: ApplicationId = null - private[spark] def addArg(optionName: String, optionalParam: String, arrayBuf: ArrayBuffer[String]) { - Option(System.getenv(optionalParam)) foreach { - optParam => { - arrayBuf += (optionName, optParam) - } + private[spark] def addArg(optionName: String, envVar: String, sysProp: String, + arrayBuf: ArrayBuffer[String]) { + if (System.getProperty(sysProp) != null) { + arrayBuf += (optionName, System.getProperty(sysProp)) + } else if (System.getenv(envVar) != null) { + arrayBuf += (optionName, System.getenv(envVar)) } } @@ -56,22 +57,24 @@ private[spark] class YarnClientSchedulerBackend( "--am-class", "org.apache.spark.deploy.yarn.ExecutorLauncher" ) - // process any optional arguments, use the defaults already defined in ClientArguments - // if things aren't specified - Map("SPARK_MASTER_MEMORY" -> "--driver-memory", - "SPARK_DRIVER_MEMORY" -> "--driver-memory", - "SPARK_WORKER_INSTANCES" -> "--num-executors", - "SPARK_WORKER_MEMORY" -> "--executor-memory", - "SPARK_WORKER_CORES" -> "--executor-cores", - "SPARK_EXECUTOR_INSTANCES" -> "--num-executors", - "SPARK_EXECUTOR_MEMORY" -> "--executor-memory", - "SPARK_EXECUTOR_CORES" -> "--executor-cores", - "SPARK_YARN_QUEUE" -> "--queue", - "SPARK_YARN_APP_NAME" -> "--name", - "SPARK_YARN_DIST_FILES" -> "--files", - "SPARK_YARN_DIST_ARCHIVES" -> "--archives") - .foreach { case (optParam, optName) => addArg(optName, optParam, argsArrayBuf) } - + // process any optional arguments, given either as environment variables + // or system properties. use the defaults already defined in ClientArguments + // if things aren't specified. system properties override environment + // variables. + List(("--driver-memory", "SPARK_MASTER_MEMORY", "spark.master.memory"), + ("--driver-memory", "SPARK_DRIVER_MEMORY", "spark.driver.memory"), + ("--num-executors", "SPARK_WORKER_INSTANCES", "spark.worker.instances"), + ("--num-executors", "SPARK_EXECUTOR_INSTANCES", "spark.executor.instances"), + ("--executor-memory", "SPARK_WORKER_MEMORY", "spark.executor.memory"), + ("--executor-memory", "SPARK_EXECUTOR_MEMORY", "spark.executor.memory"), + ("--executor-cores", "SPARK_WORKER_CORES", "spark.executor.cores"), + ("--executor-cores", "SPARK_EXECUTOR_CORES", "spark.executor.cores"), + ("--queue", "SPARK_YARN_QUEUE", "spark.yarn.queue"), + ("--name", "SPARK_YARN_APP_NAME", "spark.app.name"), + ("--files", "SPARK_YARN_DIST_FILES", "spark.yarn.dist.files"), + ("--archives", "SPARK_YARN_DIST_ARCHIVES", "spark.yarn.dist.archives")) + .foreach { case (optName, envVar, sysProp) => addArg(optName, envVar, sysProp, argsArrayBuf) } + logDebug("ClientArguments called with: " + argsArrayBuf) val args = new ClientArguments(argsArrayBuf.toArray, conf) client = new Client(args, conf) From af3746ce0d724dc624658a2187bde188ab26d084 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Sat, 29 Mar 2014 15:12:43 -0700 Subject: [PATCH 157/397] Implement the RLike & Like in catalyst This PR includes: 1) Unify the unit test for expression evaluation 2) Add implementation of RLike & Like Author: Cheng Hao Closes #224 from chenghao-intel/string_expression and squashes the following commits: 84f72e9 [Cheng Hao] fix bug in RLike/Like & Simplify the unit test aeeb1d7 [Cheng Hao] Simplify the implementation/unit test of RLike/Like 319edb7 [Cheng Hao] change to spark code style 91cfd33 [Cheng Hao] add implementation for rlike/like 2c8929e [Cheng Hao] Update the unit test for expression evaluation --- .../apache/spark/sql/catalyst/SqlParser.scala | 6 ++ .../spark/sql/catalyst/dsl/package.scala | 10 +- .../expressions/stringOperations.scala | 98 ++++++++++++++++++- .../ExpressionEvaluationSuite.scala | 83 ++++++++++++++++ .../org/apache/spark/sql/hive/HiveQl.scala | 9 +- 5 files changed, 196 insertions(+), 10 deletions(-) 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 index 9dec4e3d9e4c2..0c851c2ee2183 100644 --- 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 @@ -114,6 +114,9 @@ class SqlParser extends StandardTokenParsers { protected val NULL = Keyword("NULL") protected val ON = Keyword("ON") protected val OR = Keyword("OR") + protected val LIKE = Keyword("LIKE") + protected val RLIKE = Keyword("RLIKE") + protected val REGEXP = Keyword("REGEXP") protected val ORDER = Keyword("ORDER") protected val OUTER = Keyword("OUTER") protected val RIGHT = Keyword("RIGHT") @@ -267,6 +270,9 @@ class SqlParser extends StandardTokenParsers { 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 ~ RLIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | + termExpression ~ REGEXP ~ termExpression ^^ { case e1 ~ _ ~ e2 => RLike(e1, e2) } | + termExpression ~ LIKE ~ termExpression ^^ { case e1 ~ _ ~ e2 => Like(e1, e2) } | termExpression ~ IN ~ "(" ~ rep1sep(termExpression, ",") <~ ")" ^^ { case e1 ~ _ ~ _ ~ e2 => In(e1, e2) } | 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 67cddb351c185..44abe671c07a4 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 @@ -70,6 +70,9 @@ package object dsl { def === (other: Expression) = Equals(expr, other) def != (other: Expression) = Not(Equals(expr, other)) + def like(other: Expression) = Like(expr, other) + def rlike(other: Expression) = RLike(expr, other) + def asc = SortOrder(expr, Ascending) def desc = SortOrder(expr, Descending) @@ -90,7 +93,10 @@ package object dsl { 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 DslString(val s: String) extends ImplicitOperators { + def expr: Expression = Literal(s) + def attr = analysis.UnresolvedAttribute(s) + } abstract class ImplicitAttribute extends ImplicitOperators { def s: String @@ -110,6 +116,8 @@ package object dsl { // Protobuf terminology def required = a.withNullability(false) + + def at(ordinal: Int) = BoundReference(ordinal, a) } } 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 index e195f2ac7efd1..42b7a9b125b7a 100644 --- 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 @@ -17,11 +17,103 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.regex.Pattern + +import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.catalyst.types.BooleanType +import org.apache.spark.sql.catalyst.trees.TreeNode +import org.apache.spark.sql.catalyst.errors.`package`.TreeNodeException + + +trait StringRegexExpression { + self: BinaryExpression => + + type EvaluatedType = Any + + def escape(v: String): String + def matches(regex: Pattern, str: String): Boolean + + def nullable: Boolean = true + def dataType: DataType = BooleanType + + // try cache the pattern for Literal + private lazy val cache: Pattern = right match { + case x @ Literal(value: String, StringType) => compile(value) + case _ => null + } + + protected def compile(str: String): Pattern = if(str == null) { + null + } else { + // Let it raise exception if couldn't compile the regex string + Pattern.compile(escape(str)) + } -case class Like(left: Expression, right: Expression) extends BinaryExpression { - def dataType = BooleanType - def nullable = left.nullable // Right cannot be null. + protected def pattern(str: String) = if(cache == null) compile(str) else cache + + override def apply(input: Row): Any = { + val l = left.apply(input) + if(l == null) { + null + } else { + val r = right.apply(input) + if(r == null) { + null + } else { + val regex = pattern(r.asInstanceOf[String]) + if(regex == null) { + null + } else { + matches(regex, l.asInstanceOf[String]) + } + } + } + } +} + +/** + * Simple RegEx pattern matching function + */ +case class Like(left: Expression, right: Expression) + extends BinaryExpression with StringRegexExpression { + def symbol = "LIKE" + + // replace the _ with .{1} exactly match 1 time of any character + // replace the % with .*, match 0 or more times with any character + override def escape(v: String) = { + val sb = new StringBuilder() + var i = 0; + while (i < v.length) { + // Make a special case for "\\_" and "\\%" + val n = v.charAt(i); + if (n == '\\' && i + 1 < v.length && (v.charAt(i + 1) == '_' || v.charAt(i + 1) == '%')) { + sb.append(v.charAt(i + 1)) + i += 1 + } else { + if (n == '_') { + sb.append("."); + } else if (n == '%') { + sb.append(".*"); + } else { + sb.append(Pattern.quote(Character.toString(n))); + } + } + + i += 1 + } + + sb.toString() + } + + override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() } +case class RLike(left: Expression, right: Expression) + extends BinaryExpression with StringRegexExpression { + + def symbol = "RLIKE" + override def escape(v: String): String = v + override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) +} diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 94894adf81202..52a205be3e9f4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -109,4 +109,87 @@ class ExpressionEvaluationSuite extends FunSuite { } } } + + def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { + expression.apply(inputRow) + } + + def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { + val actual = try evaluate(expression, inputRow) catch { + case e: Exception => fail(s"Exception evaluating $expression", e) + } + if(actual != expected) { + val input = if(inputRow == EmptyRow) "" else s", input: $inputRow" + fail(s"Incorrect Evaluation: $expression, actual: $actual, expected: $expected$input") + } + } + + test("LIKE literal Regular Expression") { + checkEvaluation(Literal(null, StringType).like("a"), null) + checkEvaluation(Literal(null, StringType).like(Literal(null, StringType)), null) + checkEvaluation("abdef" like "abdef", true) + checkEvaluation("a_%b" like "a\\__b", true) + checkEvaluation("addb" like "a_%b", true) + checkEvaluation("addb" like "a\\__b", false) + checkEvaluation("addb" like "a%\\%b", false) + checkEvaluation("a_%b" like "a%\\%b", true) + checkEvaluation("addb" like "a%", true) + checkEvaluation("addb" like "**", false) + checkEvaluation("abc" like "a%", true) + checkEvaluation("abc" like "b%", false) + checkEvaluation("abc" like "bc%", false) + } + + test("LIKE Non-literal Regular Expression") { + val regEx = 'a.string.at(0) + checkEvaluation("abcd" like regEx, null, new GenericRow(Array[Any](null))) + checkEvaluation("abdef" like regEx, true, new GenericRow(Array[Any]("abdef"))) + checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a\\__b"))) + checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a_%b"))) + checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a\\__b"))) + checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("a%\\%b"))) + checkEvaluation("a_%b" like regEx, true, new GenericRow(Array[Any]("a%\\%b"))) + checkEvaluation("addb" like regEx, true, new GenericRow(Array[Any]("a%"))) + checkEvaluation("addb" like regEx, false, new GenericRow(Array[Any]("**"))) + checkEvaluation("abc" like regEx, true, new GenericRow(Array[Any]("a%"))) + checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("b%"))) + checkEvaluation("abc" like regEx, false, new GenericRow(Array[Any]("bc%"))) + } + + test("RLIKE literal Regular Expression") { + checkEvaluation("abdef" rlike "abdef", true) + checkEvaluation("abbbbc" rlike "a.*c", true) + + checkEvaluation("fofo" rlike "^fo", true) + checkEvaluation("fo\no" rlike "^fo\no$", true) + checkEvaluation("Bn" rlike "^Ba*n", true) + checkEvaluation("afofo" rlike "fo", true) + checkEvaluation("afofo" rlike "^fo", false) + checkEvaluation("Baan" rlike "^Ba?n", false) + checkEvaluation("axe" rlike "pi|apa", false) + checkEvaluation("pip" rlike "^(pi)*$", false) + + checkEvaluation("abc" rlike "^ab", true) + checkEvaluation("abc" rlike "^bc", false) + checkEvaluation("abc" rlike "^ab", true) + checkEvaluation("abc" rlike "^bc", false) + + intercept[java.util.regex.PatternSyntaxException] { + evaluate("abbbbc" rlike "**") + } + } + + test("RLIKE Non-literal Regular Expression") { + val regEx = 'a.string.at(0) + checkEvaluation("abdef" rlike regEx, true, new GenericRow(Array[Any]("abdef"))) + checkEvaluation("abbbbc" rlike regEx, true, new GenericRow(Array[Any]("a.*c"))) + checkEvaluation("fofo" rlike regEx, true, new GenericRow(Array[Any]("^fo"))) + checkEvaluation("fo\no" rlike regEx, true, new GenericRow(Array[Any]("^fo\no$"))) + checkEvaluation("Bn" rlike regEx, true, new GenericRow(Array[Any]("^Ba*n"))) + + intercept[java.util.regex.PatternSyntaxException] { + evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) + } + } } + 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 b70ec897e43e7..490a592a588d0 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 @@ -847,12 +847,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("LIKE", left :: right:: Nil) => Like(nodeToExpr(left), nodeToExpr(right)) + case Token("RLIKE", left :: right:: Nil) => RLike(nodeToExpr(left), nodeToExpr(right)) + case Token("REGEXP", left :: right:: Nil) => RLike(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) => From fda86d8b46a1cc484d11ac5446d8cc2a86429b9b Mon Sep 17 00:00:00 2001 From: Bernardo Gomez Palacio Date: Sat, 29 Mar 2014 19:49:22 -0700 Subject: [PATCH 158/397] [SPARK-1186] : Enrich the Spark Shell to support additional arguments. Enrich the Spark Shell functionality to support the following options. ``` Usage: spark-shell [OPTIONS] OPTIONS: -h --help : Print this help information. -c --cores : The maximum number of cores to be used by the Spark Shell. -em --executor-memory : The memory used by each executor of the Spark Shell, the number is followed by m for megabytes or g for gigabytes, e.g. "1g". -dm --driver-memory : The memory used by the Spark Shell, the number is followed by m for megabytes or g for gigabytes, e.g. "1g". -m --master : A full string that describes the Spark Master, defaults to "local" e.g. "spark://localhost:7077". --log-conf : Enables logging of the supplied SparkConf as INFO at start of the Spark Context. e.g. spark-shell -m spark://localhost:7077 -c 4 -dm 512m -em 2g ``` **Note**: this commit reflects the changes applied to _master_ based on [5d98cfc1]. [ticket: SPARK-1186] : Enrich the Spark Shell to support additional arguments. https://spark-project.atlassian.net/browse/SPARK-1186 Author : bernardo.gomezpalcio@gmail.com Author: Bernardo Gomez Palacio Closes #116 from berngp/feature/enrich-spark-shell and squashes the following commits: c5f455f [Bernardo Gomez Palacio] [SPARK-1186] : Enrich the Spark Shell to support additional arguments. --- bin/spark-shell | 226 +++++++++++++++++++++++++++++++++++------------- 1 file changed, 168 insertions(+), 58 deletions(-) diff --git a/bin/spark-shell b/bin/spark-shell index 861ab606540cd..fac006cf492ed 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -30,67 +30,189 @@ esac # Enter posix mode for bash set -o posix -CORE_PATTERN="^[0-9]+$" -MEM_PATTERN="^[0-9]+[m|g|M|G]$" - +## Global script variables FWDIR="$(cd `dirname $0`/..; pwd)" -if [ "$1" = "--help" ] || [ "$1" = "-h" ]; then - echo "Usage: spark-shell [OPTIONS]" - echo "OPTIONS:" - echo "-c --cores num, the maximum number of cores to be used by the spark shell" - echo "-em --execmem num[m|g], the memory used by each executor of spark shell" - echo "-dm --drivermem num[m|g], the memory used by the spark shell and driver" - echo "-h --help, print this help information" - exit -fi +SPARK_REPL_OPTS="${SPARK_REPL_OPTS:-""}" +DEFAULT_MASTER="local" +MASTER=${MASTER:-""} + +info_log=0 + +#CLI Color Templates +txtund=$(tput sgr 0 1) # Underline +txtbld=$(tput bold) # Bold +bldred=${txtbld}$(tput setaf 1) # red +bldyel=${txtbld}$(tput setaf 3) # yellow +bldblu=${txtbld}$(tput setaf 4) # blue +bldwht=${txtbld}$(tput setaf 7) # white +txtrst=$(tput sgr0) # Reset +info=${bldwht}*${txtrst} # Feedback +pass=${bldblu}*${txtrst} +warn=${bldred}*${txtrst} +ques=${bldblu}?${txtrst} + +# Helper function to describe the script usage +function usage() { + cat << EOF +${txtbld}Usage${txtrst}: spark-shell [OPTIONS] + +${txtbld}OPTIONS${txtrst}: + -h --help : Print this help information. + -c --cores : The maximum number of cores to be used by the Spark Shell. + -em --executor-memory : The memory used by each executor of the Spark Shell, the number + is followed by m for megabytes or g for gigabytes, e.g. "1g". + -dm --driver-memory : The memory used by the Spark Shell, the number is followed + by m for megabytes or g for gigabytes, e.g. "1g". + -m --master : A full string that describes the Spark Master, defaults to "local" + e.g. "spark://localhost:7077". + --log-conf : Enables logging of the supplied SparkConf as INFO at start of the + Spark Context. + +e.g. + spark-shell -m spark://localhost:7077 -c 4 -dm 512m -em 2g + +EOF +} + +function out_error(){ + echo -e "${txtund}${bldred}ERROR${txtrst}: $1" + usage + exit 1 +} + +function log_info(){ + [ $info_log -eq 1 ] && echo -e "${bldyel}INFO${txtrst}: $1" +} + +function log_warn(){ + echo -e "${txtund}${bldyel}WARN${txtrst}: $1" +} -for o in "$@"; do - if [ "$1" = "-c" -o "$1" = "--cores" ]; then - shift +# PATTERNS used to validate more than one optional arg. +ARG_FLAG_PATTERN="^-" +MEM_PATTERN="^[0-9]+[m|g|M|G]$" +NUM_PATTERN="^[0-9]+$" +PORT_PATTERN="^[0-9]+$" + +# Setters for optional args. +function set_cores(){ + CORE_PATTERN="^[0-9]+$" if [[ "$1" =~ $CORE_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1" - shift + SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1" else - echo "ERROR: wrong format for -c/--cores" - exit 1 + out_error "wrong format for $2" fi - fi - if [ "$1" = "-em" -o "$1" = "--execmem" ]; then - shift +} + +function set_em(){ if [[ $1 =~ $MEM_PATTERN ]]; then SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.executor.memory=$1" - shift else - echo "ERROR: wrong format for --execmem/-em" - exit 1 + out_error "wrong format for $2" fi - fi - if [ "$1" = "-dm" -o "$1" = "--drivermem" ]; then - shift +} + +function set_dm(){ if [[ $1 =~ $MEM_PATTERN ]]; then export SPARK_DRIVER_MEMORY=$1 - shift else - echo "ERROR: wrong format for --drivermem/-dm" - exit 1 + out_error "wrong format for $2" fi - fi -done +} + +function set_spark_log_conf(){ + SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.logConf=$1" +} -# Set MASTER from spark-env if possible -DEFAULT_SPARK_MASTER_PORT=7077 -if [ -z "$MASTER" ]; then - . $FWDIR/bin/load-spark-env.sh - if [ "x" != "x$SPARK_MASTER_IP" ]; then - if [ "y" != "y$SPARK_MASTER_PORT" ]; then - SPARK_MASTER_PORT="${SPARK_MASTER_PORT}" +function set_spark_master(){ + if ! [[ "$1" =~ $ARG_FLAG_PATTERN ]]; then + MASTER="$1" else - SPARK_MASTER_PORT=$DEFAULT_SPARK_MASTER_PORT + out_error "wrong format for $2" + fi +} + +function resolve_spark_master(){ + # Set MASTER from spark-env if possible + DEFAULT_SPARK_MASTER_PORT=7077 + if [ -z "$MASTER" ]; then + . $FWDIR/bin/load-spark-env.sh + if [ -n "$SPARK_MASTER_IP" ]; then + SPARK_MASTER_PORT="${SPARK_MASTER_PORT:-"$DEFAULT_SPARK_MASTER_PORT"}" + export MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" + fi + fi + + if [ -z "$MASTER" ]; then + MASTER="$DEFAULT_MASTER" fi - export MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - fi -fi + +} + +function main(){ + log_info "Base Directory set to $FWDIR" + + resolve_spark_master + log_info "Spark Master is $MASTER" + + log_info "Spark REPL options $SPARK_REPL_OPTS" + if $cygwin; then + # Workaround for issue involving JLine and Cygwin + # (see http://sourceforge.net/p/jline/bugs/40/). + # If you're using the Mintty terminal emulator in Cygwin, may need to set the + # "Backspace sends ^H" setting in "Keys" section of the Mintty options + # (see https://github.com/sbt/sbt/issues/562). + stty -icanon min 1 -echo > /dev/null 2>&1 + export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" + $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + stty icanon echo > /dev/null 2>&1 + else + export SPARK_REPL_OPTS + $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + fi +} + +for option in "$@" +do + case $option in + -h | --help ) + usage + exit 1 + ;; + -c | --cores) + shift + _1=$1 + shift + set_cores $_1 "-c/--cores" + ;; + -em | --executor-memory) + shift + _1=$1 + shift + set_em $_1 "-em/--executor-memory" + ;; + -dm | --driver-memory) + shift + _1=$1 + shift + set_dm $_1 "-dm/--driver-memory" + ;; + -m | --master) + shift + _1=$1 + shift + set_spark_master $_1 "-m/--master" + ;; + --log-conf) + shift + set_spark_log_conf "true" + info_log=1 + ;; + ?) + ;; + esac +done # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed @@ -120,22 +242,10 @@ if [[ ! $? ]]; then saved_stty="" fi -if $cygwin; then - # Workaround for issue involving JLine and Cygwin - # (see http://sourceforge.net/p/jline/bugs/40/). - # If you're using the Mintty terminal emulator in Cygwin, may need to set the - # "Backspace sends ^H" setting in "Keys" section of the Mintty options - # (see https://github.com/sbt/sbt/issues/562). - stty -icanon min 1 -echo > /dev/null 2>&1 - export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" - stty icanon echo > /dev/null 2>&1 -else - export SPARK_REPL_OPTS - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" -fi +main # record the exit status lest it be overwritten: # then reenable echo and propagate the code. exit_status=$? onExit + From 92b83959cacbc902ff0b50110261f097bf2df247 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 29 Mar 2014 22:01:29 -0700 Subject: [PATCH 159/397] Don't swallow all kryo errors, only those that indicate we are out of data. Author: Michael Armbrust Closes #142 from marmbrus/kryoErrors and squashes the following commits: 9c72d1f [Michael Armbrust] Make the test more future proof. 78f5a42 [Michael Armbrust] Don't swallow all kryo errors, only those that indicate we are out of data. --- .../scala/org/apache/spark/serializer/KryoSerializer.scala | 3 ++- 1 file changed, 2 insertions(+), 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 6b6d814c1fe92..926e71573be32 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -107,7 +107,8 @@ 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.toLowerCase.contains("buffer underflow") => + throw new EOFException } } From 2861b07bb030f72769f5b757b4a7d4a635807140 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Sat, 29 Mar 2014 22:02:53 -0700 Subject: [PATCH 160/397] [SQL] SPARK-1354 Fix self-joins of parquet relations @AndreSchumacher, please take a look. https://spark-project.atlassian.net/browse/SPARK-1354 Author: Michael Armbrust Closes #269 from marmbrus/parquetJoin and squashes the following commits: 4081e77 [Michael Armbrust] Create new instances of Parquet relation when multiple copies are in a single plan. --- .../spark/sql/parquet/ParquetRelation.scala | 15 +++++++++++++-- .../spark/sql/parquet/ParquetQuerySuite.scala | 19 +++++++++++++++++++ 2 files changed, 32 insertions(+), 2 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 2b825f84ee910..67a34e1f21cc7 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 @@ -36,7 +36,7 @@ import parquet.schema.{MessageType, MessageTypeParser} import parquet.schema.{PrimitiveType => ParquetPrimitiveType} import parquet.schema.{Type => ParquetType} -import org.apache.spark.sql.catalyst.analysis.UnresolvedException +import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} import org.apache.spark.sql.catalyst.types._ @@ -54,7 +54,8 @@ import org.apache.spark.sql.catalyst.types._ * @param tableName The name of the relation that can be used in queries. * @param path The path to the Parquet file. */ -case class ParquetRelation(tableName: String, path: String) extends BaseRelation { +case class ParquetRelation(tableName: String, path: String) + extends BaseRelation with MultiInstanceRelation { /** Schema derived from ParquetFile **/ def parquetSchema: MessageType = @@ -74,6 +75,16 @@ case class ParquetRelation(tableName: String, path: String) extends BaseRelation // 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 + + override def newInstance = ParquetRelation(tableName, path).asInstanceOf[this.type] + + // Equals must also take into account the output attributes so that we can distinguish between + // different instances of the same relation, + override def equals(other: Any) = other match { + case p: ParquetRelation => + p.tableName == tableName && p.path == path && p.output == output + case _ => false + } } object ParquetRelation { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index 71caa709afca6..ea1733b3614e5 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -30,6 +30,9 @@ import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.util.getTempFilePath import org.apache.spark.sql.test.TestSQLContext +// Implicits +import org.apache.spark.sql.test.TestSQLContext._ + class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { override def beforeAll() { ParquetTestData.writeFile() @@ -39,6 +42,22 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { ParquetTestData.testFile.delete() } + test("self-join parquet files") { + val x = ParquetTestData.testData.subquery('x) + val y = ParquetTestData.testData.subquery('y) + val query = x.join(y).where("x.myint".attr === "y.myint".attr) + + // Check to make sure that the attributes from either side of the join have unique expression + // ids. + query.queryExecution.analyzed.output.filter(_.name == "myint") match { + case Seq(i1, i2) if(i1.exprId == i2.exprId) => + fail(s"Duplicate expression IDs found in query plan: $query") + case Seq(_, _) => // All good + } + + // TODO: We can't run this query as it NPEs + } + test("Import of simple Parquet file") { val result = getRDD(ParquetTestData.testData).collect() assert(result.size === 15) From df1b9f7b1a07bf8d806695a7684f9d69bf705093 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sat, 29 Mar 2014 23:03:03 -0700 Subject: [PATCH 161/397] SPARK-1336 Reducing the output of run-tests script. Author: Prashant Sharma Author: Prashant Sharma Closes #262 from ScrapCodes/SPARK-1336/ReduceVerbosity and squashes the following commits: 87dfa54 [Prashant Sharma] Further reduction in noise and made pyspark tests to fail fast. 811170f [Prashant Sharma] Reducing the ouput of run-tests script. --- .gitignore | 1 + dev/run-tests | 7 +++---- dev/scalastyle | 27 +++++++++++++++++++++++++++ python/run-tests | 19 ++++++++++++------- 4 files changed, 43 insertions(+), 11 deletions(-) create mode 100755 dev/scalastyle diff --git a/.gitignore b/.gitignore index 3a68abd955b22..cd9f90d55932c 100644 --- a/.gitignore +++ b/.gitignore @@ -47,3 +47,4 @@ spark-*-bin.tar.gz unit-tests.log /lib/ rat-results.txt +scalastyle.txt diff --git a/dev/run-tests b/dev/run-tests index 6f115d2abd5b0..a6fcc40a5ba6e 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -39,18 +39,17 @@ JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..* echo "=========================================================================" echo "Running Apache RAT checks" echo "=========================================================================" - dev/check-license echo "=========================================================================" echo "Running Scala style checks" echo "=========================================================================" -sbt/sbt clean scalastyle +dev/scalastyle echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -sbt/sbt assembly test +sbt/sbt assembly test | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" echo "=========================================================================" echo "Running PySpark tests" @@ -64,5 +63,5 @@ echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore -sbt/sbt mima-report-binary-issues +sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" diff --git a/dev/scalastyle b/dev/scalastyle new file mode 100755 index 0000000000000..5a18f4d672825 --- /dev/null +++ b/dev/scalastyle @@ -0,0 +1,27 @@ +#!/usr/bin/env 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. +# + +sbt/sbt clean scalastyle > scalastyle.txt +ERRORS=$(cat scalastyle.txt | grep -e "error file") +if test ! -z "$ERRORS"; then + echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" + exit 1 +else + echo -e "Scalastyle checks passed.\n" +fi diff --git a/python/run-tests b/python/run-tests index a986ac9380be4..b2b60f08b48e2 100755 --- a/python/run-tests +++ b/python/run-tests @@ -29,8 +29,18 @@ FAILED=0 rm -f unit-tests.log function run_test() { - SPARK_TESTING=0 $FWDIR/bin/pyspark $1 2>&1 | tee -a unit-tests.log + SPARK_TESTING=0 $FWDIR/bin/pyspark $1 2>&1 | tee -a > unit-tests.log FAILED=$((PIPESTATUS[0]||$FAILED)) + + # Fail and exit on the first test failure. + if [[ $FAILED != 0 ]]; then + cat unit-tests.log | grep -v "^[0-9][0-9]*" # filter all lines starting with a number. + echo -en "\033[31m" # Red + echo "Had test failures; see logs." + echo -en "\033[0m" # No color + exit -1 + fi + } run_test "pyspark/rdd.py" @@ -46,12 +56,7 @@ run_test "pyspark/mllib/clustering.py" run_test "pyspark/mllib/recommendation.py" run_test "pyspark/mllib/regression.py" -if [[ $FAILED != 0 ]]; then - echo -en "\033[31m" # Red - echo "Had test failures; see logs." - echo -en "\033[0m" # No color - exit -1 -else +if [[ $FAILED == 0 ]]; then echo -en "\033[32m" # Green echo "Tests passed." echo -en "\033[0m" # No color From 95d7d2a3fc2adc0bbca90d015c6ca319fffb26aa Mon Sep 17 00:00:00 2001 From: jerryshao Date: Sun, 30 Mar 2014 10:03:58 -0700 Subject: [PATCH 162/397] [SPARK-1354][SQL] Add tableName as a qualifier for SimpleCatelogy Fix attribute unresolved when query with table name as a qualifier in SQLContext with SimplCatelog, details please see [SPARK-1354](https://issues.apache.org/jira/browse/SPARK-1354?jql=project%20%3D%20SPARK). Author: jerryshao Closes #272 from jerryshao/qualifier-fix and squashes the following commits: 7950170 [jerryshao] Add tableName as a qualifier for SimpleCatelogy --- .../spark/sql/catalyst/analysis/Catalog.scala | 3 ++- .../org/apache/spark/sql/SQLQuerySuite.scala | 27 +++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) 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 ff66177a03b8c..e09182dd8d5df 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 @@ -47,10 +47,11 @@ class SimpleCatalog extends Catalog { tableName: String, alias: Option[String] = None): LogicalPlan = { val table = tables.get(tableName).getOrElse(sys.error(s"Table Not Found: $tableName")) + val tableWithQualifiers = Subquery(tableName, table) // 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) + alias.map(a => Subquery(a.toLowerCase, tableWithQualifiers)).getOrElse(tableWithQualifiers) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index fa4a1d5189ea6..4c4fd6dbbedb4 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -216,4 +216,31 @@ class SQLQuerySuite extends QueryTest { (null, null, 5, "E") :: (null, null, 6, "F") :: Nil) } + + test("select with table name as qualifier") { + checkAnswer( + sql("SELECT testData.value FROM testData WHERE testData.key = 1"), + Seq(Seq("1"))) + } + + test("inner join ON with table name as qualifier") { + checkAnswer( + sql("SELECT * FROM upperCaseData JOIN lowerCaseData ON lowerCaseData.n = upperCaseData.N"), + Seq( + (1, "A", 1, "a"), + (2, "B", 2, "b"), + (3, "C", 3, "c"), + (4, "D", 4, "d"))) + } + + test("qualified select with inner join ON with table name as qualifier") { + checkAnswer( + sql("SELECT upperCaseData.N, upperCaseData.L FROM upperCaseData JOIN lowerCaseData " + + "ON lowerCaseData.n = upperCaseData.N"), + Seq( + (1, "A"), + (2, "B"), + (3, "C"), + (4, "D"))) + } } From d666053679ded5a32088c1758d20101126e23af6 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Sun, 30 Mar 2014 10:06:56 -0700 Subject: [PATCH 163/397] SPARK-1352 - Comment style single space before ending */ check. Author: Prashant Sharma Closes #261 from ScrapCodes/comment-style-check2 and squashes the following commits: 6cde61e [Prashant Sharma] comment style space before ending */ check. --- .../org/apache/spark/network/Connection.scala | 12 ++++++------ .../apache/spark/network/ConnectionManager.scala | 14 +++++++------- .../spark/network/ConnectionManagerTest.scala | 4 ++-- .../org/apache/spark/network/ReceiverTest.scala | 2 +- .../org/apache/spark/network/SenderTest.scala | 2 +- .../scala/org/apache/spark/ui/jobs/IndexPage.scala | 2 +- .../scala/org/apache/spark/util/MutablePair.scala | 4 ++-- .../examples/clickstream/PageViewGenerator.scala | 2 +- .../spark/streaming/flume/FlumeInputDStream.scala | 2 +- .../spark/graphx/impl/MessageToPartition.scala | 2 +- project/project/SparkPluginBuild.scala | 3 +-- ...la => SparkSpaceAfterCommentStartChecker.scala} | 8 +++++--- .../apache/spark/sql/parquet/ParquetRelation.scala | 6 +++--- 13 files changed, 32 insertions(+), 31 deletions(-) rename project/spark-style/src/main/scala/org/apache/spark/scalastyle/{SparkSpaceAfterCommentStyleCheck.scala => SparkSpaceAfterCommentStartChecker.scala} (89%) diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 16bd00fd189ff..2f7576c53b482 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -211,7 +211,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, def addMessage(message: Message) { messages.synchronized{ - /* messages += message*/ + /* messages += message */ messages.enqueue(message) logDebug("Added [" + message + "] to outbox for sending to " + "[" + getRemoteConnectionManagerId() + "]") @@ -222,7 +222,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, messages.synchronized { while (!messages.isEmpty) { /* nextMessageToBeUsed = nextMessageToBeUsed % messages.size */ - /* val message = messages(nextMessageToBeUsed)*/ + /* val message = messages(nextMessageToBeUsed) */ val message = messages.dequeue val chunk = message.getChunkForSending(defaultChunkSize) if (chunk.isDefined) { @@ -262,7 +262,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, val currentBuffers = new ArrayBuffer[ByteBuffer]() - /* channel.socket.setSendBufferSize(256 * 1024)*/ + /* channel.socket.setSendBufferSize(256 * 1024) */ override def getRemoteAddress() = address @@ -355,7 +355,7 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } case None => { // changeConnectionKeyInterest(0) - /* key.interestOps(0)*/ + /* key.interestOps(0) */ return false } } @@ -540,10 +540,10 @@ private[spark] class ReceivingConnection( return false } - /* logDebug("Read " + bytesRead + " bytes for the buffer")*/ + /* logDebug("Read " + bytesRead + " bytes for the buffer") */ if (currentChunk.buffer.remaining == 0) { - /* println("Filled buffer at " + System.currentTimeMillis)*/ + /* println("Filled buffer at " + System.currentTimeMillis) */ val bufferMessage = inbox.getMessageForChunk(currentChunk).get if (bufferMessage.isCompletelyReceived) { bufferMessage.flip diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 2682f9d0ed7f0..6b0a972f0bbe0 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -505,7 +505,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } handleMessageExecutor.execute(runnable) - /* handleMessage(connection, message)*/ + /* handleMessage(connection, message) */ } private def handleClientAuthentication( @@ -859,14 +859,14 @@ private[spark] object ConnectionManager { None }) - /* testSequentialSending(manager)*/ - /* System.gc()*/ + /* testSequentialSending(manager) */ + /* System.gc() */ - /* testParallelSending(manager)*/ - /* System.gc()*/ + /* testParallelSending(manager) */ + /* System.gc() */ - /* testParallelDecreasingSending(manager)*/ - /* System.gc()*/ + /* testParallelDecreasingSending(manager) */ + /* System.gc() */ testContinuousSending(manager) System.gc() diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index e5745d7daa153..9d9b9dbdd5331 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -47,8 +47,8 @@ private[spark] object ConnectionManagerTest extends Logging{ val slaves = slavesFile.mkString.split("\n") slavesFile.close() - /* println("Slaves")*/ - /* slaves.foreach(println)*/ + /* println("Slaves") */ + /* slaves.foreach(println) */ val tasknum = if (args.length > 2) args(2).toInt else slaves.length val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 val count = if (args.length > 4) args(4).toInt else 3 diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 17fd931c9f075..2b41c403b2e0a 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -27,7 +27,7 @@ private[spark] object ReceiverTest { println("Started connection manager with id = " + manager.id) manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { - /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis)*/ + /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */ val buffer = ByteBuffer.wrap("response".getBytes) Some(Message.createBufferMessage(buffer, msg.id)) }) diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 905eddfbb9450..14c094c6177d5 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -50,7 +50,7 @@ private[spark] object SenderTest { (0 until count).foreach(i => { val dataMessage = Message.createBufferMessage(buffer.duplicate) val startTime = System.currentTimeMillis - /* println("Started timer at " + startTime)*/ + /* println("Started timer at " + startTime) */ val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index f3c93d4214ad0..70d62b66a4829 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -25,7 +25,7 @@ import org.apache.spark.scheduler.Schedulable import org.apache.spark.ui.Page._ import org.apache.spark.ui.UIUtils -/** Page showing list of all ongoing and recently finished stages and pools*/ +/** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class IndexPage(parent: JobProgressUI) { private val appName = parent.appName private val basePath = parent.basePath 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 a898824cff0ca..a6b39247a54ca 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -24,8 +24,8 @@ package org.apache.spark.util * @param _1 Element 1 of this MutablePair * @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] +case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, + @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) extends Product2[T1, T2] { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala index 0ac46c31c24c8..251f65fe4df9c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewGenerator.scala @@ -21,7 +21,7 @@ import java.net.ServerSocket import java.io.PrintWriter import util.Random -/** Represents a page view on a website with associated dimension data.*/ +/** Represents a page view on a website with associated dimension data. */ class PageView(val url : String, val status : Int, val zipCode : Int, val userID : Int) extends Serializable { override def toString() : String = { diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index ce3ef47cfe4bc..34012b846e21e 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -127,7 +127,7 @@ class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { } /** A NetworkReceiver which listens for events using the - * Flume Avro interface.*/ + * Flume Avro interface. */ private[streaming] class FlumeReceiver( host: String, diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index bebe3740bc6c0..9d4f3750cb8e4 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -45,7 +45,7 @@ class VertexBroadcastMsg[@specialized(Int, Long, Double, Boolean) T]( * @param data value to send */ private[graphx] -class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef*/) T]( +class MessageToPartition[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T]( @transient var partition: PartitionID, var data: T) extends Product2[PartitionID, T] with Serializable { diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala index 43361aa2b4c41..5a307044ba123 100644 --- a/project/project/SparkPluginBuild.scala +++ b/project/project/SparkPluginBuild.scala @@ -34,8 +34,7 @@ object SparkPluginDef extends Build { version := sparkVersion, scalaVersion := "2.10.3", scalacOptions := Seq("-unchecked", "-deprecation"), - libraryDependencies ++= Dependencies.scalaStyle, - sbtPlugin := true + libraryDependencies ++= Dependencies.scalaStyle ) object Dependencies { diff --git a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala similarity index 89% rename from project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala rename to project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala index 2f3c1a182814d..80d3faa3fe749 100644 --- a/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStyleCheck.scala +++ b/project/spark-style/src/main/scala/org/apache/spark/scalastyle/SparkSpaceAfterCommentStartChecker.scala @@ -25,13 +25,15 @@ import scalariform.lexer.{MultiLineComment, ScalaDocComment, SingleLineComment, import scalariform.parser.CompilationUnit class SparkSpaceAfterCommentStartChecker extends ScalariformChecker { - val errorKey: String = "insert.a.single.space.after.comment.start" + val errorKey: String = "insert.a.single.space.after.comment.start.and.before.end" private def multiLineCommentRegex(comment: Token) = - Pattern.compile( """/\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() + Pattern.compile( """/\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() || + Pattern.compile( """/\*.*\S\*/""", Pattern.DOTALL).matcher(comment.text.trim).matches() private def scalaDocPatternRegex(comment: Token) = - Pattern.compile( """/\*\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() + Pattern.compile( """/\*\*\S+.*""", Pattern.DOTALL).matcher(comment.text.trim).matches() || + Pattern.compile( """/\*\*.*\S\*/""", Pattern.DOTALL).matcher(comment.text.trim).matches() private def singleLineCommentRegex(comment: Token): Boolean = comment.text.trim.matches( """//\S+.*""") && !comment.text.trim.matches( """///+""") 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 67a34e1f21cc7..4ab755c096bd8 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 @@ -57,19 +57,19 @@ import org.apache.spark.sql.catalyst.types._ case class ParquetRelation(tableName: String, path: String) extends BaseRelation with MultiInstanceRelation { - /** Schema derived from ParquetFile **/ + /** Schema derived from ParquetFile */ def parquetSchema: MessageType = ParquetTypesConverter .readMetaData(new Path(path)) .getFileMetaData .getSchema - /** Attributes **/ + /** Attributes */ val attributes = ParquetTypesConverter .convertToAttributes(parquetSchema) - /** Output **/ + /** Output */ override val output = attributes // Parquet files have no concepts of keys, therefore no Partitioner From 841721e03cc44ee7d8fe72c882db8c0f9f3af365 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 31 Mar 2014 12:07:14 -0700 Subject: [PATCH 164/397] SPARK-1352: Improve robustness of spark-submit script 1. Better error messages when required arguments are missing. 2. Support for unit testing cases where presented arguments are invalid. 3. Bug fix: Only use environment varaibles when they are set (otherwise will cause NPE). 4. A verbose mode to aid debugging. 5. Visibility of several variables is set to private. 6. Deprecation warning for existing scripts. Author: Patrick Wendell Closes #271 from pwendell/spark-submit and squashes the following commits: 9146def [Patrick Wendell] SPARK-1352: Improve robustness of spark-submit script --- .../org/apache/spark/deploy/Client.scala | 3 + .../org/apache/spark/deploy/SparkSubmit.scala | 67 +++++++++++------ .../spark/deploy/SparkSubmitArguments.scala | 74 +++++++++++++------ .../spark/deploy/SparkSubmitSuite.scala | 61 ++++++++++++++- .../org/apache/spark/deploy/yarn/Client.scala | 3 + .../org/apache/spark/deploy/yarn/Client.scala | 3 + 6 files changed, 163 insertions(+), 48 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index d9e3035e1ab59..8fd2c7e95b966 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -128,6 +128,9 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends */ object Client { def main(args: Array[String]) { + println("WARNING: This client is deprecated and will be removed in a future version of Spark.") + println("Use ./bin/spark-submit with \"--master spark://host:port\"") + val conf = new SparkConf() val driverArgs = new ClientArguments(args) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 24a9c98e188f6..1fa799190409f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,7 +17,7 @@ package org.apache.spark.deploy -import java.io.File +import java.io.{PrintStream, File} import java.net.URL import org.apache.spark.executor.ExecutorURLClassLoader @@ -32,38 +32,51 @@ import scala.collection.mutable.Map * modes that Spark supports. */ object SparkSubmit { - val YARN = 1 - val STANDALONE = 2 - val MESOS = 4 - val LOCAL = 8 - val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL + private val YARN = 1 + private val STANDALONE = 2 + private val MESOS = 4 + private val LOCAL = 8 + private val ALL_CLUSTER_MGRS = YARN | STANDALONE | MESOS | LOCAL - var clusterManager: Int = LOCAL + private var clusterManager: Int = LOCAL def main(args: Array[String]) { val appArgs = new SparkSubmitArguments(args) + if (appArgs.verbose) { + printStream.println(appArgs) + } val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) - launch(childArgs, classpath, sysProps, mainClass) + launch(childArgs, classpath, sysProps, mainClass, appArgs.verbose) } + // Exposed for testing + private[spark] var printStream: PrintStream = System.err + private[spark] var exitFn: () => Unit = () => System.exit(-1) + + private[spark] def printErrorAndExit(str: String) = { + printStream.println("error: " + str) + printStream.println("run with --help for more information or --verbose for debugging output") + exitFn() + } + private[spark] def printWarning(str: String) = printStream.println("warning: " + str) + /** * @return * a tuple containing the arguments for the child, a list of classpath * entries for the child, and the main class for the child */ - def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], + private[spark] def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { - if (appArgs.master.startsWith("yarn")) { + if (appArgs.master.startsWith("local")) { + clusterManager = LOCAL + } else if (appArgs.master.startsWith("yarn")) { clusterManager = YARN } else if (appArgs.master.startsWith("spark")) { clusterManager = STANDALONE } else if (appArgs.master.startsWith("mesos")) { clusterManager = MESOS - } else if (appArgs.master.startsWith("local")) { - clusterManager = LOCAL } else { - System.err.println("master must start with yarn, mesos, spark, or local") - System.exit(1) + printErrorAndExit("master must start with yarn, mesos, spark, or local") } // Because "yarn-standalone" and "yarn-client" encapsulate both the master @@ -73,12 +86,10 @@ object SparkSubmit { appArgs.deployMode = "cluster" } if (appArgs.deployMode == "cluster" && appArgs.master == "yarn-client") { - System.err.println("Deploy mode \"cluster\" and master \"yarn-client\" are at odds") - System.exit(1) + printErrorAndExit("Deploy mode \"cluster\" and master \"yarn-client\" are not compatible") } if (appArgs.deployMode == "client" && appArgs.master == "yarn-standalone") { - System.err.println("Deploy mode \"client\" and master \"yarn-standalone\" are at odds") - System.exit(1) + printErrorAndExit("Deploy mode \"client\" and master \"yarn-standalone\" are not compatible") } if (appArgs.deployMode == "cluster" && appArgs.master.startsWith("yarn")) { appArgs.master = "yarn-standalone" @@ -95,8 +106,7 @@ object SparkSubmit { var childMainClass = "" if (clusterManager == MESOS && deployOnCluster) { - System.err.println("Mesos does not support running the driver on the cluster") - System.exit(1) + printErrorAndExit("Mesos does not support running the driver on the cluster") } if (!deployOnCluster) { @@ -174,8 +184,17 @@ object SparkSubmit { (childArgs, childClasspath, sysProps, childMainClass) } - def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], - sysProps: Map[String, String], childMainClass: String) { + private def launch(childArgs: ArrayBuffer[String], childClasspath: ArrayBuffer[String], + sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) { + + if (verbose) { + System.err.println(s"Main class:\n$childMainClass") + System.err.println(s"Arguments:\n${childArgs.mkString("\n")}") + System.err.println(s"System properties:\n${sysProps.mkString("\n")}") + System.err.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") + System.err.println("\n") + } + val loader = new ExecutorURLClassLoader(new Array[URL](0), Thread.currentThread.getContextClassLoader) Thread.currentThread.setContextClassLoader(loader) @@ -193,10 +212,10 @@ object SparkSubmit { mainMethod.invoke(null, childArgs.toArray) } - def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { + private def addJarToClasspath(localJar: String, loader: ExecutorURLClassLoader) { val localJarFile = new File(localJar) if (!localJarFile.exists()) { - System.err.println("Jar does not exist: " + localJar + ". Skipping.") + printWarning(s"Jar $localJar does not exist, skipping.") } val url = localJarFile.getAbsoluteFile.toURI.toURL diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index ff2aa68908e34..9c8f54ea6f77a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -40,25 +40,45 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { var name: String = null var childArgs: ArrayBuffer[String] = new ArrayBuffer[String]() var jars: String = null + var verbose: Boolean = false loadEnvVars() - parseArgs(args.toList) - - def loadEnvVars() { - master = System.getenv("MASTER") - deployMode = System.getenv("DEPLOY_MODE") + parseOpts(args.toList) + + // Sanity checks + if (args.length == 0) printUsageAndExit(-1) + if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") + if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + + override def toString = { + s"""Parsed arguments: + | master $master + | deployMode $deployMode + | executorMemory $executorMemory + | executorCores $executorCores + | totalExecutorCores $totalExecutorCores + | driverMemory $driverMemory + | drivercores $driverCores + | supervise $supervise + | queue $queue + | numExecutors $numExecutors + | files $files + | archives $archives + | mainClass $mainClass + | primaryResource $primaryResource + | name $name + | childArgs [${childArgs.mkString(" ")}] + | jars $jars + | verbose $verbose + """.stripMargin } - def parseArgs(args: List[String]) { - if (args.size == 0) { - printUsageAndExit(1) - System.exit(1) - } - primaryResource = args(0) - parseOpts(args.tail) + private def loadEnvVars() { + Option(System.getenv("MASTER")).map(master = _) + Option(System.getenv("DEPLOY_MODE")).map(deployMode = _) } - def parseOpts(opts: List[String]): Unit = opts match { + private def parseOpts(opts: List[String]): Unit = opts match { case ("--name") :: value :: tail => name = value parseOpts(tail) @@ -73,8 +93,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { case ("--deploy-mode") :: value :: tail => if (value != "client" && value != "cluster") { - System.err.println("--deploy-mode must be either \"client\" or \"cluster\"") - System.exit(1) + SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") } deployMode = value parseOpts(tail) @@ -130,17 +149,28 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { case ("--help" | "-h") :: tail => printUsageAndExit(0) - case Nil => + case ("--verbose" | "-v") :: tail => + verbose = true + parseOpts(tail) - case _ => - printUsageAndExit(1, opts) + case value :: tail => + if (primaryResource != null) { + val error = s"Found two conflicting resources, $value and $primaryResource." + + " Expecting only one resource." + SparkSubmit.printErrorAndExit(error) + } + primaryResource = value + parseOpts(tail) + + case Nil => } - def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { + val outStream = SparkSubmit.printStream if (unknownParam != null) { - System.err.println("Unknown/unsupported param " + unknownParam) + outStream.println("Unknown/unsupported param " + unknownParam) } - System.err.println( + outStream.println( """Usage: spark-submit [options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. @@ -171,6 +201,6 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | --archives ARCHIVES Comma separated list of archives to be extracted into the | working dir of each executor.""".stripMargin ) - System.exit(exitCode) + SparkSubmit.exitFn() } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 29fef2ed8c165..4e489cd9b66a6 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -17,14 +17,71 @@ package org.apache.spark.deploy +import java.io.{OutputStream, PrintStream} + +import scala.collection.mutable.ArrayBuffer + import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers + import org.apache.spark.deploy.SparkSubmit._ + class SparkSubmitSuite extends FunSuite with ShouldMatchers { + + val noOpOutputStream = new OutputStream { + def write(b: Int) = {} + } + + /** Simple PrintStream that reads data into a buffer */ + class BufferPrintStream extends PrintStream(noOpOutputStream) { + var lineBuffer = ArrayBuffer[String]() + override def println(line: String) { + lineBuffer += line + } + } + + /** Returns true if the script exits and the given search string is printed. */ + def testPrematureExit(input: Array[String], searchString: String): Boolean = { + val printStream = new BufferPrintStream() + SparkSubmit.printStream = printStream + + @volatile var exitedCleanly = false + SparkSubmit.exitFn = () => exitedCleanly = true + + val thread = new Thread { + override def run() = try { + SparkSubmit.main(input) + } catch { + // If exceptions occur after the "exit" has happened, fine to ignore them. + // These represent code paths not reachable during normal execution. + case e: Exception => if (!exitedCleanly) throw e + } + } + thread.start() + thread.join() + printStream.lineBuffer.find(s => s.contains(searchString)).size > 0 + } + test("prints usage on empty input") { - val clArgs = Array[String]() - // val appArgs = new SparkSubmitArguments(clArgs) + testPrematureExit(Array[String](), "Usage: spark-submit") should be (true) + } + + test("prints usage with only --help") { + testPrematureExit(Array("--help"), "Usage: spark-submit") should be (true) + } + + test("handles multiple binary definitions") { + val adjacentJars = Array("foo.jar", "bar.jar") + testPrematureExit(adjacentJars, "error: Found two conflicting resources") should be (true) + + val nonAdjacentJars = + Array("foo.jar", "--master", "123", "--class", "abc", "bar.jar") + testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") should be (true) + } + + test("handle binary specified but not class") { + testPrematureExit(Array("foo.jar"), "must specify a main class") } test("handles YARN cluster mode") { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 71a64ecf5879a..0179b0600c61f 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -167,6 +167,9 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa object Client { def main(argStrings: Array[String]) { + println("WARNING: This client is deprecated and will be removed in a future version of Spark.") + println("Use ./bin/spark-submit with \"--master yarn\"") + // Set an env variable indicating we are running in YARN mode. // Note that anything with SPARK prefix gets propagated to all (remote) processes System.setProperty("SPARK_YARN_MODE", "true") diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 837b7e12cb0de..77eb1276a0c4e 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -173,6 +173,9 @@ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: Spa object Client { def main(argStrings: Array[String]) { + println("WARNING: This client is deprecated and will be removed in a future version of Spark.") + println("Use ./bin/spark-submit with \"--master yarn\"") + // Set an env variable indicating we are running in YARN mode. // Note: anything env variable with SPARK_ prefix gets propagated to all (remote) processes - // see Client#setupLaunchEnv(). From 5731af5be65ccac831445f351baf040a0d007687 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 31 Mar 2014 15:23:46 -0700 Subject: [PATCH 165/397] [SQL] Rewrite join implementation to allow streaming of one relation. Before we were materializing everything in memory. This also uses the projection interface so will be easier to plug in code gen (its ported from that branch). @rxin @liancheng Author: Michael Armbrust Closes #250 from marmbrus/hashJoin and squashes the following commits: 1ad873e [Michael Armbrust] Change hasNext logic back to the correct version. 8e6f2a2 [Michael Armbrust] Review comments. 1e9fb63 [Michael Armbrust] style bc0cb84 [Michael Armbrust] Rewrite join implementation to allow streaming of one relation. --- .../spark/sql/catalyst/expressions/Row.scala | 10 ++ .../sql/catalyst/expressions/predicates.scala | 6 + .../org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 6 +- .../apache/spark/sql/execution/joins.scala | 127 +++++++++++++----- .../apache/spark/sql/hive/HiveContext.scala | 2 +- 6 files changed, 116 insertions(+), 37 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 31d42b9ee71a0..6f939e6c41f6b 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 @@ -44,6 +44,16 @@ trait Row extends Seq[Any] with Serializable { s"[${this.mkString(",")}]" def copy(): Row + + /** Returns true if there are any NULL values in this row. */ + def anyNull: Boolean = { + var i = 0 + while (i < length) { + if (isNullAt(i)) { return true } + i += 1 + } + false + } } /** 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 722ff517d250e..02fedd16b8d4b 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 @@ -21,6 +21,12 @@ import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedException import org.apache.spark.sql.catalyst.types.{BooleanType, StringType} +object InterpretedPredicate { + def apply(expression: Expression): (Row => Boolean) = { + (r: Row) => expression.apply(r).asInstanceOf[Boolean] + } +} + trait Predicate extends Expression { self: Product => diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index cf3c06acce5b0..f950ea08ec57a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -117,7 +117,7 @@ class SQLContext(@transient val sparkContext: SparkContext) val strategies: Seq[Strategy] = TopK :: PartialAggregation :: - SparkEquiInnerJoin :: + HashJoin :: ParquetOperations :: BasicOperators :: CartesianProduct :: 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 86f9d3e0fa954..e35ac0b6ca95a 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 @@ -28,7 +28,7 @@ import org.apache.spark.sql.parquet._ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => - object SparkEquiInnerJoin extends Strategy { + object HashJoin extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case FilteredOperation(predicates, logical.Join(left, right, Inner, condition)) => logger.debug(s"Considering join: ${predicates ++ condition}") @@ -51,8 +51,8 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { val leftKeys = joinKeys.map(_._1) val rightKeys = joinKeys.map(_._2) - val joinOp = execution.SparkEquiInnerJoin( - leftKeys, rightKeys, planLater(left), planLater(right)) + val joinOp = execution.HashJoin( + leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) // Make sure other conditions are met if present. if (otherPredicates.nonEmpty) { 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 f0d21143ba5d1..c89dae9358bf7 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 @@ -17,21 +17,22 @@ package org.apache.spark.sql.execution -import scala.collection.mutable +import scala.collection.mutable.{ArrayBuffer, BitSet} -import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext -import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} -import org.apache.spark.rdd.PartitionLocalRDDFunctions._ +sealed abstract class BuildSide +case object BuildLeft extends BuildSide +case object BuildRight extends BuildSide -case class SparkEquiInnerJoin( +case class HashJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], + buildSide: BuildSide, left: SparkPlan, right: SparkPlan) extends BinaryNode { @@ -40,33 +41,93 @@ case class SparkEquiInnerJoin( override def requiredChildDistribution = ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: Nil + val (buildPlan, streamedPlan) = buildSide match { + case BuildLeft => (left, right) + case BuildRight => (right, left) + } + + val (buildKeys, streamedKeys) = buildSide match { + case BuildLeft => (leftKeys, rightKeys) + case BuildRight => (rightKeys, leftKeys) + } + def output = left.output ++ right.output - def execute() = attachTree(this, "execute") { - val leftWithKeys = left.execute().mapPartitions { iter => - val generateLeftKeys = new Projection(leftKeys, left.output) - iter.map(row => (generateLeftKeys(row), row.copy())) - } + @transient lazy val buildSideKeyGenerator = new Projection(buildKeys, buildPlan.output) + @transient lazy val streamSideKeyGenerator = + () => new MutableProjection(streamedKeys, streamedPlan.output) - val rightWithKeys = right.execute().mapPartitions { iter => - val generateRightKeys = new Projection(rightKeys, right.output) - iter.map(row => (generateRightKeys(row), row.copy())) - } + def execute() = { - // Do the join. - val joined = filterNulls(leftWithKeys).joinLocally(filterNulls(rightWithKeys)) - // Drop join keys and merge input tuples. - joined.map { case (_, (leftTuple, rightTuple)) => buildRow(leftTuple ++ rightTuple) } - } + buildPlan.execute().zipPartitions(streamedPlan.execute()) { (buildIter, streamIter) => + // TODO: Use Spark's HashMap implementation. + val hashTable = new java.util.HashMap[Row, ArrayBuffer[Row]]() + var currentRow: Row = null + + // Create a mapping of buildKeys -> rows + while (buildIter.hasNext) { + currentRow = buildIter.next() + val rowKey = buildSideKeyGenerator(currentRow) + if(!rowKey.anyNull) { + val existingMatchList = hashTable.get(rowKey) + val matchList = if (existingMatchList == null) { + val newMatchList = new ArrayBuffer[Row]() + hashTable.put(rowKey, newMatchList) + newMatchList + } else { + existingMatchList + } + matchList += currentRow.copy() + } + } + + new Iterator[Row] { + private[this] var currentStreamedRow: Row = _ + private[this] var currentHashMatches: ArrayBuffer[Row] = _ + private[this] var currentMatchPosition: Int = -1 - /** - * 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[(Row, Row)]) = - rdd.filter { - case (key: Seq[_], _) => !key.exists(_ == null) + // Mutable per row objects. + private[this] val joinRow = new JoinedRow + + private[this] val joinKeys = streamSideKeyGenerator() + + override final def hasNext: Boolean = + (currentMatchPosition != -1 && currentMatchPosition < currentHashMatches.size) || + (streamIter.hasNext && fetchNext()) + + override final def next() = { + val ret = joinRow(currentStreamedRow, currentHashMatches(currentMatchPosition)) + currentMatchPosition += 1 + ret + } + + /** + * Searches the streamed iterator for the next row that has at least one match in hashtable. + * + * @return true if the search is successful, and false the streamed iterator runs out of + * tuples. + */ + private final def fetchNext(): Boolean = { + currentHashMatches = null + currentMatchPosition = -1 + + while (currentHashMatches == null && streamIter.hasNext) { + currentStreamedRow = streamIter.next() + if (!joinKeys(currentStreamedRow).anyNull) { + currentHashMatches = hashTable.get(joinKeys.currentValue) + } + } + + if (currentHashMatches == null) { + false + } else { + currentMatchPosition = 0 + true + } + } + } } + } } case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { @@ -95,17 +156,19 @@ case class BroadcastNestedLoopJoin( def right = broadcast @transient lazy val boundCondition = - condition - .map(c => BindReferences.bindReference(c, left.output ++ right.output)) - .getOrElse(Literal(true)) + InterpretedPredicate( + condition + .map(c => BindReferences.bindReference(c, left.output ++ right.output)) + .getOrElse(Literal(true))) def execute() = { 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 matchedRows = new ArrayBuffer[Row] + // TODO: Use Spark's BitSet. + val includedBroadcastTuples = new BitSet(broadcastedRelation.value.size) val joinedRow = new JoinedRow streamedIter.foreach { streamedRow => @@ -115,7 +178,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(joinedRow(streamedRow, broadcastedRow)).asInstanceOf[Boolean]) { + if (boundCondition(joinedRow(streamedRow, broadcastedRow))) { matchedRows += buildRow(streamedRow ++ broadcastedRow) matched = true includedBroadcastTuples += i 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 fc5057b73fe24..197b557cba5f4 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 @@ -194,7 +194,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { DataSinks, Scripts, PartialAggregation, - SparkEquiInnerJoin, + HashJoin, BasicOperators, CartesianProduct, BroadcastNestedLoopJoin From 33b3c2a8c6c71b89744834017a183ea855e1697c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 31 Mar 2014 16:25:43 -0700 Subject: [PATCH 166/397] SPARK-1365 [HOTFIX] Fix RateLimitedOutputStream test This test needs to be fixed. It currently depends on Thread.sleep() having exact-timing semantics, which is not a valid assumption. Author: Patrick Wendell Closes #277 from pwendell/rate-limited-stream and squashes the following commits: 6c0ff81 [Patrick Wendell] SPARK-1365: Fix RateLimitedOutputStream test --- .../spark/streaming/util/RateLimitedOutputStreamSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala index 7d18a0fcf7ba8..9ebf7b484f421 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/util/RateLimitedOutputStreamSuite.scala @@ -36,8 +36,9 @@ class RateLimitedOutputStreamSuite extends FunSuite { val stream = new RateLimitedOutputStream(underlying, desiredBytesPerSec = 10000) val elapsedNs = benchmark { stream.write(data.getBytes("UTF-8")) } - // We accept anywhere from 4.0 to 4.99999 seconds since the value is rounded down. - assert(SECONDS.convert(elapsedNs, NANOSECONDS) === 4) + val seconds = SECONDS.convert(elapsedNs, NANOSECONDS) + assert(seconds >= 4, s"Seconds value ($seconds) is less than 4.") + assert(seconds <= 30, s"Took more than 30 seconds ($seconds) to write data.") assert(underlying.toString("UTF-8") === data) } } From 564f1c137caf07bd1f073ec6c93551dcad935ee5 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Tue, 1 Apr 2014 08:26:31 +0530 Subject: [PATCH 167/397] SPARK-1376. In the yarn-cluster submitter, rename "args" option to "arg" Author: Sandy Ryza Closes #279 from sryza/sandy-spark-1376 and squashes the following commits: d8aebfa [Sandy Ryza] SPARK-1376. In the yarn-cluster submitter, rename "args" option to "arg" --- docs/running-on-yarn.md | 7 ++++--- .../org/apache/spark/deploy/yarn/ClientArguments.scala | 9 ++++++--- 2 files changed, 10 insertions(+), 6 deletions(-) diff --git a/docs/running-on-yarn.md b/docs/running-on-yarn.md index d8657c4bc7096..982514391ac00 100644 --- a/docs/running-on-yarn.md +++ b/docs/running-on-yarn.md @@ -61,7 +61,7 @@ The command to launch the Spark application on the cluster is as follows: SPARK_JAR= ./bin/spark-class org.apache.spark.deploy.yarn.Client \ --jar \ --class \ - --args \ + --arg \ --num-executors \ --driver-memory \ --executor-memory \ @@ -72,7 +72,7 @@ The command to launch the Spark application on the cluster is as follows: --files \ --archives -For example: +To pass multiple arguments the "arg" option can be specified multiple times. For example: # Build the Spark assembly JAR and the Spark examples JAR $ SPARK_HADOOP_VERSION=2.0.5-alpha SPARK_YARN=true sbt/sbt assembly @@ -85,7 +85,8 @@ For example: ./bin/spark-class org.apache.spark.deploy.yarn.Client \ --jar examples/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-examples-assembly-{{site.SPARK_VERSION}}.jar \ --class org.apache.spark.examples.SparkPi \ - --args yarn-cluster \ + --arg yarn-cluster \ + --arg 5 \ --num-executors 3 \ --driver-memory 4g \ --executor-memory 2g \ diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala index c565f2dde24fc..3e4c739e34fe9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientArguments.scala @@ -63,7 +63,10 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { userClass = value args = tail - case ("--args") :: value :: tail => + case ("--args" | "--arg") :: value :: tail => + if (args(0) == "--args") { + println("--args is deprecated. Use --arg instead.") + } userArgsBuffer += value args = tail @@ -146,8 +149,8 @@ class ClientArguments(val args: Array[String], val sparkConf: SparkConf) { "Options:\n" + " --jar JAR_PATH Path to your application's JAR file (required in yarn-cluster mode)\n" + " --class CLASS_NAME Name of your application's main class (required)\n" + - " --args ARGS Arguments to be passed to your application's main class.\n" + - " Mutliple invocations are possible, each will be passed in order.\n" + + " --arg ARGS Argument to be passed to your application's main class.\n" + + " Multiple invocations are possible, each will be passed in order.\n" + " --num-executors NUM Number of executors to start (Default: 2)\n" + " --executor-cores NUM Number of cores for the executors (Default: 1).\n" + " --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512 Mb)\n" + From 94fe7fd4fa9749cb13e540e4f9caf28de47eaf32 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 31 Mar 2014 21:42:36 -0700 Subject: [PATCH 168/397] [SPARK-1377] Upgrade Jetty to 8.1.14v20131031 Previous version was 7.6.8v20121106. The only difference between Jetty 7 and Jetty 8 is that the former uses Servlet API 2.5, while the latter uses Servlet API 3.0. Author: Andrew Or Closes #280 from andrewor14/jetty-upgrade and squashes the following commits: dd57104 [Andrew Or] Merge github.com:apache/spark into jetty-upgrade e75fa85 [Andrew Or] Upgrade Jetty to 8.1.14v20131031 --- .../main/scala/org/apache/spark/ui/JettyUtils.scala | 3 ++- pom.xml | 8 ++++---- project/SparkBuild.scala | 12 ++++++------ 3 files changed, 12 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 6e1736f6fbc23..e1a1f209c9282 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -18,13 +18,14 @@ package org.apache.spark.ui import java.net.{InetSocketAddress, URL} +import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.annotation.tailrec import scala.util.{Failure, Success, Try} import scala.xml.Node -import org.eclipse.jetty.server.{DispatcherType, Server} +import org.eclipse.jetty.server.Server import org.eclipse.jetty.server.handler._ import org.eclipse.jetty.servlet._ import org.eclipse.jetty.util.thread.QueuedThreadPool diff --git a/pom.xml b/pom.xml index 72acf2b402703..09a449d81453f 100644 --- a/pom.xml +++ b/pom.xml @@ -192,22 +192,22 @@ org.eclipse.jetty jetty-util - 7.6.8.v20121106 + 8.1.14.v20131031 org.eclipse.jetty jetty-security - 7.6.8.v20121106 + 8.1.14.v20131031 org.eclipse.jetty jetty-plus - 7.6.8.v20121106 + 8.1.14.v20131031 org.eclipse.jetty jetty-server - 7.6.8.v20121106 + 8.1.14.v20131031 com.google.guava diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 2549bc9710f1f..7457ff456ade4 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -248,13 +248,13 @@ object SparkBuild extends Build { */ libraryDependencies ++= Seq( - "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % "7.6.8.v20121106", - "org.eclipse.jetty" % "jetty-util" % "7.6.8.v20121106", - "org.eclipse.jetty" % "jetty-plus" % "7.6.8.v20121106", - "org.eclipse.jetty" % "jetty-security" % "7.6.8.v20121106", + "io.netty" % "netty-all" % "4.0.17.Final", + "org.eclipse.jetty" % "jetty-server" % "8.1.14.v20131031", + "org.eclipse.jetty" % "jetty-util" % "8.1.14.v20131031", + "org.eclipse.jetty" % "jetty-plus" % "8.1.14.v20131031", + "org.eclipse.jetty" % "jetty-security" % "8.1.14.v20131031", /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ - "org.eclipse.jetty.orbit" % "javax.servlet" % "2.5.0.v201103041518" artifacts Artifact("javax.servlet", "jar", "jar"), + "org.eclipse.jetty.orbit" % "javax.servlet" % "3.0.0.v201112011016" artifacts Artifact("javax.servlet", "jar", "jar"), "org.scalatest" %% "scalatest" % "1.9.1" % "test", "org.scalacheck" %% "scalacheck" % "1.10.0" % "test", "com.novocode" % "junit-interface" % "0.10" % "test", From ada310a9d3d5419e101b24d9b41398f609da1ad3 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 31 Mar 2014 23:01:14 -0700 Subject: [PATCH 169/397] [Hot Fix #42] Persisted RDD disappears on storage page if re-used If a previously persisted RDD is re-used, its information disappears from the Storage page. This is because the tasks associated with re-using the RDD do not report the RDD's blocks as updated (which is correct). On stage submit, however, we overwrite any existing information regarding that RDD with a fresh one, whether or not the information for the RDD already exists. Author: Andrew Or Closes #281 from andrewor14/ui-storage-fix and squashes the following commits: 408585a [Andrew Or] Fix storage UI bug --- .../main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index 4d8b01dbe6e1b..a7b24ff695214 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -84,7 +84,7 @@ private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListe override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { val rddInfo = stageSubmitted.stageInfo.rddInfo - _rddInfoMap(rddInfo.id) = rddInfo + _rddInfoMap.getOrElseUpdate(rddInfo.id, rddInfo) } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { From f5c418da044ef7f3d7185cc5bb1bef79d7f4e25c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 1 Apr 2014 14:45:44 -0700 Subject: [PATCH 170/397] [SQL] SPARK-1372 Support for caching and uncaching tables in a SQLContext. This doesn't yet support different databases in Hive (though you can probably workaround this by calling `USE `). However, given the time constraints for 1.0 I think its probably worth including this now and extending the functionality in the next release. Author: Michael Armbrust Closes #282 from marmbrus/cacheTables and squashes the following commits: 83785db [Michael Armbrust] Support for caching and uncaching tables in a SQLContext. --- .../spark/sql/catalyst/analysis/Catalog.scala | 11 +++- .../org/apache/spark/sql/SQLContext.scala | 32 +++++++++- .../apache/spark/sql/CachedTableSuite.scala | 61 +++++++++++++++++++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 7 +++ ...d table-0-ce3797dc14a603cba2a5e58c8612de5b | 1 + ...d table-0-ce3797dc14a603cba2a5e58c8612de5b | 1 + .../spark/sql/hive/CachedTableSuite.scala | 58 ++++++++++++++++++ 7 files changed, 169 insertions(+), 2 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala create mode 100644 sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b create mode 100644 sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala 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 e09182dd8d5df..6b58b9322c4bf 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 @@ -31,6 +31,7 @@ trait Catalog { alias: Option[String] = None): LogicalPlan def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit + def unregisterTable(databaseName: Option[String], tableName: String): Unit } class SimpleCatalog extends Catalog { @@ -40,7 +41,7 @@ class SimpleCatalog extends Catalog { tables += ((tableName, plan)) } - def dropTable(tableName: String) = tables -= tableName + def unregisterTable(databaseName: Option[String], tableName: String) = { tables -= tableName } def lookupRelation( databaseName: Option[String], @@ -87,6 +88,10 @@ trait OverrideCatalog extends Catalog { plan: LogicalPlan): Unit = { overrides.put((databaseName, tableName), plan) } + + override def unregisterTable(databaseName: Option[String], tableName: String): Unit = { + overrides.remove((databaseName, tableName)) + } } /** @@ -104,4 +109,8 @@ object EmptyCatalog extends Catalog { def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = { throw new UnsupportedOperationException } + + def unregisterTable(databaseName: Option[String], tableName: String): Unit = { + throw new UnsupportedOperationException + } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index f950ea08ec57a..69bbbdc8943fa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -26,8 +26,9 @@ import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.optimizer.Optimizer -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{Subquery, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan import org.apache.spark.sql.execution._ /** @@ -111,6 +112,35 @@ class SQLContext(@transient val sparkContext: SparkContext) result } + /** Returns the specified table as a SchemaRDD */ + def table(tableName: String): SchemaRDD = + new SchemaRDD(this, catalog.lookupRelation(None, tableName)) + + /** Caches the specified table in-memory. */ + def cacheTable(tableName: String): Unit = { + val currentTable = catalog.lookupRelation(None, tableName) + val asInMemoryRelation = + InMemoryColumnarTableScan(currentTable.output, executePlan(currentTable).executedPlan) + + catalog.registerTable(None, tableName, SparkLogicalPlan(asInMemoryRelation)) + } + + /** Removes the specified table from the in-memory cache. */ + def uncacheTable(tableName: String): Unit = { + EliminateAnalysisOperators(catalog.lookupRelation(None, tableName)) match { + // This is kind of a hack to make sure that if this was just an RDD registered as a table, + // we reregister the RDD as a table. + case SparkLogicalPlan(inMem @ InMemoryColumnarTableScan(_, e: ExistingRdd)) => + inMem.cachedColumnBuffers.unpersist() + catalog.unregisterTable(None, tableName) + catalog.registerTable(None, tableName, SparkLogicalPlan(e)) + case SparkLogicalPlan(inMem: InMemoryColumnarTableScan) => + inMem.cachedColumnBuffers.unpersist() + catalog.unregisterTable(None, tableName) + case plan => throw new IllegalArgumentException(s"Table $tableName is not cached: $plan") + } + } + protected[sql] class SparkPlanner extends SparkStrategies { val sparkContext = self.sparkContext diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala new file mode 100644 index 0000000000000..e5902c3cae381 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -0,0 +1,61 @@ +/* + * 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.FunSuite +import org.apache.spark.sql.TestData._ +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan + +class CachedTableSuite extends QueryTest { + TestData // Load test tables. + + test("read from cached table and uncache") { + TestSQLContext.cacheTable("testData") + + checkAnswer( + TestSQLContext.table("testData"), + testData.collect().toSeq + ) + + TestSQLContext.table("testData").queryExecution.analyzed match { + case SparkLogicalPlan(_ : InMemoryColumnarTableScan) => // Found evidence of caching + case noCache => fail(s"No cache node found in plan $noCache") + } + + TestSQLContext.uncacheTable("testData") + + checkAnswer( + TestSQLContext.table("testData"), + testData.collect().toSeq + ) + + TestSQLContext.table("testData").queryExecution.analyzed match { + case cachePlan @ SparkLogicalPlan(_ : InMemoryColumnarTableScan) => + fail(s"Table still cached after uncache: $cachePlan") + case noCache => // Table uncached successfully + } + } + + test("correct error on uncache of non-cached table") { + intercept[IllegalArgumentException] { + TestSQLContext.uncacheTable("testData") + } + } +} 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 4f8353666a12b..29834a11f41dc 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 @@ -141,6 +141,13 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { */ override def registerTable( databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit = ??? + + /** + * UNIMPLEMENTED: It needs to be decided how we will persist in-memory tables to the metastore. + * For now, if this functionality is desired mix in the in-memory [[OverrideCatalog]]. + */ + override def unregisterTable( + databaseName: Option[String], tableName: String): Unit = ??? } object HiveMetastoreTypes extends RegexParsers { diff --git a/sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b new file mode 100644 index 0000000000000..60878ffb77064 --- /dev/null +++ b/sql/hive/src/test/resources/golden/read from cached table-0-ce3797dc14a603cba2a5e58c8612de5b @@ -0,0 +1 @@ +238 val_238 diff --git a/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b b/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b new file mode 100644 index 0000000000000..60878ffb77064 --- /dev/null +++ b/sql/hive/src/test/resources/golden/read from uncached table-0-ce3797dc14a603cba2a5e58c8612de5b @@ -0,0 +1 @@ +238 val_238 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala new file mode 100644 index 0000000000000..68d45e53cdf26 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -0,0 +1,58 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive + +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan +import org.apache.spark.sql.hive.execution.HiveComparisonTest + +class CachedTableSuite extends HiveComparisonTest { + TestHive.loadTestTable("src") + + test("cache table") { + TestHive.cacheTable("src") + } + + createQueryTest("read from cached table", + "SELECT * FROM src LIMIT 1") + + test("check that table is cached and uncache") { + TestHive.table("src").queryExecution.analyzed match { + case SparkLogicalPlan(_ : InMemoryColumnarTableScan) => // Found evidence of caching + case noCache => fail(s"No cache node found in plan $noCache") + } + TestHive.uncacheTable("src") + } + + createQueryTest("read from uncached table", + "SELECT * FROM src LIMIT 1") + + test("make sure table is uncached") { + TestHive.table("src").queryExecution.analyzed match { + case cachePlan @ SparkLogicalPlan(_ : InMemoryColumnarTableScan) => + fail(s"Table still cached after uncache: $cachePlan") + case noCache => // Table uncached successfully + } + } + + test("correct error on uncache of non-cached table") { + intercept[IllegalArgumentException] { + TestHive.uncacheTable("src") + } + } +} From 764353d2c5162352781c273dd3d4af6a309190c7 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Tue, 1 Apr 2014 18:35:50 -0700 Subject: [PATCH 171/397] [SPARK-1342] Scala 2.10.4 Just a Scala version increment Author: Mark Hamstra Closes #259 from markhamstra/scala-2.10.4 and squashes the following commits: fbec547 [Mark Hamstra] [SPARK-1342] Bumped Scala version to 2.10.4 --- core/pom.xml | 2 +- dev/audit-release/README.md | 2 +- dev/audit-release/audit_release.py | 2 +- docker/spark-test/base/Dockerfile | 2 +- docs/_config.yml | 2 +- pom.xml | 4 ++-- project/SparkBuild.scala | 2 +- project/plugins.sbt | 2 +- project/project/SparkPluginBuild.scala | 2 +- sql/README.md | 2 +- 10 files changed, 11 insertions(+), 11 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index eb6cc4d3105e9..e4c32eff0cd77 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -150,7 +150,7 @@ json4s-jackson_${scala.binary.version} 3.2.6 diff --git a/dev/audit-release/README.md b/dev/audit-release/README.md index 2437a98672177..38becda0eae92 100644 --- a/dev/audit-release/README.md +++ b/dev/audit-release/README.md @@ -4,7 +4,7 @@ run them locally by setting appropriate environment variables. ``` $ cd sbt_app_core -$ SCALA_VERSION=2.10.3 \ +$ SCALA_VERSION=2.10.4 \ SPARK_VERSION=1.0.0-SNAPSHOT \ SPARK_RELEASE_REPOSITORY=file:///home/patrick/.ivy2/local \ sbt run diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index 52c367d9b030d..fa2f02dfecc75 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -35,7 +35,7 @@ RELEASE_KEY = "9E4FE3AF" RELEASE_REPOSITORY = "https://repository.apache.org/content/repositories/orgapachespark-1006/" RELEASE_VERSION = "1.0.0" -SCALA_VERSION = "2.10.3" +SCALA_VERSION = "2.10.4" SCALA_BINARY_VERSION = "2.10" ## diff --git a/docker/spark-test/base/Dockerfile b/docker/spark-test/base/Dockerfile index e543db6143e4d..5956d59130fbf 100644 --- a/docker/spark-test/base/Dockerfile +++ b/docker/spark-test/base/Dockerfile @@ -25,7 +25,7 @@ RUN apt-get update # install a few other useful packages plus Open Jdk 7 RUN apt-get install -y less openjdk-7-jre-headless net-tools vim-tiny sudo openssh-server -ENV SCALA_VERSION 2.10.3 +ENV SCALA_VERSION 2.10.4 ENV CDH_VERSION cdh4 ENV SCALA_HOME /opt/scala-$SCALA_VERSION ENV SPARK_HOME /opt/spark diff --git a/docs/_config.yml b/docs/_config.yml index aa5a5adbc1743..d585b8c5ea763 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -6,7 +6,7 @@ markdown: kramdown SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" -SCALA_VERSION: "2.10.3" +SCALA_VERSION: "2.10.4" MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/pom.xml b/pom.xml index 09a449d81453f..7d58060cba606 100644 --- a/pom.xml +++ b/pom.xml @@ -110,7 +110,7 @@ 1.6 - 2.10.3 + 2.10.4 2.10 0.13.0 org.spark-project.akka @@ -380,7 +380,7 @@ lift-json_${scala.binary.version} 2.5.1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 7457ff456ade4..c5c697e8e2427 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -152,7 +152,7 @@ object SparkBuild extends Build { def sharedSettings = Defaults.defaultSettings ++ MimaBuild.mimaSettings(file(sparkHome)) ++ Seq( organization := "org.apache.spark", version := SPARK_VERSION, - scalaVersion := "2.10.3", + scalaVersion := "2.10.4", scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), diff --git a/project/plugins.sbt b/project/plugins.sbt index 5aa8a1ec2409b..d787237ddc540 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -1,4 +1,4 @@ -scalaVersion := "2.10.3" +scalaVersion := "2.10.4" resolvers += Resolver.url("artifactory", url("http://scalasbt.artifactoryonline.com/scalasbt/sbt-plugin-releases"))(Resolver.ivyStylePatterns) diff --git a/project/project/SparkPluginBuild.scala b/project/project/SparkPluginBuild.scala index 5a307044ba123..0142256e90fb7 100644 --- a/project/project/SparkPluginBuild.scala +++ b/project/project/SparkPluginBuild.scala @@ -32,7 +32,7 @@ object SparkPluginDef extends Build { name := "spark-style", organization := "org.apache.spark", version := sparkVersion, - scalaVersion := "2.10.3", + scalaVersion := "2.10.4", scalacOptions := Seq("-unchecked", "-deprecation"), libraryDependencies ++= Dependencies.scalaStyle ) diff --git a/sql/README.md b/sql/README.md index 4192fecb92fb0..14d5555f0c713 100644 --- a/sql/README.md +++ b/sql/README.md @@ -38,7 +38,7 @@ 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._ -Welcome to Scala version 2.10.3 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45). +Welcome to Scala version 2.10.4 (Java HotSpot(TM) 64-Bit Server VM, Java 1.7.0_45). Type in expressions to have them evaluated. Type :help for more information. From afb5ea62786e3ca055e247176def3e7ecf0d2c9d Mon Sep 17 00:00:00 2001 From: Diana Carroll Date: Tue, 1 Apr 2014 19:29:26 -0700 Subject: [PATCH 172/397] [Spark-1134] only call ipython if no arguments are given; remove IPYTHONOPTS from call see comments on Pull Request https://github.com/apache/spark/pull/38 (i couldn't figure out how to modify an existing pull request, so I'm hoping I can withdraw that one and replace it with this one.) Author: Diana Carroll Closes #227 from dianacarroll/spark-1134 and squashes the following commits: ffe47f2 [Diana Carroll] [spark-1134] remove ipythonopts from ipython command b673bf7 [Diana Carroll] Merge branch 'master' of github.com:apache/spark 0309cf9 [Diana Carroll] SPARK-1134 bug with ipython prevents non-interactive use with spark; only call ipython if no command line arguments were supplied --- bin/pyspark | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index 67e1f61eeb1e5..7932a247b54d0 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -55,8 +55,9 @@ if [ -n "$IPYTHON_OPTS" ]; then IPYTHON=1 fi -if [[ "$IPYTHON" = "1" ]] ; then - exec ipython $IPYTHON_OPTS +# Only use ipython if no command line arguments were provided [SPARK-1134] +if [[ "$IPYTHON" = "1" && $# = 0 ]] ; then + exec ipython else exec "$PYSPARK_PYTHON" "$@" fi From 45df9127365f8942794273b8ada004bf6ea3ef10 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Tue, 1 Apr 2014 19:31:50 -0700 Subject: [PATCH 173/397] Revert "[Spark-1134] only call ipython if no arguments are given; remove IPYTHONOPTS from call" This reverts commit afb5ea62786e3ca055e247176def3e7ecf0d2c9d. --- bin/pyspark | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/bin/pyspark b/bin/pyspark index 7932a247b54d0..67e1f61eeb1e5 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -55,9 +55,8 @@ if [ -n "$IPYTHON_OPTS" ]; then IPYTHON=1 fi -# Only use ipython if no command line arguments were provided [SPARK-1134] -if [[ "$IPYTHON" = "1" && $# = 0 ]] ; then - exec ipython +if [[ "$IPYTHON" = "1" ]] ; then + exec ipython $IPYTHON_OPTS else exec "$PYSPARK_PYTHON" "$@" fi From 8b3045ceab591a3f3ca18823c7e2c5faca38a06e Mon Sep 17 00:00:00 2001 From: Manish Amde Date: Tue, 1 Apr 2014 21:40:49 -0700 Subject: [PATCH 174/397] MLI-1 Decision Trees Joint work with @hirakendu, @etrain, @atalwalkar and @harsha2010. Key features: + Supports binary classification and regression + Supports gini, entropy and variance for information gain calculation + Supports both continuous and categorical features The algorithm has gone through several development iterations over the last few months leading to a highly optimized implementation. Optimizations include: 1. Level-wise training to reduce passes over the entire dataset. 2. Bin-wise split calculation to reduce computation overhead. 3. Aggregation over partitions before combining to reduce communication overhead. Author: Manish Amde Author: manishamde Author: Xiangrui Meng Closes #79 from manishamde/tree and squashes the following commits: 1e8c704 [Manish Amde] remove numBins field in the Strategy class 7d54b4f [manishamde] Merge pull request #4 from mengxr/dtree f536ae9 [Xiangrui Meng] another pass on code style e1dd86f [Manish Amde] implementing code style suggestions 62dc723 [Manish Amde] updating javadoc and converting helper methods to package private to allow unit testing 201702f [Manish Amde] making some more methods private f963ef5 [Manish Amde] making methods private c487e6a [manishamde] Merge pull request #1 from mengxr/dtree 24500c5 [Xiangrui Meng] minor style updates 4576b64 [Manish Amde] documentation and for to while loop conversion ff363a7 [Manish Amde] binary search for bins and while loop for categorical feature bins 632818f [Manish Amde] removing threshold for classification predict method 2116360 [Manish Amde] removing dummy bin calculation for categorical variables 6068356 [Manish Amde] ensuring num bins is always greater than max number of categories 62c2562 [Manish Amde] fixing comment indentation ad1fc21 [Manish Amde] incorporated mengxr's code style suggestions d1ef4f6 [Manish Amde] more documentation 794ff4d [Manish Amde] minor improvements to docs and style eb8fcbe [Manish Amde] minor code style updates cd2c2b4 [Manish Amde] fixing code style based on feedback 63e786b [Manish Amde] added multiple train methods for java compatability d3023b3 [Manish Amde] adding more docs for nested methods 84f85d6 [Manish Amde] code documentation 9372779 [Manish Amde] code style: max line lenght <= 100 dd0c0d7 [Manish Amde] minor: some docs 0dd7659 [manishamde] basic doc 5841c28 [Manish Amde] unit tests for categorical features f067d68 [Manish Amde] minor cleanup c0e522b [Manish Amde] updated predict and split threshold logic b09dc98 [Manish Amde] minor refactoring 6b7de78 [Manish Amde] minor refactoring and tests d504eb1 [Manish Amde] more tests for categorical features dbb7ac1 [Manish Amde] categorical feature support 6df35b9 [Manish Amde] regression predict logic 53108ed [Manish Amde] fixing index for highest bin e23c2e5 [Manish Amde] added regression support c8f6d60 [Manish Amde] adding enum for feature type b0e3e76 [Manish Amde] adding enum for feature type 154aa77 [Manish Amde] enums for configurations 733d6dd [Manish Amde] fixed tests 02c595c [Manish Amde] added command line parsing 98ec8d5 [Manish Amde] tree building and prediction logic b0eb866 [Manish Amde] added logic to handle leaf nodes 80e8c66 [Manish Amde] working version of multi-level split calculation 4798aae [Manish Amde] added gain stats class dad0afc [Manish Amde] decison stump functionality working 03f534c [Manish Amde] some more tests 0012a77 [Manish Amde] basic stump working 8bca1e2 [Manish Amde] additional code for creating intermediate RDD 92cedce [Manish Amde] basic building blocks for intermediate RDD calculation. untested. cd53eae [Manish Amde] skeletal framework --- .../spark/mllib/tree/DecisionTree.scala | 1150 +++++++++++++++++ .../org/apache/spark/mllib/tree/README.md | 17 + .../spark/mllib/tree/configuration/Algo.scala | 26 + .../tree/configuration/FeatureType.scala | 26 + .../tree/configuration/QuantileStrategy.scala | 26 + .../mllib/tree/configuration/Strategy.scala | 43 + .../spark/mllib/tree/impurity/Entropy.scala | 47 + .../spark/mllib/tree/impurity/Gini.scala | 46 + .../spark/mllib/tree/impurity/Impurity.scala | 42 + .../spark/mllib/tree/impurity/Variance.scala | 37 + .../apache/spark/mllib/tree/model/Bin.scala | 33 + .../mllib/tree/model/DecisionTreeModel.scala | 49 + .../spark/mllib/tree/model/Filter.scala | 28 + .../tree/model/InformationGainStats.scala | 39 + .../apache/spark/mllib/tree/model/Node.scala | 90 ++ .../apache/spark/mllib/tree/model/Split.scala | 64 + .../spark/mllib/tree/DecisionTreeSuite.scala | 425 ++++++ 17 files changed, 2188 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/README.md create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala new file mode 100644 index 0000000000000..33205b919db8f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -0,0 +1,1150 @@ +/* + * 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.mllib.tree + +import scala.util.control.Breaks._ + +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.FeatureType._ +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ +import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance} +import org.apache.spark.mllib.tree.model._ +import org.apache.spark.rdd.RDD +import org.apache.spark.util.random.XORShiftRandom + +/** + * A class that implements a decision tree algorithm for classification and regression. It + * supports both continuous and categorical features. + * @param strategy The configuration parameters for the tree algorithm which specify the type + * of algorithm (classification, regression, etc.), feature type (continuous, + * categorical), depth of the tree, quantile calculation strategy, etc. + */ +class DecisionTree private(val strategy: Strategy) extends Serializable with Logging { + + /** + * Method to train a decision tree model over an RDD + * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data + * @return a DecisionTreeModel that can be used for prediction + */ + def train(input: RDD[LabeledPoint]): DecisionTreeModel = { + + // Cache input RDD for speedup during multiple passes. + input.cache() + logDebug("algo = " + strategy.algo) + + // Find the splits and the corresponding bins (interval between the splits) using a sample + // of the input data. + val (splits, bins) = DecisionTree.findSplitsBins(input, strategy) + logDebug("numSplits = " + bins(0).length) + + // depth of the decision tree + val maxDepth = strategy.maxDepth + // the max number of nodes possible given the depth of the tree + val maxNumNodes = scala.math.pow(2, maxDepth).toInt - 1 + // Initialize an array to hold filters applied to points for each node. + val filters = new Array[List[Filter]](maxNumNodes) + // The filter at the top node is an empty list. + filters(0) = List() + // Initialize an array to hold parent impurity calculations for each node. + val parentImpurities = new Array[Double](maxNumNodes) + // dummy value for top node (updated during first split calculation) + val nodes = new Array[Node](maxNumNodes) + + + /* + * The main idea here is to perform level-wise training of the decision tree nodes thus + * reducing the passes over the data from l to log2(l) where l is the total number of nodes. + * Each data sample is checked for validity w.r.t to each node at a given level -- i.e., + * the sample is only used for the split calculation at the node if the sampled would have + * still survived the filters of the parent nodes. + */ + + // TODO: Convert for loop to while loop + breakable { + for (level <- 0 until maxDepth) { + + logDebug("#####################################") + logDebug("level = " + level) + logDebug("#####################################") + + // Find best split for all nodes at a level. + val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, + level, filters, splits, bins) + + for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { + // Extract info for nodes at the current level. + extractNodeInfo(nodeSplitStats, level, index, nodes) + // Extract info for nodes at the next lower level. + extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, + filters) + logDebug("final best split = " + nodeSplitStats._1) + } + require(scala.math.pow(2, level) == splitsStatsForLevel.length) + // Check whether all the nodes at the current level at leaves. + val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) + logDebug("all leaf = " + allLeaf) + if (allLeaf) break // no more tree construction + } + } + + // Initialize the top or root node of the tree. + val topNode = nodes(0) + // Build the full tree using the node info calculated in the level-wise best split calculations. + topNode.build(nodes) + + new DecisionTreeModel(topNode, strategy.algo) + } + + /** + * Extract the decision tree node information for the given tree level and node index + */ + private def extractNodeInfo( + nodeSplitStats: (Split, InformationGainStats), + level: Int, + index: Int, + nodes: Array[Node]): Unit = { + val split = nodeSplitStats._1 + val stats = nodeSplitStats._2 + val nodeIndex = scala.math.pow(2, level).toInt - 1 + index + val isLeaf = (stats.gain <= 0) || (level == strategy.maxDepth - 1) + val node = new Node(nodeIndex, stats.predict, isLeaf, Some(split), None, None, Some(stats)) + logDebug("Node = " + node) + nodes(nodeIndex) = node + } + + /** + * Extract the decision tree node information for the children of the node + */ + private def extractInfoForLowerLevels( + level: Int, + index: Int, + maxDepth: Int, + nodeSplitStats: (Split, InformationGainStats), + parentImpurities: Array[Double], + filters: Array[List[Filter]]): Unit = { + // 0 corresponds to the left child node and 1 corresponds to the right child node. + // TODO: Convert to while loop + for (i <- 0 to 1) { + // Calculate the index of the node from the node level and the index at the current level. + val nodeIndex = scala.math.pow(2, level + 1).toInt - 1 + 2 * index + i + if (level < maxDepth - 1) { + val impurity = if (i == 0) { + nodeSplitStats._2.leftImpurity + } else { + nodeSplitStats._2.rightImpurity + } + logDebug("nodeIndex = " + nodeIndex + ", impurity = " + impurity) + // noting the parent impurities + parentImpurities(nodeIndex) = impurity + // noting the parents filters for the child nodes + val childFilter = new Filter(nodeSplitStats._1, if (i == 0) -1 else 1) + filters(nodeIndex) = childFilter :: filters((nodeIndex - 1) / 2) + for (filter <- filters(nodeIndex)) { + logDebug("Filter = " + filter) + } + } + } + } +} + +object DecisionTree extends Serializable with Logging { + + /** + * Method to train a decision tree model where the instances are represented as an RDD of + * (label, features) pairs. The method supports binary classification and regression. For the + * binary classification, the label for each instance should either be 0 or 1 to denote the two + * classes. The parameters for the algorithm are specified using the strategy parameter. + * + * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data + * for DecisionTree + * @param strategy The configuration parameters for the tree algorithm which specify the type + * of algorithm (classification, regression, etc.), feature type (continuous, + * categorical), depth of the tree, quantile calculation strategy, etc. + * @return a DecisionTreeModel that can be used for prediction + */ + def train(input: RDD[LabeledPoint], strategy: Strategy): DecisionTreeModel = { + new DecisionTree(strategy).train(input: RDD[LabeledPoint]) + } + + /** + * Method to train a decision tree model where the instances are represented as an RDD of + * (label, features) pairs. The method supports binary classification and regression. For the + * binary classification, the label for each instance should either be 0 or 1 to denote the two + * classes. + * + * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as + * training data + * @param algo algorithm, classification or regression + * @param impurity impurity criterion used for information gain calculation + * @param maxDepth maxDepth maximum depth of the tree + * @return a DecisionTreeModel that can be used for prediction + */ + def train( + input: RDD[LabeledPoint], + algo: Algo, + impurity: Impurity, + maxDepth: Int): DecisionTreeModel = { + val strategy = new Strategy(algo,impurity,maxDepth) + new DecisionTree(strategy).train(input: RDD[LabeledPoint]) + } + + + /** + * Method to train a decision tree model where the instances are represented as an RDD of + * (label, features) pairs. The decision tree method supports binary classification and + * regression. For the binary classification, the label for each instance should either be 0 or + * 1 to denote the two classes. The method also supports categorical features inputs where the + * number of categories can specified using the categoricalFeaturesInfo option. + * + * @param input input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as + * training data for DecisionTree + * @param algo classification or regression + * @param impurity criterion used for information gain calculation + * @param maxDepth maximum depth of the tree + * @param maxBins maximum number of bins used for splitting features + * @param quantileCalculationStrategy algorithm for calculating quantiles + * @param categoricalFeaturesInfo A map storing information about the categorical variables and + * the number of discrete values they take. For example, + * an entry (n -> k) implies the feature n is categorical with k + * categories 0, 1, 2, ... , k-1. It's important to note that + * features are zero-indexed. + * @return a DecisionTreeModel that can be used for prediction + */ + def train( + input: RDD[LabeledPoint], + algo: Algo, + impurity: Impurity, + maxDepth: Int, + maxBins: Int, + quantileCalculationStrategy: QuantileStrategy, + categoricalFeaturesInfo: Map[Int,Int]): DecisionTreeModel = { + val strategy = new Strategy(algo, impurity, maxDepth, maxBins, quantileCalculationStrategy, + categoricalFeaturesInfo) + new DecisionTree(strategy).train(input: RDD[LabeledPoint]) + } + + private val InvalidBinIndex = -1 + + /** + * Returns an array of optimal splits for all nodes at a given level + * + * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data + * for DecisionTree + * @param parentImpurities Impurities for all parent nodes for the current level + * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing + * parameters for construction the DecisionTree + * @param level Level of the tree + * @param filters Filters for all nodes at a given level + * @param splits possible splits for all features + * @param bins possible bins for all features + * @return array of splits with best splits for all nodes at a given level. + */ + protected[tree] def findBestSplits( + input: RDD[LabeledPoint], + parentImpurities: Array[Double], + strategy: Strategy, + level: Int, + filters: Array[List[Filter]], + splits: Array[Array[Split]], + bins: Array[Array[Bin]]): Array[(Split, InformationGainStats)] = { + + /* + * The high-level description for the best split optimizations are noted here. + * + * *Level-wise training* + * We perform bin calculations for all nodes at the given level to avoid making multiple + * passes over the data. Thus, for a slightly increased computation and storage cost we save + * several iterations over the data especially at higher levels of the decision tree. + * + * *Bin-wise computation* + * We use a bin-wise best split computation strategy instead of a straightforward best split + * computation strategy. Instead of analyzing each sample for contribution to the left/right + * child node impurity of every split, we first categorize each feature of a sample into a + * bin. Each bin is an interval between a low and high split. Since each splits, and thus bin, + * is ordered (read ordering for categorical variables in the findSplitsBins method), + * we exploit this structure to calculate aggregates for bins and then use these aggregates + * to calculate information gain for each split. + * + * *Aggregation over partitions* + * Instead of performing a flatMap/reduceByKey operation, we exploit the fact that we know + * the number of splits in advance. Thus, we store the aggregates (at the appropriate + * indices) in a single array for all bins and rely upon the RDD aggregate method to + * drastically reduce the communication overhead. + */ + + // common calculations for multiple nested methods + val numNodes = scala.math.pow(2, level).toInt + logDebug("numNodes = " + numNodes) + // Find the number of features by looking at the first sample. + val numFeatures = input.first().features.length + logDebug("numFeatures = " + numFeatures) + val numBins = bins(0).length + logDebug("numBins = " + numBins) + + /** Find the filters used before reaching the current code. */ + def findParentFilters(nodeIndex: Int): List[Filter] = { + if (level == 0) { + List[Filter]() + } else { + val nodeFilterIndex = scala.math.pow(2, level).toInt - 1 + nodeIndex + filters(nodeFilterIndex) + } + } + + /** + * Find whether the sample is valid input for the current node, i.e., whether it passes through + * all the filters for the current node. + */ + def isSampleValid(parentFilters: List[Filter], labeledPoint: LabeledPoint): Boolean = { + // leaf + if ((level > 0) & (parentFilters.length == 0)) { + return false + } + + // Apply each filter and check sample validity. Return false when invalid condition found. + for (filter <- parentFilters) { + val features = labeledPoint.features + val featureIndex = filter.split.feature + val threshold = filter.split.threshold + val comparison = filter.comparison + val categories = filter.split.categories + val isFeatureContinuous = filter.split.featureType == Continuous + val feature = features(featureIndex) + if (isFeatureContinuous) { + comparison match { + case -1 => if (feature > threshold) return false + case 1 => if (feature <= threshold) return false + } + } else { + val containsFeature = categories.contains(feature) + comparison match { + case -1 => if (!containsFeature) return false + case 1 => if (containsFeature) return false + } + + } + } + + // Return true when the sample is valid for all filters. + true + } + + /** + * Find bin for one feature. + */ + def findBin( + featureIndex: Int, + labeledPoint: LabeledPoint, + isFeatureContinuous: Boolean): Int = { + val binForFeatures = bins(featureIndex) + val feature = labeledPoint.features(featureIndex) + + /** + * Binary search helper method for continuous feature. + */ + def binarySearchForBins(): Int = { + var left = 0 + var right = binForFeatures.length - 1 + while (left <= right) { + val mid = left + (right - left) / 2 + val bin = binForFeatures(mid) + val lowThreshold = bin.lowSplit.threshold + val highThreshold = bin.highSplit.threshold + if ((lowThreshold < feature) & (highThreshold >= feature)){ + return mid + } + else if (lowThreshold >= feature) { + right = mid - 1 + } + else { + left = mid + 1 + } + } + -1 + } + + /** + * Sequential search helper method to find bin for categorical feature. + */ + def sequentialBinSearchForCategoricalFeature(): Int = { + val numCategoricalBins = strategy.categoricalFeaturesInfo(featureIndex) + var binIndex = 0 + while (binIndex < numCategoricalBins) { + val bin = bins(featureIndex)(binIndex) + val category = bin.category + val features = labeledPoint.features + if (category == features(featureIndex)) { + return binIndex + } + binIndex += 1 + } + -1 + } + + if (isFeatureContinuous) { + // Perform binary search for finding bin for continuous features. + val binIndex = binarySearchForBins() + if (binIndex == -1){ + throw new UnknownError("no bin was found for continuous variable.") + } + binIndex + } else { + // Perform sequential search to find bin for categorical features. + val binIndex = sequentialBinSearchForCategoricalFeature() + if (binIndex == -1){ + throw new UnknownError("no bin was found for categorical variable.") + } + binIndex + } + } + + /** + * Finds bins for all nodes (and all features) at a given level. + * For l nodes, k features the storage is as follows: + * label, b_11, b_12, .. , b_1k, b_21, b_22, .. , b_2k, b_l1, b_l2, .. , b_lk, + * where b_ij is an integer between 0 and numBins - 1. + * Invalid sample is denoted by noting bin for feature 1 as -1. + */ + def findBinsForLevel(labeledPoint: LabeledPoint): Array[Double] = { + // Calculate bin index and label per feature per node. + val arr = new Array[Double](1 + (numFeatures * numNodes)) + arr(0) = labeledPoint.label + var nodeIndex = 0 + while (nodeIndex < numNodes) { + val parentFilters = findParentFilters(nodeIndex) + // Find out whether the sample qualifies for the particular node. + val sampleValid = isSampleValid(parentFilters, labeledPoint) + val shift = 1 + numFeatures * nodeIndex + if (!sampleValid) { + // Mark one bin as -1 is sufficient. + arr(shift) = InvalidBinIndex + } else { + var featureIndex = 0 + while (featureIndex < numFeatures) { + val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + arr(shift + featureIndex) = findBin(featureIndex, labeledPoint,isFeatureContinuous) + featureIndex += 1 + } + } + nodeIndex += 1 + } + arr + } + + /** + * Performs a sequential aggregation over a partition for classification. For l nodes, + * k features, either the left count or the right count of one of the p bins is + * incremented based upon whether the feature is classified as 0 or 1. + * + * @param agg Array[Double] storing aggregate calculation of size + * 2 * numSplits * numFeatures*numNodes for classification + * @param arr Array[Double] of size 1 + (numFeatures * numNodes) + * @return Array[Double] storing aggregate calculation of size + * 2 * numSplits * numFeatures * numNodes for classification + */ + def classificationBinSeqOp(arr: Array[Double], agg: Array[Double]) { + // Iterate over all nodes. + var nodeIndex = 0 + while (nodeIndex < numNodes) { + // Check whether the instance was valid for this nodeIndex. + val validSignalIndex = 1 + numFeatures * nodeIndex + val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex + if (isSampleValidForNode) { + // actual class label + val label = arr(0) + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + // Find the bin index for this feature. + val arrShift = 1 + numFeatures * nodeIndex + val arrIndex = arrShift + featureIndex + // Update the left or right count for one bin. + val aggShift = 2 * numBins * numFeatures * nodeIndex + val aggIndex = aggShift + 2 * featureIndex * numBins + arr(arrIndex).toInt * 2 + label match { + case 0.0 => agg(aggIndex) = agg(aggIndex) + 1 + case 1.0 => agg(aggIndex + 1) = agg(aggIndex + 1) + 1 + } + featureIndex += 1 + } + } + nodeIndex += 1 + } + } + + /** + * Performs a sequential aggregation over a partition for regression. For l nodes, k features, + * the count, sum, sum of squares of one of the p bins is incremented. + * + * @param agg Array[Double] storing aggregate calculation of size + * 3 * numSplits * numFeatures * numNodes for classification + * @param arr Array[Double] of size 1 + (numFeatures * numNodes) + * @return Array[Double] storing aggregate calculation of size + * 3 * numSplits * numFeatures * numNodes for regression + */ + def regressionBinSeqOp(arr: Array[Double], agg: Array[Double]) { + // Iterate over all nodes. + var nodeIndex = 0 + while (nodeIndex < numNodes) { + // Check whether the instance was valid for this nodeIndex. + val validSignalIndex = 1 + numFeatures * nodeIndex + val isSampleValidForNode = arr(validSignalIndex) != InvalidBinIndex + if (isSampleValidForNode) { + // actual class label + val label = arr(0) + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + // Find the bin index for this feature. + val arrShift = 1 + numFeatures * nodeIndex + val arrIndex = arrShift + featureIndex + // Update count, sum, and sum^2 for one bin. + val aggShift = 3 * numBins * numFeatures * nodeIndex + val aggIndex = aggShift + 3 * featureIndex * numBins + arr(arrIndex).toInt * 3 + agg(aggIndex) = agg(aggIndex) + 1 + agg(aggIndex + 1) = agg(aggIndex + 1) + label + agg(aggIndex + 2) = agg(aggIndex + 2) + label*label + featureIndex += 1 + } + } + nodeIndex += 1 + } + } + + /** + * Performs a sequential aggregation over a partition. + */ + def binSeqOp(agg: Array[Double], arr: Array[Double]): Array[Double] = { + strategy.algo match { + case Classification => classificationBinSeqOp(arr, agg) + case Regression => regressionBinSeqOp(arr, agg) + } + agg + } + + // Calculate bin aggregate length for classification or regression. + val binAggregateLength = strategy.algo match { + case Classification => 2 * numBins * numFeatures * numNodes + case Regression => 3 * numBins * numFeatures * numNodes + } + logDebug("binAggregateLength = " + binAggregateLength) + + /** + * Combines the aggregates from partitions. + * @param agg1 Array containing aggregates from one or more partitions + * @param agg2 Array containing aggregates from one or more partitions + * @return Combined aggregate from agg1 and agg2 + */ + def binCombOp(agg1: Array[Double], agg2: Array[Double]): Array[Double] = { + var index = 0 + val combinedAggregate = new Array[Double](binAggregateLength) + while (index < binAggregateLength) { + combinedAggregate(index) = agg1(index) + agg2(index) + index += 1 + } + combinedAggregate + } + + // Find feature bins for all nodes at a level. + val binMappedRDD = input.map(x => findBinsForLevel(x)) + + // Calculate bin aggregates. + val binAggregates = { + binMappedRDD.aggregate(Array.fill[Double](binAggregateLength)(0))(binSeqOp,binCombOp) + } + logDebug("binAggregates.length = " + binAggregates.length) + + /** + * Calculates the information gain for all splits based upon left/right split aggregates. + * @param leftNodeAgg left node aggregates + * @param featureIndex feature index + * @param splitIndex split index + * @param rightNodeAgg right node aggregate + * @param topImpurity impurity of the parent node + * @return information gain and statistics for all splits + */ + def calculateGainForSplit( + leftNodeAgg: Array[Array[Double]], + featureIndex: Int, + splitIndex: Int, + rightNodeAgg: Array[Array[Double]], + topImpurity: Double): InformationGainStats = { + strategy.algo match { + case Classification => + val left0Count = leftNodeAgg(featureIndex)(2 * splitIndex) + val left1Count = leftNodeAgg(featureIndex)(2 * splitIndex + 1) + val leftCount = left0Count + left1Count + + val right0Count = rightNodeAgg(featureIndex)(2 * splitIndex) + val right1Count = rightNodeAgg(featureIndex)(2 * splitIndex + 1) + val rightCount = right0Count + right1Count + + val impurity = { + if (level > 0) { + topImpurity + } else { + // Calculate impurity for root node. + strategy.impurity.calculate(left0Count + right0Count, left1Count + right1Count) + } + } + + if (leftCount == 0) { + return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity,1) + } + if (rightCount == 0) { + return new InformationGainStats(0, topImpurity, topImpurity, Double.MinValue,0) + } + + val leftImpurity = strategy.impurity.calculate(left0Count, left1Count) + val rightImpurity = strategy.impurity.calculate(right0Count, right1Count) + + val leftWeight = leftCount.toDouble / (leftCount + rightCount) + val rightWeight = rightCount.toDouble / (leftCount + rightCount) + + val gain = { + if (level > 0) { + impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + } else { + impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + } + } + + val predict = (left1Count + right1Count) / (leftCount + rightCount) + + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) + case Regression => + val leftCount = leftNodeAgg(featureIndex)(3 * splitIndex) + val leftSum = leftNodeAgg(featureIndex)(3 * splitIndex + 1) + val leftSumSquares = leftNodeAgg(featureIndex)(3 * splitIndex + 2) + + val rightCount = rightNodeAgg(featureIndex)(3 * splitIndex) + val rightSum = rightNodeAgg(featureIndex)(3 * splitIndex + 1) + val rightSumSquares = rightNodeAgg(featureIndex)(3 * splitIndex + 2) + + val impurity = { + if (level > 0) { + topImpurity + } else { + // Calculate impurity for root node. + val count = leftCount + rightCount + val sum = leftSum + rightSum + val sumSquares = leftSumSquares + rightSumSquares + strategy.impurity.calculate(count, sum, sumSquares) + } + } + + if (leftCount == 0) { + return new InformationGainStats(0, topImpurity, Double.MinValue, topImpurity, + rightSum / rightCount) + } + if (rightCount == 0) { + return new InformationGainStats(0, topImpurity ,topImpurity, + Double.MinValue, leftSum / leftCount) + } + + val leftImpurity = strategy.impurity.calculate(leftCount, leftSum, leftSumSquares) + val rightImpurity = strategy.impurity.calculate(rightCount, rightSum, rightSumSquares) + + val leftWeight = leftCount.toDouble / (leftCount + rightCount) + val rightWeight = rightCount.toDouble / (leftCount + rightCount) + + val gain = { + if (level > 0) { + impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + } else { + impurity - leftWeight * leftImpurity - rightWeight * rightImpurity + } + } + + val predict = (leftSum + rightSum) / (leftCount + rightCount) + new InformationGainStats(gain, impurity, leftImpurity, rightImpurity, predict) + } + } + + /** + * Extracts left and right split aggregates. + * @param binData Array[Double] of size 2*numFeatures*numSplits + * @return (leftNodeAgg, rightNodeAgg) tuple of type (Array[Double], + * Array[Double]) where each array is of size(numFeature,2*(numSplits-1)) + */ + def extractLeftRightNodeAggregates( + binData: Array[Double]): (Array[Array[Double]], Array[Array[Double]]) = { + strategy.algo match { + case Classification => + // Initialize left and right split aggregates. + val leftNodeAgg = Array.ofDim[Double](numFeatures, 2 * (numBins - 1)) + val rightNodeAgg = Array.ofDim[Double](numFeatures, 2 * (numBins - 1)) + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + // shift for this featureIndex + val shift = 2 * featureIndex * numBins + + // left node aggregate for the lowest split + leftNodeAgg(featureIndex)(0) = binData(shift + 0) + leftNodeAgg(featureIndex)(1) = binData(shift + 1) + + // right node aggregate for the highest split + rightNodeAgg(featureIndex)(2 * (numBins - 2)) + = binData(shift + (2 * (numBins - 1))) + rightNodeAgg(featureIndex)(2 * (numBins - 2) + 1) + = binData(shift + (2 * (numBins - 1)) + 1) + + // Iterate over all splits. + var splitIndex = 1 + while (splitIndex < numBins - 1) { + // calculating left node aggregate for a split as a sum of left node aggregate of a + // lower split and the left bin aggregate of a bin where the split is a high split + leftNodeAgg(featureIndex)(2 * splitIndex) = binData(shift + 2 * splitIndex) + + leftNodeAgg(featureIndex)(2 * splitIndex - 2) + leftNodeAgg(featureIndex)(2 * splitIndex + 1) = binData(shift + 2 * splitIndex + 1) + + leftNodeAgg(featureIndex)(2 * splitIndex - 2 + 1) + + // calculating right node aggregate for a split as a sum of right node aggregate of a + // higher split and the right bin aggregate of a bin where the split is a low split + rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex)) = + binData(shift + (2 *(numBins - 2 - splitIndex))) + + rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex)) + rightNodeAgg(featureIndex)(2 * (numBins - 2 - splitIndex) + 1) = + binData(shift + (2* (numBins - 2 - splitIndex) + 1)) + + rightNodeAgg(featureIndex)(2 * (numBins - 1 - splitIndex) + 1) + + splitIndex += 1 + } + featureIndex += 1 + } + (leftNodeAgg, rightNodeAgg) + case Regression => + // Initialize left and right split aggregates. + val leftNodeAgg = Array.ofDim[Double](numFeatures, 3 * (numBins - 1)) + val rightNodeAgg = Array.ofDim[Double](numFeatures, 3 * (numBins - 1)) + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + // shift for this featureIndex + val shift = 3 * featureIndex * numBins + // left node aggregate for the lowest split + leftNodeAgg(featureIndex)(0) = binData(shift + 0) + leftNodeAgg(featureIndex)(1) = binData(shift + 1) + leftNodeAgg(featureIndex)(2) = binData(shift + 2) + + // right node aggregate for the highest split + rightNodeAgg(featureIndex)(3 * (numBins - 2)) = + binData(shift + (3 * (numBins - 1))) + rightNodeAgg(featureIndex)(3 * (numBins - 2) + 1) = + binData(shift + (3 * (numBins - 1)) + 1) + rightNodeAgg(featureIndex)(3 * (numBins - 2) + 2) = + binData(shift + (3 * (numBins - 1)) + 2) + + // Iterate over all splits. + var splitIndex = 1 + while (splitIndex < numBins - 1) { + // calculating left node aggregate for a split as a sum of left node aggregate of a + // lower split and the left bin aggregate of a bin where the split is a high split + leftNodeAgg(featureIndex)(3 * splitIndex) = binData(shift + 3 * splitIndex) + + leftNodeAgg(featureIndex)(3 * splitIndex - 3) + leftNodeAgg(featureIndex)(3 * splitIndex + 1) = binData(shift + 3 * splitIndex + 1) + + leftNodeAgg(featureIndex)(3 * splitIndex - 3 + 1) + leftNodeAgg(featureIndex)(3 * splitIndex + 2) = binData(shift + 3 * splitIndex + 2) + + leftNodeAgg(featureIndex)(3 * splitIndex - 3 + 2) + + // calculating right node aggregate for a split as a sum of right node aggregate of a + // higher split and the right bin aggregate of a bin where the split is a low split + rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex)) = + binData(shift + (3 * (numBins - 2 - splitIndex))) + + rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex)) + rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 1) = + binData(shift + (3 * (numBins - 2 - splitIndex) + 1)) + + rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 1) + rightNodeAgg(featureIndex)(3 * (numBins - 2 - splitIndex) + 2) = + binData(shift + (3 * (numBins - 2 - splitIndex) + 2)) + + rightNodeAgg(featureIndex)(3 * (numBins - 1 - splitIndex) + 2) + + splitIndex += 1 + } + featureIndex += 1 + } + (leftNodeAgg, rightNodeAgg) + } + } + + /** + * Calculates information gain for all nodes splits. + */ + def calculateGainsForAllNodeSplits( + leftNodeAgg: Array[Array[Double]], + rightNodeAgg: Array[Array[Double]], + nodeImpurity: Double): Array[Array[InformationGainStats]] = { + val gains = Array.ofDim[InformationGainStats](numFeatures, numBins - 1) + + for (featureIndex <- 0 until numFeatures) { + for (splitIndex <- 0 until numBins - 1) { + gains(featureIndex)(splitIndex) = calculateGainForSplit(leftNodeAgg, featureIndex, + splitIndex, rightNodeAgg, nodeImpurity) + } + } + gains + } + + /** + * Find the best split for a node. + * @param binData Array[Double] of size 2 * numSplits * numFeatures + * @param nodeImpurity impurity of the top node + * @return tuple of split and information gain + */ + def binsToBestSplit( + binData: Array[Double], + nodeImpurity: Double): (Split, InformationGainStats) = { + + logDebug("node impurity = " + nodeImpurity) + + // Extract left right node aggregates. + val (leftNodeAgg, rightNodeAgg) = extractLeftRightNodeAggregates(binData) + + // Calculate gains for all splits. + val gains = calculateGainsForAllNodeSplits(leftNodeAgg, rightNodeAgg, nodeImpurity) + + val (bestFeatureIndex,bestSplitIndex, gainStats) = { + // Initialize with infeasible values. + var bestFeatureIndex = Int.MinValue + var bestSplitIndex = Int.MinValue + var bestGainStats = new InformationGainStats(Double.MinValue, -1.0, -1.0, -1.0, -1.0) + // Iterate over features. + var featureIndex = 0 + while (featureIndex < numFeatures) { + // Iterate over all splits. + var splitIndex = 0 + while (splitIndex < numBins - 1) { + val gainStats = gains(featureIndex)(splitIndex) + if (gainStats.gain > bestGainStats.gain) { + bestGainStats = gainStats + bestFeatureIndex = featureIndex + bestSplitIndex = splitIndex + } + splitIndex += 1 + } + featureIndex += 1 + } + (bestFeatureIndex, bestSplitIndex, bestGainStats) + } + + logDebug("best split bin = " + bins(bestFeatureIndex)(bestSplitIndex)) + logDebug("best split bin = " + splits(bestFeatureIndex)(bestSplitIndex)) + + (splits(bestFeatureIndex)(bestSplitIndex), gainStats) + } + + /** + * Get bin data for one node. + */ + def getBinDataForNode(node: Int): Array[Double] = { + strategy.algo match { + case Classification => + val shift = 2 * node * numBins * numFeatures + val binsForNode = binAggregates.slice(shift, shift + 2 * numBins * numFeatures) + binsForNode + case Regression => + val shift = 3 * node * numBins * numFeatures + val binsForNode = binAggregates.slice(shift, shift + 3 * numBins * numFeatures) + binsForNode + } + } + + // Calculate best splits for all nodes at a given level + val bestSplits = new Array[(Split, InformationGainStats)](numNodes) + // Iterating over all nodes at this level + var node = 0 + while (node < numNodes) { + val nodeImpurityIndex = scala.math.pow(2, level).toInt - 1 + node + val binsForNode: Array[Double] = getBinDataForNode(node) + logDebug("nodeImpurityIndex = " + nodeImpurityIndex) + val parentNodeImpurity = parentImpurities(nodeImpurityIndex) + logDebug("node impurity = " + parentNodeImpurity) + bestSplits(node) = binsToBestSplit(binsForNode, parentNodeImpurity) + node += 1 + } + + bestSplits + } + + /** + * Returns split and bins for decision tree calculation. + * @param input RDD of [[org.apache.spark.mllib.regression.LabeledPoint]] used as training data + * for DecisionTree + * @param strategy [[org.apache.spark.mllib.tree.configuration.Strategy]] instance containing + * parameters for construction the DecisionTree + * @return a tuple of (splits,bins) where splits is an Array of [org.apache.spark.mllib.tree + * .model.Split] of size (numFeatures, numSplits-1) and bins is an Array of [org.apache + * .spark.mllib.tree.model.Bin] of size (numFeatures, numSplits1) + */ + protected[tree] def findSplitsBins( + input: RDD[LabeledPoint], + strategy: Strategy): (Array[Array[Split]], Array[Array[Bin]]) = { + val count = input.count() + + // Find the number of features by looking at the first sample + val numFeatures = input.take(1)(0).features.length + + val maxBins = strategy.maxBins + val numBins = if (maxBins <= count) maxBins else count.toInt + logDebug("numBins = " + numBins) + + /* + * TODO: Add a require statement ensuring #bins is always greater than the categories. + * It's a limitation of the current implementation but a reasonable trade-off since features + * with large number of categories get favored over continuous features. + */ + if (strategy.categoricalFeaturesInfo.size > 0) { + val maxCategoriesForFeatures = strategy.categoricalFeaturesInfo.maxBy(_._2)._2 + require(numBins >= maxCategoriesForFeatures) + } + + // Calculate the number of sample for approximate quantile calculation. + val requiredSamples = numBins*numBins + val fraction = if (requiredSamples < count) requiredSamples.toDouble / count else 1.0 + logDebug("fraction of data used for calculating quantiles = " + fraction) + + // sampled input for RDD calculation + val sampledInput = input.sample(false, fraction, new XORShiftRandom().nextInt()).collect() + val numSamples = sampledInput.length + + val stride: Double = numSamples.toDouble / numBins + logDebug("stride = " + stride) + + strategy.quantileCalculationStrategy match { + case Sort => + val splits = Array.ofDim[Split](numFeatures, numBins - 1) + val bins = Array.ofDim[Bin](numFeatures, numBins) + + // Find all splits. + + // Iterate over all features. + var featureIndex = 0 + while (featureIndex < numFeatures){ + // Check whether the feature is continuous. + val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + if (isFeatureContinuous) { + val featureSamples = sampledInput.map(lp => lp.features(featureIndex)).sorted + val stride: Double = numSamples.toDouble / numBins + logDebug("stride = " + stride) + for (index <- 0 until numBins - 1) { + val sampleIndex = (index + 1) * stride.toInt + val split = new Split(featureIndex, featureSamples(sampleIndex), Continuous, List()) + splits(featureIndex)(index) = split + } + } else { + val maxFeatureValue = strategy.categoricalFeaturesInfo(featureIndex) + require(maxFeatureValue < numBins, "number of categories should be less than number " + + "of bins") + + // For categorical variables, each bin is a category. The bins are sorted and they + // are ordered by calculating the centroid of their corresponding labels. + val centroidForCategories = + sampledInput.map(lp => (lp.features(featureIndex),lp.label)) + .groupBy(_._1) + .mapValues(x => x.map(_._2).sum / x.map(_._1).length) + + // Check for missing categorical variables and putting them last in the sorted list. + val fullCentroidForCategories = scala.collection.mutable.Map[Double,Double]() + for (i <- 0 until maxFeatureValue) { + if (centroidForCategories.contains(i)) { + fullCentroidForCategories(i) = centroidForCategories(i) + } else { + fullCentroidForCategories(i) = Double.MaxValue + } + } + + // bins sorted by centroids + val categoriesSortedByCentroid = fullCentroidForCategories.toList.sortBy(_._2) + + logDebug("centriod for categorical variable = " + categoriesSortedByCentroid) + + var categoriesForSplit = List[Double]() + categoriesSortedByCentroid.iterator.zipWithIndex.foreach { + case ((key, value), index) => + categoriesForSplit = key :: categoriesForSplit + splits(featureIndex)(index) = new Split(featureIndex, Double.MinValue, Categorical, + categoriesForSplit) + bins(featureIndex)(index) = { + if (index == 0) { + new Bin(new DummyCategoricalSplit(featureIndex, Categorical), + splits(featureIndex)(0), Categorical, key) + } else { + new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), + Categorical, key) + } + } + } + } + featureIndex += 1 + } + + // Find all bins. + featureIndex = 0 + while (featureIndex < numFeatures) { + val isFeatureContinuous = strategy.categoricalFeaturesInfo.get(featureIndex).isEmpty + if (isFeatureContinuous) { // Bins for categorical variables are already assigned. + bins(featureIndex)(0) = new Bin(new DummyLowSplit(featureIndex, Continuous), + splits(featureIndex)(0), Continuous, Double.MinValue) + for (index <- 1 until numBins - 1){ + val bin = new Bin(splits(featureIndex)(index-1), splits(featureIndex)(index), + Continuous, Double.MinValue) + bins(featureIndex)(index) = bin + } + bins(featureIndex)(numBins-1) = new Bin(splits(featureIndex)(numBins-2), + new DummyHighSplit(featureIndex, Continuous), Continuous, Double.MinValue) + } + featureIndex += 1 + } + (splits,bins) + case MinMax => + throw new UnsupportedOperationException("minmax not supported yet.") + case ApproxHist => + throw new UnsupportedOperationException("approximate histogram not supported yet.") + } + } + + val usage = """ + Usage: DecisionTreeRunner [slices] --algo --trainDataDir path --testDataDir path --maxDepth num [--impurity ] [--maxBins num] + """ + + def main(args: Array[String]) { + + if (args.length < 2) { + System.err.println(usage) + System.exit(1) + } + + val sc = new SparkContext(args(0), "DecisionTree") + + val argList = args.toList.drop(1) + type OptionMap = Map[Symbol, Any] + + def nextOption(map : OptionMap, list: List[String]): OptionMap = { + list match { + case Nil => map + case "--algo" :: string :: tail => nextOption(map ++ Map('algo -> string), tail) + case "--impurity" :: string :: tail => nextOption(map ++ Map('impurity -> string), tail) + case "--maxDepth" :: string :: tail => nextOption(map ++ Map('maxDepth -> string), tail) + case "--maxBins" :: string :: tail => nextOption(map ++ Map('maxBins -> string), tail) + case "--trainDataDir" :: string :: tail => nextOption(map ++ Map('trainDataDir -> string) + , tail) + case "--testDataDir" :: string :: tail => nextOption(map ++ Map('testDataDir -> string), + tail) + case string :: Nil => nextOption(map ++ Map('infile -> string), list.tail) + case option :: tail => logError("Unknown option " + option) + sys.exit(1) + } + } + val options = nextOption(Map(), argList) + logDebug(options.toString()) + + // Load training data. + val trainData = loadLabeledData(sc, options.get('trainDataDir).get.toString) + + // Identify the type of algorithm. + val algoStr = options.get('algo).get.toString + val algo = algoStr match { + case "Classification" => Classification + case "Regression" => Regression + } + + // Identify the type of impurity. + val impurityStr = options.getOrElse('impurity, + if (algo == Classification) "Gini" else "Variance").toString + val impurity = impurityStr match { + case "Gini" => Gini + case "Entropy" => Entropy + case "Variance" => Variance + } + + val maxDepth = options.getOrElse('maxDepth, "1").toString.toInt + val maxBins = options.getOrElse('maxBins, "100").toString.toInt + + val strategy = new Strategy(algo, impurity, maxDepth, maxBins) + val model = DecisionTree.train(trainData, strategy) + + // Load test data. + val testData = loadLabeledData(sc, options.get('testDataDir).get.toString) + + // Measure algorithm accuracy + if (algo == Classification) { + val accuracy = accuracyScore(model, testData) + logDebug("accuracy = " + accuracy) + } + + if (algo == Regression) { + val mse = meanSquaredError(model, testData) + logDebug("mean square error = " + mse) + } + + sc.stop() + } + + /** + * Load labeled data from a file. The data format used here is + * , ..., + * where , are feature values in Double and is the corresponding label as Double. + * + * @param sc SparkContext + * @param dir Directory to the input data files. + * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is + * the label, and the second element represents the feature values (an array of Double). + */ + def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { + sc.textFile(dir).map { line => + val parts = line.trim().split(",") + val label = parts(0).toDouble + val features = parts.slice(1,parts.length).map(_.toDouble) + LabeledPoint(label, features) + } + } + + // TODO: Port this method to a generic metrics package. + /** + * Calculates the classifier accuracy. + */ + private def accuracyScore(model: DecisionTreeModel, data: RDD[LabeledPoint], + threshold: Double = 0.5): Double = { + def predictedValue(features: Array[Double]) = { + if (model.predict(features) < threshold) 0.0 else 1.0 + } + val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() + val count = data.count() + logDebug("correct prediction count = " + correctCount) + logDebug("data count = " + count) + correctCount.toDouble / count + } + + // TODO: Port this method to a generic metrics package + /** + * Calculates the mean squared error for regression. + */ + private def meanSquaredError(tree: DecisionTreeModel, data: RDD[LabeledPoint]): Double = { + data.map { y => + val err = tree.predict(y.features) - y.label + err * err + }.mean() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md b/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md new file mode 100644 index 0000000000000..0fd71aa9735bc --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/README.md @@ -0,0 +1,17 @@ +This package contains the default implementation of the decision tree algorithm. + +The decision tree algorithm supports: ++ Binary classification ++ Regression ++ Information loss calculation with entropy and gini for classification and variance for regression ++ Both continuous and categorical features + +# Tree improvements ++ Node model pruning ++ Printing to dot files + +# Future Ensemble Extensions + ++ Random forests ++ Boosting ++ Extremely randomized trees diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala new file mode 100644 index 0000000000000..2dd1f0f27b8f5 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.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.mllib.tree.configuration + +/** + * Enum to select the algorithm for the decision tree + */ +object Algo extends Enumeration { + type Algo = Value + val Classification, Regression = Value +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala new file mode 100644 index 0000000000000..09ee0586c58fa --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.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.mllib.tree.configuration + +/** + * Enum to describe whether a feature is "continuous" or "categorical" + */ +object FeatureType extends Enumeration { + type FeatureType = Value + val Continuous, Categorical = Value +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala new file mode 100644 index 0000000000000..2457a480c2a14 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.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.mllib.tree.configuration + +/** + * Enum for selecting the quantile calculation strategy + */ +object QuantileStrategy extends Enumeration { + type QuantileStrategy = Value + val Sort, MinMax, ApproxHist = Value +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala new file mode 100644 index 0000000000000..df565f3eb8859 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -0,0 +1,43 @@ +/* + * 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.mllib.tree.configuration + +import org.apache.spark.mllib.tree.impurity.Impurity +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ + +/** + * Stores all the configuration options for tree construction + * @param algo classification or regression + * @param impurity criterion used for information gain calculation + * @param maxDepth maximum depth of the tree + * @param maxBins maximum number of bins used for splitting features + * @param quantileCalculationStrategy algorithm for calculating quantiles + * @param categoricalFeaturesInfo A map storing information about the categorical variables and the + * number of discrete values they take. For example, an entry (n -> + * k) implies the feature n is categorical with k categories 0, + * 1, 2, ... , k-1. It's important to note that features are + * zero-indexed. + */ +class Strategy ( + val algo: Algo, + val impurity: Impurity, + val maxDepth: Int, + val maxBins: Int = 100, + val quantileCalculationStrategy: QuantileStrategy = Sort, + val categoricalFeaturesInfo: Map[Int,Int] = Map[Int,Int]()) extends Serializable diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala new file mode 100644 index 0000000000000..b93995fcf9441 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -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. + */ + +package org.apache.spark.mllib.tree.impurity + +/** + * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during + * binary classification. + */ +object Entropy extends Impurity { + + def log2(x: Double) = scala.math.log(x) / scala.math.log(2) + + /** + * entropy calculation + * @param c0 count of instances with label 0 + * @param c1 count of instances with label 1 + * @return entropy value + */ + def calculate(c0: Double, c1: Double): Double = { + if (c0 == 0 || c1 == 0) { + 0 + } else { + val total = c0 + c1 + val f0 = c0 / total + val f1 = c1 / total + -(f0 * log2(f0)) - (f1 * log2(f1)) + } + } + + def calculate(count: Double, sum: Double, sumSquares: Double): Double = + throw new UnsupportedOperationException("Entropy.calculate") +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala new file mode 100644 index 0000000000000..c0407554a91b3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.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.mllib.tree.impurity + +/** + * Class for calculating the + * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] + * during binary classification. + */ +object Gini extends Impurity { + + /** + * Gini coefficient calculation + * @param c0 count of instances with label 0 + * @param c1 count of instances with label 1 + * @return Gini coefficient value + */ + override def calculate(c0: Double, c1: Double): Double = { + if (c0 == 0 || c1 == 0) { + 0 + } else { + val total = c0 + c1 + val f0 = c0 / total + val f1 = c1 / total + 1 - f0 * f0 - f1 * f1 + } + } + + def calculate(count: Double, sum: Double, sumSquares: Double): Double = + throw new UnsupportedOperationException("Gini.calculate") +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala new file mode 100644 index 0000000000000..a4069063af2ad --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -0,0 +1,42 @@ +/* + * 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.mllib.tree.impurity + +/** + * Trait for calculating information gain. + */ +trait Impurity extends Serializable { + + /** + * information calculation for binary classification + * @param c0 count of instances with label 0 + * @param c1 count of instances with label 1 + * @return information value + */ + def calculate(c0 : Double, c1 : Double): Double + + /** + * information calculation for regression + * @param count number of instances + * @param sum sum of labels + * @param sumSquares summation of squares of the labels + * @return information value + */ + def calculate(count: Double, sum: Double, sumSquares: Double): Double + +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala new file mode 100644 index 0000000000000..b74577dcec167 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.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.mllib.tree.impurity + +/** + * Class for calculating variance during regression + */ +object Variance extends Impurity { + override def calculate(c0: Double, c1: Double): Double = + throw new UnsupportedOperationException("Variance.calculate") + + /** + * variance calculation + * @param count number of instances + * @param sum sum of labels + * @param sumSquares summation of squares of the labels + */ + override def calculate(count: Double, sum: Double, sumSquares: Double): Double = { + val squaredLoss = sumSquares - (sum * sum) / count + squaredLoss / count + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala new file mode 100644 index 0000000000000..a57faa13745f7 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.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.mllib.tree.model + +import org.apache.spark.mllib.tree.configuration.FeatureType._ + +/** + * Used for "binning" the features bins for faster best split calculation. For a continuous + * feature, a bin is determined by a low and a high "split". For a categorical feature, + * the a bin is determined using a single label value (category). + * @param lowSplit signifying the lower threshold for the continuous feature to be + * accepted in the bin + * @param highSplit signifying the upper threshold for the continuous feature to be + * accepted in the bin + * @param featureType type of feature -- categorical or continuous + * @param category categorical label value accepted in the bin + */ +case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala new file mode 100644 index 0000000000000..a8bbf21daec01 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -0,0 +1,49 @@ +/* + * 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.mllib.tree.model + +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.rdd.RDD + +/** + * Model to store the decision tree parameters + * @param topNode root node + * @param algo algorithm type -- classification or regression + */ +class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable { + + /** + * Predict values for a single data point using the model trained. + * + * @param features array representing a single data point + * @return Double prediction from the trained model + */ + def predict(features: Array[Double]): Double = { + topNode.predictIfLeaf(features) + } + + /** + * Predict values for the given data set using the model trained. + * + * @param features RDD representing data points to be predicted + * @return RDD[Int] where each entry contains the corresponding prediction + */ + def predict(features: RDD[Array[Double]]): RDD[Double] = { + features.map(x => predict(x)) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala new file mode 100644 index 0000000000000..ebc9595eafef3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala @@ -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. + */ + +package org.apache.spark.mllib.tree.model + +/** + * Filter specifying a split and type of comparison to be applied on features + * @param split split specifying the feature index, type and threshold + * @param comparison integer specifying <,=,> + */ +case class Filter(split: Split, comparison: Int) { + // Comparison -1,0,1 signifies <.=,> + override def toString = " split = " + split + "comparison = " + comparison +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala new file mode 100644 index 0000000000000..99bf79cf12e45 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -0,0 +1,39 @@ +/* + * 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.mllib.tree.model + +/** + * Information gain statistics for each split + * @param gain information gain value + * @param impurity current node impurity + * @param leftImpurity left node impurity + * @param rightImpurity right node impurity + * @param predict predicted value + */ +class InformationGainStats( + val gain: Double, + val impurity: Double, + val leftImpurity: Double, + val rightImpurity: Double, + val predict: Double) extends Serializable { + + override def toString = { + "gain = %f, impurity = %f, left impurity = %f, right impurity = %f, predict = %f" + .format(gain, impurity, leftImpurity, rightImpurity, predict) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala new file mode 100644 index 0000000000000..ea4693c5c2f4e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -0,0 +1,90 @@ +/* + * 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.mllib.tree.model + +import org.apache.spark.Logging +import org.apache.spark.mllib.tree.configuration.FeatureType._ + +/** + * Node in a decision tree + * @param id integer node id + * @param predict predicted value at the node + * @param isLeaf whether the leaf is a node + * @param split split to calculate left and right nodes + * @param leftNode left child + * @param rightNode right child + * @param stats information gain stats + */ +class Node ( + val id: Int, + val predict: Double, + val isLeaf: Boolean, + val split: Option[Split], + var leftNode: Option[Node], + var rightNode: Option[Node], + val stats: Option[InformationGainStats]) extends Serializable with Logging { + + override def toString = "id = " + id + ", isLeaf = " + isLeaf + ", predict = " + predict + ", " + + "split = " + split + ", stats = " + stats + + /** + * build the left node and right nodes if not leaf + * @param nodes array of nodes + */ + def build(nodes: Array[Node]): Unit = { + + logDebug("building node " + id + " at level " + + (scala.math.log(id + 1)/scala.math.log(2)).toInt ) + logDebug("id = " + id + ", split = " + split) + logDebug("stats = " + stats) + logDebug("predict = " + predict) + if (!isLeaf) { + val leftNodeIndex = id*2 + 1 + val rightNodeIndex = id*2 + 2 + leftNode = Some(nodes(leftNodeIndex)) + rightNode = Some(nodes(rightNodeIndex)) + leftNode.get.build(nodes) + rightNode.get.build(nodes) + } + } + + /** + * predict value if node is not leaf + * @param feature feature value + * @return predicted value + */ + def predictIfLeaf(feature: Array[Double]) : Double = { + if (isLeaf) { + predict + } else{ + if (split.get.featureType == Continuous) { + if (feature(split.get.feature) <= split.get.threshold) { + leftNode.get.predictIfLeaf(feature) + } else { + rightNode.get.predictIfLeaf(feature) + } + } else { + if (split.get.categories.contains(feature(split.get.feature))) { + leftNode.get.predictIfLeaf(feature) + } else { + rightNode.get.predictIfLeaf(feature) + } + } + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala new file mode 100644 index 0000000000000..4e64a81dda74e --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -0,0 +1,64 @@ +/* + * 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.mllib.tree.model + +import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType + +/** + * Split applied to a feature + * @param feature feature index + * @param threshold threshold for continuous feature + * @param featureType type of feature -- categorical or continuous + * @param categories accepted values for categorical variables + */ +case class Split( + feature: Int, + threshold: Double, + featureType: FeatureType, + categories: List[Double]){ + + override def toString = + "Feature = " + feature + ", threshold = " + threshold + ", featureType = " + featureType + + ", categories = " + categories +} + +/** + * Split with minimum threshold for continuous features. Helps with the smallest bin creation. + * @param feature feature index + * @param featureType type of feature -- categorical or continuous + */ +class DummyLowSplit(feature: Int, featureType: FeatureType) + extends Split(feature, Double.MinValue, featureType, List()) + +/** + * Split with maximum threshold for continuous features. Helps with the highest bin creation. + * @param feature feature index + * @param featureType type of feature -- categorical or continuous + */ +class DummyHighSplit(feature: Int, featureType: FeatureType) + extends Split(feature, Double.MaxValue, featureType, List()) + +/** + * Split with no acceptable feature values for categorical features. Helps with the first bin + * creation. + * @param feature feature index + * @param featureType type of feature -- categorical or continuous + */ +class DummyCategoricalSplit(feature: Int, featureType: FeatureType) + extends Split(feature, Double.MaxValue, featureType, List()) + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala new file mode 100644 index 0000000000000..4349c7000a0ae --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -0,0 +1,425 @@ +/* + * 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.mllib.tree + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Variance} +import org.apache.spark.mllib.tree.model.Filter +import org.apache.spark.mllib.tree.configuration.Strategy +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.configuration.FeatureType._ + +class DecisionTreeSuite extends FunSuite with BeforeAndAfterAll { + + @transient private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + } + + override def afterAll() { + sc.stop() + System.clearProperty("spark.driver.port") + } + + test("split and bin calculation") { + val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(Classification, Gini, 3, 100) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + assert(splits.length === 2) + assert(bins.length === 2) + assert(splits(0).length === 99) + assert(bins(0).length === 100) + } + + test("split and bin calculation for categorical variables") { + val arr = DecisionTreeSuite.generateCategoricalDataPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Classification, + Gini, + maxDepth = 3, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 2, 1-> 2)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + assert(splits.length === 2) + assert(bins.length === 2) + assert(splits(0).length === 99) + assert(bins(0).length === 100) + + // Check splits. + + assert(splits(0)(0).feature === 0) + assert(splits(0)(0).threshold === Double.MinValue) + assert(splits(0)(0).featureType === Categorical) + assert(splits(0)(0).categories.length === 1) + assert(splits(0)(0).categories.contains(1.0)) + + assert(splits(0)(1).feature === 0) + assert(splits(0)(1).threshold === Double.MinValue) + assert(splits(0)(1).featureType === Categorical) + assert(splits(0)(1).categories.length === 2) + assert(splits(0)(1).categories.contains(1.0)) + assert(splits(0)(1).categories.contains(0.0)) + + assert(splits(0)(2) === null) + + assert(splits(1)(0).feature === 1) + assert(splits(1)(0).threshold === Double.MinValue) + assert(splits(1)(0).featureType === Categorical) + assert(splits(1)(0).categories.length === 1) + assert(splits(1)(0).categories.contains(0.0)) + + assert(splits(1)(1).feature === 1) + assert(splits(1)(1).threshold === Double.MinValue) + assert(splits(1)(1).featureType === Categorical) + assert(splits(1)(1).categories.length === 2) + assert(splits(1)(1).categories.contains(1.0)) + assert(splits(1)(1).categories.contains(0.0)) + + assert(splits(1)(2) === null) + + // Check bins. + + assert(bins(0)(0).category === 1.0) + assert(bins(0)(0).lowSplit.categories.length === 0) + assert(bins(0)(0).highSplit.categories.length === 1) + assert(bins(0)(0).highSplit.categories.contains(1.0)) + + assert(bins(0)(1).category === 0.0) + assert(bins(0)(1).lowSplit.categories.length === 1) + assert(bins(0)(1).lowSplit.categories.contains(1.0)) + assert(bins(0)(1).highSplit.categories.length === 2) + assert(bins(0)(1).highSplit.categories.contains(1.0)) + assert(bins(0)(1).highSplit.categories.contains(0.0)) + + assert(bins(0)(2) === null) + + assert(bins(1)(0).category === 0.0) + assert(bins(1)(0).lowSplit.categories.length === 0) + assert(bins(1)(0).highSplit.categories.length === 1) + assert(bins(1)(0).highSplit.categories.contains(0.0)) + + assert(bins(1)(1).category === 1.0) + assert(bins(1)(1).lowSplit.categories.length === 1) + assert(bins(1)(1).lowSplit.categories.contains(0.0)) + assert(bins(1)(1).highSplit.categories.length === 2) + assert(bins(1)(1).highSplit.categories.contains(0.0)) + assert(bins(1)(1).highSplit.categories.contains(1.0)) + + assert(bins(1)(2) === null) + } + + test("split and bin calculations for categorical variables with no sample for one category") { + val arr = DecisionTreeSuite.generateCategoricalDataPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Classification, + Gini, + maxDepth = 3, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + + // Check splits. + + assert(splits(0)(0).feature === 0) + assert(splits(0)(0).threshold === Double.MinValue) + assert(splits(0)(0).featureType === Categorical) + assert(splits(0)(0).categories.length === 1) + assert(splits(0)(0).categories.contains(1.0)) + + assert(splits(0)(1).feature === 0) + assert(splits(0)(1).threshold === Double.MinValue) + assert(splits(0)(1).featureType === Categorical) + assert(splits(0)(1).categories.length === 2) + assert(splits(0)(1).categories.contains(1.0)) + assert(splits(0)(1).categories.contains(0.0)) + + assert(splits(0)(2).feature === 0) + assert(splits(0)(2).threshold === Double.MinValue) + assert(splits(0)(2).featureType === Categorical) + assert(splits(0)(2).categories.length === 3) + assert(splits(0)(2).categories.contains(1.0)) + assert(splits(0)(2).categories.contains(0.0)) + assert(splits(0)(2).categories.contains(2.0)) + + assert(splits(0)(3) === null) + + assert(splits(1)(0).feature === 1) + assert(splits(1)(0).threshold === Double.MinValue) + assert(splits(1)(0).featureType === Categorical) + assert(splits(1)(0).categories.length === 1) + assert(splits(1)(0).categories.contains(0.0)) + + assert(splits(1)(1).feature === 1) + assert(splits(1)(1).threshold === Double.MinValue) + assert(splits(1)(1).featureType === Categorical) + assert(splits(1)(1).categories.length === 2) + assert(splits(1)(1).categories.contains(1.0)) + assert(splits(1)(1).categories.contains(0.0)) + + assert(splits(1)(2).feature === 1) + assert(splits(1)(2).threshold === Double.MinValue) + assert(splits(1)(2).featureType === Categorical) + assert(splits(1)(2).categories.length === 3) + assert(splits(1)(2).categories.contains(1.0)) + assert(splits(1)(2).categories.contains(0.0)) + assert(splits(1)(2).categories.contains(2.0)) + + assert(splits(1)(3) === null) + + // Check bins. + + assert(bins(0)(0).category === 1.0) + assert(bins(0)(0).lowSplit.categories.length === 0) + assert(bins(0)(0).highSplit.categories.length === 1) + assert(bins(0)(0).highSplit.categories.contains(1.0)) + + assert(bins(0)(1).category === 0.0) + assert(bins(0)(1).lowSplit.categories.length === 1) + assert(bins(0)(1).lowSplit.categories.contains(1.0)) + assert(bins(0)(1).highSplit.categories.length === 2) + assert(bins(0)(1).highSplit.categories.contains(1.0)) + assert(bins(0)(1).highSplit.categories.contains(0.0)) + + assert(bins(0)(2).category === 2.0) + assert(bins(0)(2).lowSplit.categories.length === 2) + assert(bins(0)(2).lowSplit.categories.contains(1.0)) + assert(bins(0)(2).lowSplit.categories.contains(0.0)) + assert(bins(0)(2).highSplit.categories.length === 3) + assert(bins(0)(2).highSplit.categories.contains(1.0)) + assert(bins(0)(2).highSplit.categories.contains(0.0)) + assert(bins(0)(2).highSplit.categories.contains(2.0)) + + assert(bins(0)(3) === null) + + assert(bins(1)(0).category === 0.0) + assert(bins(1)(0).lowSplit.categories.length === 0) + assert(bins(1)(0).highSplit.categories.length === 1) + assert(bins(1)(0).highSplit.categories.contains(0.0)) + + assert(bins(1)(1).category === 1.0) + assert(bins(1)(1).lowSplit.categories.length === 1) + assert(bins(1)(1).lowSplit.categories.contains(0.0)) + assert(bins(1)(1).highSplit.categories.length === 2) + assert(bins(1)(1).highSplit.categories.contains(0.0)) + assert(bins(1)(1).highSplit.categories.contains(1.0)) + + assert(bins(1)(2).category === 2.0) + assert(bins(1)(2).lowSplit.categories.length === 2) + assert(bins(1)(2).lowSplit.categories.contains(0.0)) + assert(bins(1)(2).lowSplit.categories.contains(1.0)) + assert(bins(1)(2).highSplit.categories.length === 3) + assert(bins(1)(2).highSplit.categories.contains(0.0)) + assert(bins(1)(2).highSplit.categories.contains(1.0)) + assert(bins(1)(2).highSplit.categories.contains(2.0)) + + assert(bins(1)(3) === null) + } + + test("classification stump with all categorical variables") { + val arr = DecisionTreeSuite.generateCategoricalDataPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Classification, + Gini, + maxDepth = 3, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, + Array[List[Filter]](), splits, bins) + + val split = bestSplits(0)._1 + assert(split.categories.length === 1) + assert(split.categories.contains(1.0)) + assert(split.featureType === Categorical) + assert(split.threshold === Double.MinValue) + + val stats = bestSplits(0)._2 + assert(stats.gain > 0) + assert(stats.predict > 0.4) + assert(stats.predict < 0.5) + assert(stats.impurity > 0.2) + } + + test("regression stump with all categorical variables") { + val arr = DecisionTreeSuite.generateCategoricalDataPoints() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy( + Regression, + Variance, + maxDepth = 3, + maxBins = 100, + categoricalFeaturesInfo = Map(0 -> 3, 1-> 3)) + val (splits, bins) = DecisionTree.findSplitsBins(rdd,strategy) + val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, + Array[List[Filter]](), splits, bins) + + val split = bestSplits(0)._1 + assert(split.categories.length === 1) + assert(split.categories.contains(1.0)) + assert(split.featureType === Categorical) + assert(split.threshold === Double.MinValue) + + val stats = bestSplits(0)._2 + assert(stats.gain > 0) + assert(stats.predict > 0.4) + assert(stats.predict < 0.5) + assert(stats.impurity > 0.2) + } + + test("stump with fixed label 0 for Gini") { + val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(Classification, Gini, 3, 100) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + assert(splits.length === 2) + assert(splits(0).length === 99) + assert(bins.length === 2) + assert(bins(0).length === 100) + assert(splits(0).length === 99) + assert(bins(0).length === 100) + + val bestSplits = DecisionTree.findBestSplits(rdd, new Array(7), strategy, 0, + Array[List[Filter]](), splits, bins) + assert(bestSplits.length === 1) + assert(bestSplits(0)._1.feature === 0) + assert(bestSplits(0)._1.threshold === 10) + assert(bestSplits(0)._2.gain === 0) + assert(bestSplits(0)._2.leftImpurity === 0) + assert(bestSplits(0)._2.rightImpurity === 0) + } + + test("stump with fixed label 1 for Gini") { + val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(Classification, Gini, 3, 100) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + assert(splits.length === 2) + assert(splits(0).length === 99) + assert(bins.length === 2) + assert(bins(0).length === 100) + assert(splits(0).length === 99) + assert(bins(0).length === 100) + + val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, + Array[List[Filter]](), splits, bins) + assert(bestSplits.length === 1) + assert(bestSplits(0)._1.feature === 0) + assert(bestSplits(0)._1.threshold === 10) + assert(bestSplits(0)._2.gain === 0) + assert(bestSplits(0)._2.leftImpurity === 0) + assert(bestSplits(0)._2.rightImpurity === 0) + assert(bestSplits(0)._2.predict === 1) + } + + test("stump with fixed label 0 for Entropy") { + val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel0() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(Classification, Entropy, 3, 100) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + assert(splits.length === 2) + assert(splits(0).length === 99) + assert(bins.length === 2) + assert(bins(0).length === 100) + assert(splits(0).length === 99) + assert(bins(0).length === 100) + + val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, + Array[List[Filter]](), splits, bins) + assert(bestSplits.length === 1) + assert(bestSplits(0)._1.feature === 0) + assert(bestSplits(0)._1.threshold === 10) + assert(bestSplits(0)._2.gain === 0) + assert(bestSplits(0)._2.leftImpurity === 0) + assert(bestSplits(0)._2.rightImpurity === 0) + assert(bestSplits(0)._2.predict === 0) + } + + test("stump with fixed label 1 for Entropy") { + val arr = DecisionTreeSuite.generateOrderedLabeledPointsWithLabel1() + assert(arr.length === 1000) + val rdd = sc.parallelize(arr) + val strategy = new Strategy(Classification, Entropy, 3, 100) + val (splits, bins) = DecisionTree.findSplitsBins(rdd, strategy) + assert(splits.length === 2) + assert(splits(0).length === 99) + assert(bins.length === 2) + assert(bins(0).length === 100) + assert(splits(0).length === 99) + assert(bins(0).length === 100) + + val bestSplits = DecisionTree.findBestSplits(rdd, Array(0.0), strategy, 0, + Array[List[Filter]](), splits, bins) + assert(bestSplits.length === 1) + assert(bestSplits(0)._1.feature === 0) + assert(bestSplits(0)._1.threshold === 10) + assert(bestSplits(0)._2.gain === 0) + assert(bestSplits(0)._2.leftImpurity === 0) + assert(bestSplits(0)._2.rightImpurity === 0) + assert(bestSplits(0)._2.predict === 1) + } +} + +object DecisionTreeSuite { + + def generateOrderedLabeledPointsWithLabel0(): Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](1000) + for (i <- 0 until 1000){ + val lp = new LabeledPoint(0.0,Array(i.toDouble,1000.0-i)) + arr(i) = lp + } + arr + } + + def generateOrderedLabeledPointsWithLabel1(): Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](1000) + for (i <- 0 until 1000){ + val lp = new LabeledPoint(1.0,Array(i.toDouble,999.0-i)) + arr(i) = lp + } + arr + } + + def generateCategoricalDataPoints(): Array[LabeledPoint] = { + val arr = new Array[LabeledPoint](1000) + for (i <- 0 until 1000){ + if (i < 600){ + arr(i) = new LabeledPoint(1.0,Array(0.0,1.0)) + } else { + arr(i) = new LabeledPoint(0.0,Array(1.0,0.0)) + } + } + arr + } +} From ea9de658a365dca2b7403d8fab68a8a87c4e06c8 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 1 Apr 2014 23:54:38 -0700 Subject: [PATCH 175/397] Remove * from test case golden filename. @rxin mentioned this might cause issues on windows machines. Author: Michael Armbrust Closes #297 from marmbrus/noStars and squashes the following commits: 263122a [Michael Armbrust] Remove * from test case golden filename. --- .../resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d | 1 - .../golden/alias.star-0-7bdb861d11e895aaea545810cdac316d | 1 + .../apache/spark/sql/hive/execution/HiveResolutionSuite.scala | 2 +- 3 files changed, 2 insertions(+), 2 deletions(-) delete mode 100644 sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d create mode 100644 sql/hive/src/test/resources/golden/alias.star-0-7bdb861d11e895aaea545810cdac316d diff --git a/sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d b/sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d deleted file mode 100644 index 5f4de85940513..0000000000000 --- a/sql/hive/src/test/resources/golden/alias.*-0-7bdb861d11e895aaea545810cdac316d +++ /dev/null @@ -1 +0,0 @@ -0 val_0 \ No newline at end of file diff --git a/sql/hive/src/test/resources/golden/alias.star-0-7bdb861d11e895aaea545810cdac316d b/sql/hive/src/test/resources/golden/alias.star-0-7bdb861d11e895aaea545810cdac316d new file mode 100644 index 0000000000000..016f64cc26f2a --- /dev/null +++ b/sql/hive/src/test/resources/golden/alias.star-0-7bdb861d11e895aaea545810cdac316d @@ -0,0 +1 @@ +0 val_0 diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index d77900ddc950c..40c4e23f90fb8 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -48,7 +48,7 @@ class HiveResolutionSuite extends HiveComparisonTest { createQueryTest("attr", "SELECT key FROM src a ORDER BY key LIMIT 1") - createQueryTest("alias.*", + createQueryTest("alias.star", "SELECT a.* FROM src a ORDER BY key LIMIT 1") test("case insensitivity with scala reflection") { From 11973a7bdad58fdb759033c232d87f0b279c83b4 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Wed, 2 Apr 2014 10:35:52 -0700 Subject: [PATCH 176/397] Renamed stageIdToActiveJob to jobIdToActiveJob. This data structure was misused and, as a result, later renamed to an incorrect name. This data structure seems to have gotten into this tangled state as a result of @henrydavidge using the stageID instead of the job Id to index into it and later @andrewor14 renaming the data structure to reflect this misunderstanding. This patch renames it and removes an incorrect indexing into it. The incorrect indexing into it meant that the code added by @henrydavidge to warn when a task size is too large (added here https://github.com/apache/spark/commit/57579934f0454f258615c10e69ac2adafc5b9835) was not always executed; this commit fixes that. Author: Kay Ousterhout Closes #301 from kayousterhout/fixCancellation and squashes the following commits: bd3d3a4 [Kay Ousterhout] Renamed stageIdToActiveJob to jobIdToActiveJob. --- .../apache/spark/scheduler/DAGScheduler.scala | 21 +++++++++---------- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- 2 files changed, 11 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 4fce47e1ee8de..ef3d24d746829 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -84,7 +84,7 @@ class DAGScheduler( private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]] private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage] private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] - private[scheduler] val stageIdToActiveJob = new HashMap[Int, ActiveJob] + private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] @@ -536,7 +536,7 @@ class DAGScheduler( listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) runLocally(job) } else { - stageIdToActiveJob(jobId) = job + jobIdToActiveJob(jobId) = job activeJobs += job resultStageToJob(finalStage) = job listenerBus.post( @@ -559,7 +559,7 @@ class DAGScheduler( // Cancel all running jobs. runningStages.map(_.jobId).foreach(handleJobCancellation) activeJobs.clear() // These should already be empty by this point, - stageIdToActiveJob.clear() // but just in case we lost track of some jobs... + jobIdToActiveJob.clear() // but just in case we lost track of some jobs... case ExecutorAdded(execId, host) => handleExecutorAdded(execId, host) @@ -569,7 +569,6 @@ class DAGScheduler( case BeginEvent(task, taskInfo) => for ( - job <- stageIdToActiveJob.get(task.stageId); stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage) ) { @@ -697,7 +696,7 @@ class DAGScheduler( private def activeJobForStage(stage: Stage): Option[Int] = { if (stageIdToJobIds.contains(stage.id)) { val jobsThatUseStage: Array[Int] = stageIdToJobIds(stage.id).toArray.sorted - jobsThatUseStage.find(stageIdToActiveJob.contains) + jobsThatUseStage.find(jobIdToActiveJob.contains) } else { None } @@ -750,8 +749,8 @@ class DAGScheduler( } } - val properties = if (stageIdToActiveJob.contains(jobId)) { - stageIdToActiveJob(stage.jobId).properties + val properties = if (jobIdToActiveJob.contains(jobId)) { + jobIdToActiveJob(stage.jobId).properties } else { // this stage will be assigned to "default" pool null @@ -827,7 +826,7 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - stageIdToActiveJob -= stage.jobId + jobIdToActiveJob -= stage.jobId activeJobs -= job resultStageToJob -= stage markStageAsFinished(stage) @@ -986,11 +985,11 @@ class DAGScheduler( val independentStages = removeJobAndIndependentStages(jobId) independentStages.foreach(taskScheduler.cancelTasks) val error = new SparkException("Job %d cancelled".format(jobId)) - val job = stageIdToActiveJob(jobId) + val job = jobIdToActiveJob(jobId) job.listener.jobFailed(error) jobIdToStageIds -= jobId activeJobs -= job - stageIdToActiveJob -= jobId + jobIdToActiveJob -= jobId listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) } } @@ -1011,7 +1010,7 @@ class DAGScheduler( val error = new SparkException("Job aborted: " + reason) job.listener.jobFailed(error) jobIdToStageIdsRemove(job.jobId) - stageIdToActiveJob -= resultStage.jobId + jobIdToActiveJob -= resultStage.jobId activeJobs -= job resultStageToJob -= resultStage listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index c97543f57d8f3..ce567b0cde85d 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -428,7 +428,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(scheduler.pendingTasks.isEmpty) assert(scheduler.activeJobs.isEmpty) assert(scheduler.failedStages.isEmpty) - assert(scheduler.stageIdToActiveJob.isEmpty) + assert(scheduler.jobIdToActiveJob.isEmpty) assert(scheduler.jobIdToStageIds.isEmpty) assert(scheduler.stageIdToJobIds.isEmpty) assert(scheduler.stageIdToStage.isEmpty) From de8eefa804e229635eaa29a78b9e9ce161ac58e1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 2 Apr 2014 10:43:09 -0700 Subject: [PATCH 177/397] [SPARK-1385] Use existing code for JSON de/serialization of BlockId `BlockId.scala` offers a way to reconstruct a BlockId from a string through regex matching. `util/JsonProtocol.scala` duplicates this functionality by explicitly matching on the BlockId type. With this PR, the de/serialization of BlockIds will go through the first (older) code path. (Most of the line changes in this PR involve changing `==` to `===` in `JsonProtocolSuite.scala`) Author: Andrew Or Closes #289 from andrewor14/blockid-json and squashes the following commits: 409d226 [Andrew Or] Simplify JSON de/serialization for BlockId --- .../org/apache/spark/util/JsonProtocol.scala | 77 +--------- .../apache/spark/util/JsonProtocolSuite.scala | 141 +++++++++--------- 2 files changed, 72 insertions(+), 146 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 346f2b7856791..d9a6af61872d1 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -195,7 +195,7 @@ private[spark] object JsonProtocol { taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => JArray(blocks.toList.map { case (id, status) => - ("Block ID" -> blockIdToJson(id)) ~ + ("Block ID" -> id.toString) ~ ("Status" -> blockStatusToJson(status)) }) }.getOrElse(JNothing) @@ -284,35 +284,6 @@ private[spark] object JsonProtocol { ("Replication" -> storageLevel.replication) } - def blockIdToJson(blockId: BlockId): JValue = { - val blockType = Utils.getFormattedClassName(blockId) - val json: JObject = blockId match { - case rddBlockId: RDDBlockId => - ("RDD ID" -> rddBlockId.rddId) ~ - ("Split Index" -> rddBlockId.splitIndex) - case shuffleBlockId: ShuffleBlockId => - ("Shuffle ID" -> shuffleBlockId.shuffleId) ~ - ("Map ID" -> shuffleBlockId.mapId) ~ - ("Reduce ID" -> shuffleBlockId.reduceId) - case broadcastBlockId: BroadcastBlockId => - "Broadcast ID" -> broadcastBlockId.broadcastId - case broadcastHelperBlockId: BroadcastHelperBlockId => - ("Broadcast Block ID" -> blockIdToJson(broadcastHelperBlockId.broadcastId)) ~ - ("Helper Type" -> broadcastHelperBlockId.hType) - case taskResultBlockId: TaskResultBlockId => - "Task ID" -> taskResultBlockId.taskId - case streamBlockId: StreamBlockId => - ("Stream ID" -> streamBlockId.streamId) ~ - ("Unique ID" -> streamBlockId.uniqueId) - case tempBlockId: TempBlockId => - val uuid = UUIDToJson(tempBlockId.id) - "Temp ID" -> uuid - case testBlockId: TestBlockId => - "Test ID" -> testBlockId.id - } - ("Type" -> blockType) ~ json - } - def blockStatusToJson(blockStatus: BlockStatus): JValue = { val storageLevel = storageLevelToJson(blockStatus.storageLevel) ("Storage Level" -> storageLevel) ~ @@ -513,7 +484,7 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => value.extract[List[JValue]].map { block => - val id = blockIdFromJson(block \ "Block ID") + val id = BlockId((block \ "Block ID").extract[String]) val status = blockStatusFromJson(block \ "Status") (id, status) } @@ -616,50 +587,6 @@ private[spark] object JsonProtocol { StorageLevel(useDisk, useMemory, deserialized, replication) } - def blockIdFromJson(json: JValue): BlockId = { - val rddBlockId = Utils.getFormattedClassName(RDDBlockId) - val shuffleBlockId = Utils.getFormattedClassName(ShuffleBlockId) - val broadcastBlockId = Utils.getFormattedClassName(BroadcastBlockId) - val broadcastHelperBlockId = Utils.getFormattedClassName(BroadcastHelperBlockId) - val taskResultBlockId = Utils.getFormattedClassName(TaskResultBlockId) - val streamBlockId = Utils.getFormattedClassName(StreamBlockId) - val tempBlockId = Utils.getFormattedClassName(TempBlockId) - val testBlockId = Utils.getFormattedClassName(TestBlockId) - - (json \ "Type").extract[String] match { - case `rddBlockId` => - val rddId = (json \ "RDD ID").extract[Int] - val splitIndex = (json \ "Split Index").extract[Int] - new RDDBlockId(rddId, splitIndex) - case `shuffleBlockId` => - val shuffleId = (json \ "Shuffle ID").extract[Int] - val mapId = (json \ "Map ID").extract[Int] - val reduceId = (json \ "Reduce ID").extract[Int] - new ShuffleBlockId(shuffleId, mapId, reduceId) - case `broadcastBlockId` => - val broadcastId = (json \ "Broadcast ID").extract[Long] - new BroadcastBlockId(broadcastId) - case `broadcastHelperBlockId` => - val broadcastBlockId = - blockIdFromJson(json \ "Broadcast Block ID").asInstanceOf[BroadcastBlockId] - val hType = (json \ "Helper Type").extract[String] - new BroadcastHelperBlockId(broadcastBlockId, hType) - case `taskResultBlockId` => - val taskId = (json \ "Task ID").extract[Long] - new TaskResultBlockId(taskId) - case `streamBlockId` => - val streamId = (json \ "Stream ID").extract[Int] - val uniqueId = (json \ "Unique ID").extract[Long] - new StreamBlockId(streamId, uniqueId) - case `tempBlockId` => - val tempId = UUIDFromJson(json \ "Temp ID") - new TempBlockId(tempId) - case `testBlockId` => - val testId = (json \ "Test ID").extract[String] - new TestBlockId(testId) - } - } - def blockStatusFromJson(json: JValue): BlockStatus = { val storageLevel = storageLevelFromJson(json \ "Storage Level") val memorySize = (json \ "Memory Size").extract[Long] diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 67c0a434c9b52..40c29014c4b59 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -112,7 +112,6 @@ class JsonProtocolSuite extends FunSuite { testBlockId(BroadcastHelperBlockId(BroadcastBlockId(2L), "Spark")) testBlockId(TaskResultBlockId(1L)) testBlockId(StreamBlockId(1, 2L)) - testBlockId(TempBlockId(UUID.randomUUID())) } @@ -168,8 +167,8 @@ class JsonProtocolSuite extends FunSuite { } private def testBlockId(blockId: BlockId) { - val newBlockId = JsonProtocol.blockIdFromJson(JsonProtocol.blockIdToJson(blockId)) - blockId == newBlockId + val newBlockId = BlockId(blockId.toString) + assert(blockId === newBlockId) } @@ -180,90 +179,90 @@ class JsonProtocolSuite extends FunSuite { private def assertEquals(event1: SparkListenerEvent, event2: SparkListenerEvent) { (event1, event2) match { case (e1: SparkListenerStageSubmitted, e2: SparkListenerStageSubmitted) => - assert(e1.properties == e2.properties) + assert(e1.properties === e2.properties) assertEquals(e1.stageInfo, e2.stageInfo) case (e1: SparkListenerStageCompleted, e2: SparkListenerStageCompleted) => assertEquals(e1.stageInfo, e2.stageInfo) case (e1: SparkListenerTaskStart, e2: SparkListenerTaskStart) => - assert(e1.stageId == e2.stageId) + assert(e1.stageId === e2.stageId) assertEquals(e1.taskInfo, e2.taskInfo) case (e1: SparkListenerTaskGettingResult, e2: SparkListenerTaskGettingResult) => assertEquals(e1.taskInfo, e2.taskInfo) case (e1: SparkListenerTaskEnd, e2: SparkListenerTaskEnd) => - assert(e1.stageId == e2.stageId) - assert(e1.taskType == e2.taskType) + assert(e1.stageId === e2.stageId) + assert(e1.taskType === e2.taskType) assertEquals(e1.reason, e2.reason) assertEquals(e1.taskInfo, e2.taskInfo) assertEquals(e1.taskMetrics, e2.taskMetrics) case (e1: SparkListenerJobStart, e2: SparkListenerJobStart) => - assert(e1.jobId == e2.jobId) - assert(e1.properties == e2.properties) - assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 == i2)) + assert(e1.jobId === e2.jobId) + assert(e1.properties === e2.properties) + assertSeqEquals(e1.stageIds, e2.stageIds, (i1: Int, i2: Int) => assert(i1 === i2)) case (e1: SparkListenerJobEnd, e2: SparkListenerJobEnd) => - assert(e1.jobId == e2.jobId) + assert(e1.jobId === e2.jobId) assertEquals(e1.jobResult, e2.jobResult) case (e1: SparkListenerEnvironmentUpdate, e2: SparkListenerEnvironmentUpdate) => assertEquals(e1.environmentDetails, e2.environmentDetails) case (e1: SparkListenerBlockManagerAdded, e2: SparkListenerBlockManagerAdded) => - assert(e1.maxMem == e2.maxMem) + assert(e1.maxMem === e2.maxMem) assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => - assert(e1.rddId == e2.rddId) + assert(e1.rddId === e2.rddId) case (SparkListenerShutdown, SparkListenerShutdown) => case _ => fail("Events don't match in types!") } } private def assertEquals(info1: StageInfo, info2: StageInfo) { - assert(info1.stageId == info2.stageId) - assert(info1.name == info2.name) - assert(info1.numTasks == info2.numTasks) - assert(info1.submissionTime == info2.submissionTime) - assert(info1.completionTime == info2.completionTime) - assert(info1.emittedTaskSizeWarning == info2.emittedTaskSizeWarning) + assert(info1.stageId === info2.stageId) + assert(info1.name === info2.name) + assert(info1.numTasks === info2.numTasks) + assert(info1.submissionTime === info2.submissionTime) + assert(info1.completionTime === info2.completionTime) + assert(info1.emittedTaskSizeWarning === info2.emittedTaskSizeWarning) assertEquals(info1.rddInfo, info2.rddInfo) } private def assertEquals(info1: RDDInfo, info2: RDDInfo) { - assert(info1.id == info2.id) - assert(info1.name == info2.name) - assert(info1.numPartitions == info2.numPartitions) - assert(info1.numCachedPartitions == info2.numCachedPartitions) - assert(info1.memSize == info2.memSize) - assert(info1.diskSize == info2.diskSize) + assert(info1.id === info2.id) + assert(info1.name === info2.name) + assert(info1.numPartitions === info2.numPartitions) + assert(info1.numCachedPartitions === info2.numCachedPartitions) + assert(info1.memSize === info2.memSize) + assert(info1.diskSize === info2.diskSize) assertEquals(info1.storageLevel, info2.storageLevel) } private def assertEquals(level1: StorageLevel, level2: StorageLevel) { - assert(level1.useDisk == level2.useDisk) - assert(level1.useMemory == level2.useMemory) - assert(level1.deserialized == level2.deserialized) - assert(level1.replication == level2.replication) + assert(level1.useDisk === level2.useDisk) + assert(level1.useMemory === level2.useMemory) + assert(level1.deserialized === level2.deserialized) + assert(level1.replication === level2.replication) } private def assertEquals(info1: TaskInfo, info2: TaskInfo) { - assert(info1.taskId == info2.taskId) - assert(info1.index == info2.index) - assert(info1.launchTime == info2.launchTime) - assert(info1.executorId == info2.executorId) - assert(info1.host == info2.host) - assert(info1.taskLocality == info2.taskLocality) - assert(info1.gettingResultTime == info2.gettingResultTime) - assert(info1.finishTime == info2.finishTime) - assert(info1.failed == info2.failed) - assert(info1.serializedSize == info2.serializedSize) + assert(info1.taskId === info2.taskId) + assert(info1.index === info2.index) + assert(info1.launchTime === info2.launchTime) + assert(info1.executorId === info2.executorId) + assert(info1.host === info2.host) + assert(info1.taskLocality === info2.taskLocality) + assert(info1.gettingResultTime === info2.gettingResultTime) + assert(info1.finishTime === info2.finishTime) + assert(info1.failed === info2.failed) + assert(info1.serializedSize === info2.serializedSize) } private def assertEquals(metrics1: TaskMetrics, metrics2: TaskMetrics) { - assert(metrics1.hostname == metrics2.hostname) - assert(metrics1.executorDeserializeTime == metrics2.executorDeserializeTime) - assert(metrics1.resultSize == metrics2.resultSize) - assert(metrics1.jvmGCTime == metrics2.jvmGCTime) - assert(metrics1.resultSerializationTime == metrics2.resultSerializationTime) - assert(metrics1.memoryBytesSpilled == metrics2.memoryBytesSpilled) - assert(metrics1.diskBytesSpilled == metrics2.diskBytesSpilled) + assert(metrics1.hostname === metrics2.hostname) + assert(metrics1.executorDeserializeTime === metrics2.executorDeserializeTime) + assert(metrics1.resultSize === metrics2.resultSize) + assert(metrics1.jvmGCTime === metrics2.jvmGCTime) + assert(metrics1.resultSerializationTime === metrics2.resultSerializationTime) + assert(metrics1.memoryBytesSpilled === metrics2.memoryBytesSpilled) + assert(metrics1.diskBytesSpilled === metrics2.diskBytesSpilled) assertOptionEquals( metrics1.shuffleReadMetrics, metrics2.shuffleReadMetrics, assertShuffleReadEquals) assertOptionEquals( @@ -272,31 +271,31 @@ class JsonProtocolSuite extends FunSuite { } private def assertEquals(metrics1: ShuffleReadMetrics, metrics2: ShuffleReadMetrics) { - assert(metrics1.shuffleFinishTime == metrics2.shuffleFinishTime) - assert(metrics1.totalBlocksFetched == metrics2.totalBlocksFetched) - assert(metrics1.remoteBlocksFetched == metrics2.remoteBlocksFetched) - assert(metrics1.localBlocksFetched == metrics2.localBlocksFetched) - assert(metrics1.fetchWaitTime == metrics2.fetchWaitTime) - assert(metrics1.remoteBytesRead == metrics2.remoteBytesRead) + assert(metrics1.shuffleFinishTime === metrics2.shuffleFinishTime) + assert(metrics1.totalBlocksFetched === metrics2.totalBlocksFetched) + assert(metrics1.remoteBlocksFetched === metrics2.remoteBlocksFetched) + assert(metrics1.localBlocksFetched === metrics2.localBlocksFetched) + assert(metrics1.fetchWaitTime === metrics2.fetchWaitTime) + assert(metrics1.remoteBytesRead === metrics2.remoteBytesRead) } private def assertEquals(metrics1: ShuffleWriteMetrics, metrics2: ShuffleWriteMetrics) { - assert(metrics1.shuffleBytesWritten == metrics2.shuffleBytesWritten) - assert(metrics1.shuffleWriteTime == metrics2.shuffleWriteTime) + assert(metrics1.shuffleBytesWritten === metrics2.shuffleBytesWritten) + assert(metrics1.shuffleWriteTime === metrics2.shuffleWriteTime) } private def assertEquals(bm1: BlockManagerId, bm2: BlockManagerId) { - assert(bm1.executorId == bm2.executorId) - assert(bm1.host == bm2.host) - assert(bm1.port == bm2.port) - assert(bm1.nettyPort == bm2.nettyPort) + assert(bm1.executorId === bm2.executorId) + assert(bm1.host === bm2.host) + assert(bm1.port === bm2.port) + assert(bm1.nettyPort === bm2.nettyPort) } private def assertEquals(result1: JobResult, result2: JobResult) { (result1, result2) match { case (JobSucceeded, JobSucceeded) => case (r1: JobFailed, r2: JobFailed) => - assert(r1.failedStageId == r2.failedStageId) + assert(r1.failedStageId === r2.failedStageId) assertEquals(r1.exception, r2.exception) case _ => fail("Job results don't match in types!") } @@ -307,13 +306,13 @@ class JsonProtocolSuite extends FunSuite { case (Success, Success) => case (Resubmitted, Resubmitted) => case (r1: FetchFailed, r2: FetchFailed) => - assert(r1.shuffleId == r2.shuffleId) - assert(r1.mapId == r2.mapId) - assert(r1.reduceId == r2.reduceId) + assert(r1.shuffleId === r2.shuffleId) + assert(r1.mapId === r2.mapId) + assert(r1.reduceId === r2.reduceId) assertEquals(r1.bmAddress, r2.bmAddress) case (r1: ExceptionFailure, r2: ExceptionFailure) => - assert(r1.className == r2.className) - assert(r1.description == r2.description) + assert(r1.className === r2.className) + assert(r1.description === r2.description) assertSeqEquals(r1.stackTrace, r2.stackTrace, assertStackTraceElementEquals) assertOptionEquals(r1.metrics, r2.metrics, assertTaskMetricsEquals) case (TaskResultLost, TaskResultLost) => @@ -329,13 +328,13 @@ class JsonProtocolSuite extends FunSuite { details2: Map[String, Seq[(String, String)]]) { details1.zip(details2).foreach { case ((key1, values1: Seq[(String, String)]), (key2, values2: Seq[(String, String)])) => - assert(key1 == key2) - values1.zip(values2).foreach { case (v1, v2) => assert(v1 == v2) } + assert(key1 === key2) + values1.zip(values2).foreach { case (v1, v2) => assert(v1 === v2) } } } private def assertEquals(exception1: Exception, exception2: Exception) { - assert(exception1.getMessage == exception2.getMessage) + assert(exception1.getMessage === exception2.getMessage) assertSeqEquals( exception1.getStackTrace, exception2.getStackTrace, @@ -344,11 +343,11 @@ class JsonProtocolSuite extends FunSuite { private def assertJsonStringEquals(json1: String, json2: String) { val formatJsonString = (json: String) => json.replaceAll("[\\s|]", "") - formatJsonString(json1) == formatJsonString(json2) + formatJsonString(json1) === formatJsonString(json2) } private def assertSeqEquals[T](seq1: Seq[T], seq2: Seq[T], assertEquals: (T, T) => Unit) { - assert(seq1.length == seq2.length) + assert(seq1.length === seq2.length) seq1.zip(seq2).foreach { case (t1, t2) => assertEquals(t1, t2) } @@ -389,11 +388,11 @@ class JsonProtocolSuite extends FunSuite { } private def assertBlockEquals(b1: (BlockId, BlockStatus), b2: (BlockId, BlockStatus)) { - assert(b1 == b2) + assert(b1 === b2) } private def assertStackTraceElementEquals(ste1: StackTraceElement, ste2: StackTraceElement) { - assert(ste1 == ste2) + assert(ste1 === ste2) } From 78236334e4ca7518b6d7d9b38464dbbda854a777 Mon Sep 17 00:00:00 2001 From: Daniel Darabos Date: Wed, 2 Apr 2014 12:27:37 -0700 Subject: [PATCH 178/397] Do not re-use objects in the EdgePartition/EdgeTriplet iterators. This avoids a silent data corruption issue (https://spark-project.atlassian.net/browse/SPARK-1188) and has no performance impact by my measurements. It also simplifies the code. As far as I can tell the object re-use was nothing but premature optimization. I did actual benchmarks for all the included changes, and there is no performance difference. I am not sure where to put the benchmarks. Does Spark not have a benchmark suite? This is an example benchmark I did: test("benchmark") { val builder = new EdgePartitionBuilder[Int] for (i <- (1 to 10000000)) { builder.add(i.toLong, i.toLong, i) } val p = builder.toEdgePartition p.map(_.attr + 1).iterator.toList } It ran for 10 seconds both before and after this change. Author: Daniel Darabos Closes #276 from darabos/spark-1188 and squashes the following commits: 574302b [Daniel Darabos] Restore "manual" copying in EdgePartition.map(Iterator). Add comment to discourage novices like myself from trying to simplify the code. 4117a64 [Daniel Darabos] Revert EdgePartitionSuite. 4955697 [Daniel Darabos] Create a copy of the Edge objects in EdgeRDD.compute(). This avoids exposing the object re-use, while still enables the more efficient behavior for internal code. 4ec77f8 [Daniel Darabos] Add comments about object re-use to the affected functions. 2da5e87 [Daniel Darabos] Restore object re-use in EdgePartition. 0182f2b [Daniel Darabos] Do not re-use objects in the EdgePartition/EdgeTriplet iterators. This avoids a silent data corruption issue (SPARK-1188) and has no performance impact in my measurements. It also simplifies the code. c55f52f [Daniel Darabos] Tests that reproduce the problems from SPARK-1188. --- .../org/apache/spark/graphx/EdgeRDD.scala | 3 +- .../spark/graphx/impl/EdgePartition.scala | 15 +++++-- .../graphx/impl/EdgeTripletIterator.scala | 7 +-- .../impl/EdgeTripletIteratorSuite.scala | 43 +++++++++++++++++++ 4 files changed, 58 insertions(+), 10 deletions(-) create mode 100644 graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala diff --git a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala index f2296a865e1b3..6d04bf790e3a5 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/EdgeRDD.scala @@ -45,7 +45,8 @@ class EdgeRDD[@specialized ED: ClassTag]( partitionsRDD.partitioner.orElse(Some(Partitioner.defaultPartitioner(partitionsRDD))) override def compute(part: Partition, context: TaskContext): Iterator[Edge[ED]] = { - firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context).next._2.iterator + val p = firstParent[(PartitionID, EdgePartition[ED])].iterator(part, context) + p.next._2.iterator.map(_.copy()) } override def collect(): Array[Edge[ED]] = this.map(_.copy()).collect() diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 57fa5eefd5e09..2e05f5d4e4969 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -56,6 +56,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * Construct a new edge partition by applying the function f to all * edges in this partition. * + * Be careful not to keep references to the objects passed to `f`. + * To improve GC performance the same object is re-used for each call. + * * @param f a function from an edge to a new attribute * @tparam ED2 the type of the new attribute * @return a new edge partition with the result of the function `f` @@ -84,12 +87,12 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) * order of the edges returned by `EdgePartition.iterator` and * should return attributes equal to the number of edges. * - * @param f a function from an edge to a new attribute + * @param iter an iterator for the new attribute values * @tparam ED2 the type of the new attribute - * @return a new edge partition with the result of the function `f` - * applied to each edge + * @return a new edge partition with the attribute values replaced */ def map[ED2: ClassTag](iter: Iterator[ED2]): EdgePartition[ED2] = { + // Faster than iter.toArray, because the expected size is known. val newData = new Array[ED2](data.size) var i = 0 while (iter.hasNext) { @@ -188,6 +191,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) /** * Get an iterator over the edges in this partition. * + * Be careful not to keep references to the objects from this iterator. + * To improve GC performance the same object is re-used in `next()`. + * * @return an iterator over edges in the partition */ def iterator = new Iterator[Edge[ED]] { @@ -216,6 +222,9 @@ class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) /** * Get an iterator over the cluster of edges in this partition with source vertex id `srcId`. The * cluster must start at position `index`. + * + * Be careful not to keep references to the objects from this iterator. To improve GC performance + * the same object is re-used in `next()`. */ private def clusterIterator(srcId: VertexId, index: Int) = new Iterator[Edge[ED]] { private[this] val edge = new Edge[ED] diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala index 886c250d7cffd..220a89d73d711 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgeTripletIterator.scala @@ -37,20 +37,15 @@ class EdgeTripletIterator[VD: ClassTag, ED: ClassTag]( // Current position in the array. private var pos = 0 - // A triplet object that this iterator.next() call returns. We reuse this object to avoid - // allocating too many temporary Java objects. - private val triplet = new EdgeTriplet[VD, ED] - private val vmap = new PrimitiveKeyOpenHashMap[VertexId, VD](vidToIndex, vertexArray) override def hasNext: Boolean = pos < edgePartition.size override def next() = { + val triplet = new EdgeTriplet[VD, ED] triplet.srcId = edgePartition.srcIds(pos) - // assert(vmap.containsKey(e.src.id)) triplet.srcAttr = vmap(triplet.srcId) triplet.dstId = edgePartition.dstIds(pos) - // assert(vmap.containsKey(e.dst.id)) triplet.dstAttr = vmap(triplet.dstId) triplet.attr = edgePartition.data(pos) pos += 1 diff --git a/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala new file mode 100644 index 0000000000000..9cbb2d2acdc2d --- /dev/null +++ b/graphx/src/test/scala/org/apache/spark/graphx/impl/EdgeTripletIteratorSuite.scala @@ -0,0 +1,43 @@ +/* + * 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.graphx.impl + +import scala.reflect.ClassTag +import scala.util.Random + +import org.scalatest.FunSuite + +import org.apache.spark.graphx._ + +class EdgeTripletIteratorSuite extends FunSuite { + test("iterator.toList") { + val builder = new EdgePartitionBuilder[Int] + builder.add(1, 2, 0) + builder.add(1, 3, 0) + builder.add(1, 4, 0) + val vidmap = new VertexIdToIndexMap + vidmap.add(1) + vidmap.add(2) + vidmap.add(3) + vidmap.add(4) + val vs = Array.fill(vidmap.capacity)(0) + val iter = new EdgeTripletIterator[Int, Int](vidmap, vs, builder.toEdgePartition) + val result = iter.toList.map(et => (et.srcId, et.dstId)) + assert(result === Seq((1, 2), (1, 3), (1, 4))) + } +} From 1faa57971192226837bea32eb29eae5bfb425a7e Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Wed, 2 Apr 2014 12:47:22 -0700 Subject: [PATCH 179/397] [SPARK-1371][WIP] Compression support for Spark SQL in-memory columnar storage JIRA issue: [SPARK-1373](https://issues.apache.org/jira/browse/SPARK-1373) (Although tagged as WIP, this PR is structurally complete. The only things left unimplemented are 3 more compression algorithms: `BooleanBitSet`, `IntDelta` and `LongDelta`, which are trivial to add later in this or another separate PR.) This PR contains compression support for Spark SQL in-memory columnar storage. Main interfaces include: * `CompressionScheme` Each `CompressionScheme` represents a concrete compression algorithm, which basically consists of an `Encoder` for compression and a `Decoder` for decompression. Algorithms implemented include: * `RunLengthEncoding` * `DictionaryEncoding` Algorithms to be implemented include: * `BooleanBitSet` * `IntDelta` * `LongDelta` * `CompressibleColumnBuilder` A stackable `ColumnBuilder` trait used to build byte buffers for compressible columns. A best `CompressionScheme` that exhibits lowest compression ratio is chosen for each column according to statistical information gathered while elements are appended into the `ColumnBuilder`. However, if no `CompressionScheme` can achieve a compression ratio better than 80%, no compression will be done for this column to save CPU time. Memory layout of the final byte buffer is showed below: ``` .--------------------------- Column type ID (4 bytes) | .----------------------- Null count N (4 bytes) | | .------------------- Null positions (4 x N bytes, empty if null count is zero) | | | .------------- Compression scheme ID (4 bytes) | | | | .--------- Compressed non-null elements V V V V V +---+---+-----+---+---------+ | | | ... | | ... ... | +---+---+-----+---+---------+ \-----------/ \-----------/ header body ``` * `CompressibleColumnAccessor` A stackable `ColumnAccessor` trait used to iterate (possibly) compressed data column. * `ColumnStats` Used to collect statistical information while loading data into in-memory columnar table. Optimizations like partition pruning rely on this information. Strictly speaking, `ColumnStats` related code is not part of the compression support. It's contained in this PR to ensure and validate the row-based API design (which is used to avoid boxing/unboxing cost whenever possible). A major refactoring change since PR #205 is: * Refactored all getter/setter methods for primitive types in various places into `ColumnType` classes to remove duplicated code. Author: Cheng Lian Closes #285 from liancheng/memColumnarCompression and squashes the following commits: ed71bbd [Cheng Lian] Addressed all PR comments by @marmbrus d3a4fa9 [Cheng Lian] Removed Ordering[T] in ColumnStats for better performance 5034453 [Cheng Lian] Bug fix, more tests, and more refactoring c298b76 [Cheng Lian] Test suites refactored 2780d6a [Cheng Lian] [WIP] in-memory columnar compression support 211331c [Cheng Lian] WIP: in-memory columnar compression support 85cc59b [Cheng Lian] Refactored ColumnAccessors & ColumnBuilders to remove duplicate code --- .../spark/sql/columnar/ColumnAccessor.scala | 103 ++--- .../spark/sql/columnar/ColumnBuilder.scala | 125 +++--- .../spark/sql/columnar/ColumnStats.scala | 360 ++++++++++++++++++ .../spark/sql/columnar/ColumnType.scala | 87 ++++- ....scala => InMemoryColumnarTableScan.scala} | 7 +- .../sql/columnar/NullableColumnAccessor.scala | 2 +- .../sql/columnar/NullableColumnBuilder.scala | 29 +- .../CompressibleColumnAccessor.scala | 36 ++ .../CompressibleColumnBuilder.scala | 95 +++++ .../compression/CompressionScheme.scala | 94 +++++ .../compression/compressionSchemes.scala | 288 ++++++++++++++ .../spark/sql/columnar/ColumnStatsSuite.scala | 61 +++ .../spark/sql/columnar/ColumnTypeSuite.scala | 216 +++++------ .../sql/columnar/ColumnarQuerySuite.scala | 4 +- .../spark/sql/columnar/ColumnarTestData.scala | 55 --- .../sql/columnar/ColumnarTestUtils.scala | 100 +++++ .../NullableColumnAccessorSuite.scala | 43 ++- .../columnar/NullableColumnBuilderSuite.scala | 61 +-- .../compression/DictionaryEncodingSuite.scala | 113 ++++++ .../compression/RunLengthEncodingSuite.scala | 130 +++++++ .../TestCompressibleColumnBuilder.scala | 43 +++ 21 files changed, 1644 insertions(+), 408 deletions(-) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala rename sql/core/src/main/scala/org/apache/spark/sql/columnar/{inMemoryColumnarOperators.scala => InMemoryColumnarTableScan.scala} (93%) create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala delete mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index e0c98ecdf8f22..ffd4894b5213d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -21,7 +21,7 @@ import java.nio.{ByteOrder, ByteBuffer} import org.apache.spark.sql.catalyst.types.{BinaryType, NativeType, DataType} import org.apache.spark.sql.catalyst.expressions.MutableRow -import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.columnar.compression.CompressibleColumnAccessor /** * An `Iterator` like trait used to extract values from columnar byte buffer. When a value is @@ -41,121 +41,66 @@ private[sql] trait ColumnAccessor { protected def underlyingBuffer: ByteBuffer } -private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType](buffer: ByteBuffer) +private[sql] abstract class BasicColumnAccessor[T <: DataType, JvmType]( + protected val buffer: ByteBuffer, + protected val columnType: ColumnType[T, JvmType]) extends ColumnAccessor { protected def initialize() {} - def columnType: ColumnType[T, JvmType] - def hasNext = buffer.hasRemaining def extractTo(row: MutableRow, ordinal: Int) { - doExtractTo(row, ordinal) + columnType.setField(row, ordinal, extractSingle(buffer)) } - protected def doExtractTo(row: MutableRow, ordinal: Int) + def extractSingle(buffer: ByteBuffer): JvmType = columnType.extract(buffer) protected def underlyingBuffer = buffer } private[sql] abstract class NativeColumnAccessor[T <: NativeType]( - buffer: ByteBuffer, - val columnType: NativeColumnType[T]) - extends BasicColumnAccessor[T, T#JvmType](buffer) + override protected val buffer: ByteBuffer, + override protected val columnType: NativeColumnType[T]) + extends BasicColumnAccessor(buffer, columnType) with NullableColumnAccessor + with CompressibleColumnAccessor[T] private[sql] class BooleanColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, BOOLEAN) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setBoolean(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, BOOLEAN) private[sql] class IntColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, INT) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setInt(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, INT) private[sql] class ShortColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, SHORT) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setShort(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, SHORT) private[sql] class LongColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, LONG) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setLong(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, LONG) private[sql] class ByteColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, BYTE) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setByte(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, BYTE) private[sql] class DoubleColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, DOUBLE) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setDouble(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, DOUBLE) private[sql] class FloatColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, FLOAT) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setFloat(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, FLOAT) private[sql] class StringColumnAccessor(buffer: ByteBuffer) - extends NativeColumnAccessor(buffer, STRING) { - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row.setString(ordinal, columnType.extract(buffer)) - } -} + extends NativeColumnAccessor(buffer, STRING) private[sql] class BinaryColumnAccessor(buffer: ByteBuffer) - extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer) - with NullableColumnAccessor { - - def columnType = BINARY - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - row(ordinal) = columnType.extract(buffer) - } -} + extends BasicColumnAccessor[BinaryType.type, Array[Byte]](buffer, BINARY) + with NullableColumnAccessor private[sql] class GenericColumnAccessor(buffer: ByteBuffer) - extends BasicColumnAccessor[DataType, Array[Byte]](buffer) - with NullableColumnAccessor { - - def columnType = GENERIC - - override protected def doExtractTo(row: MutableRow, ordinal: Int) { - val serialized = columnType.extract(buffer) - row(ordinal) = SparkSqlSerializer.deserialize[Any](serialized) - } -} + extends BasicColumnAccessor[DataType, Array[Byte]](buffer, GENERIC) + with NullableColumnAccessor private[sql] object ColumnAccessor { - def apply(b: ByteBuffer): ColumnAccessor = { - // The first 4 bytes in the buffer indicates the column type. - val buffer = b.duplicate().order(ByteOrder.nativeOrder()) + def apply(buffer: ByteBuffer): ColumnAccessor = { + // The first 4 bytes in the buffer indicate the column type. val columnTypeId = buffer.getInt() columnTypeId match { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala index 3e622adfd3d6a..048ee66bff44b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala @@ -22,7 +22,7 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar.ColumnBuilder._ -import org.apache.spark.sql.execution.SparkSqlSerializer +import org.apache.spark.sql.columnar.compression.{AllCompressionSchemes, CompressibleColumnBuilder} private[sql] trait ColumnBuilder { /** @@ -30,37 +30,44 @@ private[sql] trait ColumnBuilder { */ def initialize(initialSize: Int, columnName: String = "") + /** + * Appends `row(ordinal)` to the column builder. + */ def appendFrom(row: Row, ordinal: Int) + /** + * Column statistics information + */ + def columnStats: ColumnStats[_, _] + + /** + * Returns the final columnar byte buffer. + */ def build(): ByteBuffer } -private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends ColumnBuilder { +private[sql] class BasicColumnBuilder[T <: DataType, JvmType]( + val columnStats: ColumnStats[T, JvmType], + val columnType: ColumnType[T, JvmType]) + extends ColumnBuilder { - private var columnName: String = _ - protected var buffer: ByteBuffer = _ + protected var columnName: String = _ - def columnType: ColumnType[T, JvmType] + protected var buffer: ByteBuffer = _ override def initialize(initialSize: Int, columnName: String = "") = { val size = if (initialSize == 0) DEFAULT_INITIAL_BUFFER_SIZE else initialSize this.columnName = columnName - buffer = ByteBuffer.allocate(4 + 4 + size * columnType.defaultSize) + + // Reserves 4 bytes for column type ID + buffer = ByteBuffer.allocate(4 + size * columnType.defaultSize) buffer.order(ByteOrder.nativeOrder()).putInt(columnType.typeId) } - // Have to give a concrete implementation to make mixin possible override def appendFrom(row: Row, ordinal: Int) { - doAppendFrom(row, ordinal) - } - - // Concrete `ColumnBuilder`s can override this method to append values - protected def doAppendFrom(row: Row, ordinal: Int) - - // Helper method to append primitive values (to avoid boxing cost) - protected def appendValue(v: JvmType) { - buffer = ensureFreeSpace(buffer, columnType.actualSize(v)) - columnType.append(v, buffer) + val field = columnType.getField(row, ordinal) + buffer = ensureFreeSpace(buffer, columnType.actualSize(field)) + columnType.append(field, buffer) } override def build() = { @@ -69,83 +76,39 @@ private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends C } } -private[sql] abstract class NativeColumnBuilder[T <: NativeType]( - val columnType: NativeColumnType[T]) - extends BasicColumnBuilder[T, T#JvmType] +private[sql] abstract class ComplexColumnBuilder[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType]) + extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType) with NullableColumnBuilder -private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(BOOLEAN) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getBoolean(ordinal)) - } -} - -private[sql] class IntColumnBuilder extends NativeColumnBuilder(INT) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getInt(ordinal)) - } -} +private[sql] abstract class NativeColumnBuilder[T <: NativeType]( + override val columnStats: NativeColumnStats[T], + override val columnType: NativeColumnType[T]) + extends BasicColumnBuilder[T, T#JvmType](columnStats, columnType) + with NullableColumnBuilder + with AllCompressionSchemes + with CompressibleColumnBuilder[T] -private[sql] class ShortColumnBuilder extends NativeColumnBuilder(SHORT) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getShort(ordinal)) - } -} +private[sql] class BooleanColumnBuilder extends NativeColumnBuilder(new BooleanColumnStats, BOOLEAN) -private[sql] class LongColumnBuilder extends NativeColumnBuilder(LONG) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getLong(ordinal)) - } -} +private[sql] class IntColumnBuilder extends NativeColumnBuilder(new IntColumnStats, INT) -private[sql] class ByteColumnBuilder extends NativeColumnBuilder(BYTE) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getByte(ordinal)) - } -} +private[sql] class ShortColumnBuilder extends NativeColumnBuilder(new ShortColumnStats, SHORT) -private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(DOUBLE) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getDouble(ordinal)) - } -} +private[sql] class LongColumnBuilder extends NativeColumnBuilder(new LongColumnStats, LONG) -private[sql] class FloatColumnBuilder extends NativeColumnBuilder(FLOAT) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getFloat(ordinal)) - } -} +private[sql] class ByteColumnBuilder extends NativeColumnBuilder(new ByteColumnStats, BYTE) -private[sql] class StringColumnBuilder extends NativeColumnBuilder(STRING) { - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row.getString(ordinal)) - } -} +private[sql] class DoubleColumnBuilder extends NativeColumnBuilder(new DoubleColumnStats, DOUBLE) -private[sql] class BinaryColumnBuilder - extends BasicColumnBuilder[BinaryType.type, Array[Byte]] - with NullableColumnBuilder { +private[sql] class FloatColumnBuilder extends NativeColumnBuilder(new FloatColumnStats, FLOAT) - def columnType = BINARY +private[sql] class StringColumnBuilder extends NativeColumnBuilder(new StringColumnStats, STRING) - override def doAppendFrom(row: Row, ordinal: Int) { - appendValue(row(ordinal).asInstanceOf[Array[Byte]]) - } -} +private[sql] class BinaryColumnBuilder extends ComplexColumnBuilder(BINARY) // TODO (lian) Add support for array, struct and map -private[sql] class GenericColumnBuilder - extends BasicColumnBuilder[DataType, Array[Byte]] - with NullableColumnBuilder { - - def columnType = GENERIC - - override def doAppendFrom(row: Row, ordinal: Int) { - val serialized = SparkSqlSerializer.serialize(row(ordinal)) - buffer = ColumnBuilder.ensureFreeSpace(buffer, columnType.actualSize(serialized)) - columnType.append(serialized, buffer) - } -} +private[sql] class GenericColumnBuilder extends ComplexColumnBuilder(GENERIC) private[sql] object ColumnBuilder { val DEFAULT_INITIAL_BUFFER_SIZE = 10 * 1024 * 104 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala new file mode 100644 index 0000000000000..30c6bdc7912fc --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -0,0 +1,360 @@ +/* + * 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.columnar + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.types._ + +private[sql] sealed abstract class ColumnStats[T <: DataType, JvmType] extends Serializable { + /** + * Closed lower bound of this column. + */ + def lowerBound: JvmType + + /** + * Closed upper bound of this column. + */ + def upperBound: JvmType + + /** + * Gathers statistics information from `row(ordinal)`. + */ + def gatherStats(row: Row, ordinal: Int) + + /** + * Returns `true` if `lower <= row(ordinal) <= upper`. + */ + def contains(row: Row, ordinal: Int): Boolean + + /** + * Returns `true` if `row(ordinal) < upper` holds. + */ + def isAbove(row: Row, ordinal: Int): Boolean + + /** + * Returns `true` if `lower < row(ordinal)` holds. + */ + def isBelow(row: Row, ordinal: Int): Boolean + + /** + * Returns `true` if `row(ordinal) <= upper` holds. + */ + def isAtOrAbove(row: Row, ordinal: Int): Boolean + + /** + * Returns `true` if `lower <= row(ordinal)` holds. + */ + def isAtOrBelow(row: Row, ordinal: Int): Boolean +} + +private[sql] sealed abstract class NativeColumnStats[T <: NativeType] + extends ColumnStats[T, T#JvmType] { + + type JvmType = T#JvmType + + protected var (_lower, _upper) = initialBounds + + def initialBounds: (JvmType, JvmType) + + protected def columnType: NativeColumnType[T] + + override def lowerBound: T#JvmType = _lower + + override def upperBound: T#JvmType = _upper + + override def isAtOrAbove(row: Row, ordinal: Int) = { + contains(row, ordinal) || isAbove(row, ordinal) + } + + override def isAtOrBelow(row: Row, ordinal: Int) = { + contains(row, ordinal) || isBelow(row, ordinal) + } +} + +private[sql] class NoopColumnStats[T <: DataType, JvmType] extends ColumnStats[T, JvmType] { + override def isAtOrBelow(row: Row, ordinal: Int) = true + + override def isAtOrAbove(row: Row, ordinal: Int) = true + + override def isBelow(row: Row, ordinal: Int) = true + + override def isAbove(row: Row, ordinal: Int) = true + + override def contains(row: Row, ordinal: Int) = true + + override def gatherStats(row: Row, ordinal: Int) {} + + override def upperBound = null.asInstanceOf[JvmType] + + override def lowerBound = null.asInstanceOf[JvmType] +} + +private[sql] abstract class BasicColumnStats[T <: NativeType]( + protected val columnType: NativeColumnType[T]) + extends NativeColumnStats[T] + +private[sql] class BooleanColumnStats extends BasicColumnStats(BOOLEAN) { + override def initialBounds = (true, false) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } +} + +private[sql] class ByteColumnStats extends BasicColumnStats(BYTE) { + override def initialBounds = (Byte.MaxValue, Byte.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } +} + +private[sql] class ShortColumnStats extends BasicColumnStats(SHORT) { + override def initialBounds = (Short.MaxValue, Short.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } +} + +private[sql] class LongColumnStats extends BasicColumnStats(LONG) { + override def initialBounds = (Long.MaxValue, Long.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } +} + +private[sql] class DoubleColumnStats extends BasicColumnStats(DOUBLE) { + override def initialBounds = (Double.MaxValue, Double.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } +} + +private[sql] class FloatColumnStats extends BasicColumnStats(FLOAT) { + override def initialBounds = (Float.MaxValue, Float.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + } +} + +private[sql] object IntColumnStats { + val UNINITIALIZED = 0 + val INITIALIZED = 1 + val ASCENDING = 2 + val DESCENDING = 3 + val UNORDERED = 4 +} + +/** + * Statistical information for `Int` columns. More information is collected since `Int` is + * frequently used. Extra information include: + * + * - Ordering state (ascending/descending/unordered), may be used to decide whether binary search + * is applicable when searching elements. + * - Maximum delta between adjacent elements, may be used to guide the `IntDelta` compression + * scheme. + * + * (This two kinds of information are not used anywhere yet and might be removed later.) + */ +private[sql] class IntColumnStats extends BasicColumnStats(INT) { + import IntColumnStats._ + + private var orderedState = UNINITIALIZED + private var lastValue: Int = _ + private var _maxDelta: Int = _ + + def isAscending = orderedState != DESCENDING && orderedState != UNORDERED + def isDescending = orderedState != ASCENDING && orderedState != UNORDERED + def isOrdered = isAscending || isDescending + def maxDelta = _maxDelta + + override def initialBounds = (Int.MaxValue, Int.MinValue) + + override def isBelow(row: Row, ordinal: Int) = { + lowerBound < columnType.getField(row, ordinal) + } + + override def isAbove(row: Row, ordinal: Int) = { + columnType.getField(row, ordinal) < upperBound + } + + override def contains(row: Row, ordinal: Int) = { + val field = columnType.getField(row, ordinal) + lowerBound <= field && field <= upperBound + } + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + + if (field > upperBound) _upper = field + if (field < lowerBound) _lower = field + + orderedState = orderedState match { + case UNINITIALIZED => + lastValue = field + INITIALIZED + + case INITIALIZED => + // If all the integers in the column are the same, ordered state is set to Ascending. + // TODO (lian) Confirm whether this is the standard behaviour. + val nextState = if (field >= lastValue) ASCENDING else DESCENDING + _maxDelta = math.abs(field - lastValue) + lastValue = field + nextState + + case ASCENDING if field < lastValue => + UNORDERED + + case DESCENDING if field > lastValue => + UNORDERED + + case state @ (ASCENDING | DESCENDING) => + _maxDelta = _maxDelta.max(field - lastValue) + lastValue = field + state + + case _ => + orderedState + } + } +} + +private[sql] class StringColumnStats extends BasicColumnStats(STRING) { + override def initialBounds = (null, null) + + override def gatherStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + if ((upperBound eq null) || field.compareTo(upperBound) > 0) _upper = field + if ((lowerBound eq null) || field.compareTo(lowerBound) < 0) _lower = field + } + + override def contains(row: Row, ordinal: Int) = { + !(upperBound eq null) && { + val field = columnType.getField(row, ordinal) + lowerBound.compareTo(field) <= 0 && field.compareTo(upperBound) <= 0 + } + } + + override def isAbove(row: Row, ordinal: Int) = { + !(upperBound eq null) && { + val field = columnType.getField(row, ordinal) + field.compareTo(upperBound) < 0 + } + } + + override def isBelow(row: Row, ordinal: Int) = { + !(lowerBound eq null) && { + val field = columnType.getField(row, ordinal) + lowerBound.compareTo(field) < 0 + } + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index a452b86f0cda3..5be76890afe31 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -19,7 +19,12 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer +import scala.reflect.runtime.universe.TypeTag + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.MutableRow import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.execution.SparkSqlSerializer /** * An abstract class that represents type of a column. Used to append/extract Java objects into/from @@ -50,10 +55,24 @@ private[sql] sealed abstract class ColumnType[T <: DataType, JvmType]( */ def actualSize(v: JvmType): Int = defaultSize + /** + * Returns `row(ordinal)`. Subclasses should override this method to avoid boxing/unboxing costs + * whenever possible. + */ + def getField(row: Row, ordinal: Int): JvmType + + /** + * Sets `row(ordinal)` to `field`. Subclasses should override this method to avoid boxing/unboxing + * costs whenever possible. + */ + def setField(row: MutableRow, ordinal: Int, value: JvmType) + /** * Creates a duplicated copy of the value. */ def clone(v: JvmType): JvmType = v + + override def toString = getClass.getSimpleName.stripSuffix("$") } private[sql] abstract class NativeColumnType[T <: NativeType]( @@ -65,7 +84,7 @@ private[sql] abstract class NativeColumnType[T <: NativeType]( /** * Scala TypeTag. Can be used to create primitive arrays and hash tables. */ - def scalaTag = dataType.tag + def scalaTag: TypeTag[dataType.JvmType] = dataType.tag } private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { @@ -76,6 +95,12 @@ private[sql] object INT extends NativeColumnType(IntegerType, 0, 4) { def extract(buffer: ByteBuffer) = { buffer.getInt() } + + override def setField(row: MutableRow, ordinal: Int, value: Int) { + row.setInt(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getInt(ordinal) } private[sql] object LONG extends NativeColumnType(LongType, 1, 8) { @@ -86,6 +111,12 @@ private[sql] object LONG extends NativeColumnType(LongType, 1, 8) { override def extract(buffer: ByteBuffer) = { buffer.getLong() } + + override def setField(row: MutableRow, ordinal: Int, value: Long) { + row.setLong(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getLong(ordinal) } private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) { @@ -96,6 +127,12 @@ private[sql] object FLOAT extends NativeColumnType(FloatType, 2, 4) { override def extract(buffer: ByteBuffer) = { buffer.getFloat() } + + override def setField(row: MutableRow, ordinal: Int, value: Float) { + row.setFloat(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getFloat(ordinal) } private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) { @@ -106,6 +143,12 @@ private[sql] object DOUBLE extends NativeColumnType(DoubleType, 3, 8) { override def extract(buffer: ByteBuffer) = { buffer.getDouble() } + + override def setField(row: MutableRow, ordinal: Int, value: Double) { + row.setDouble(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getDouble(ordinal) } private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) { @@ -116,6 +159,12 @@ private[sql] object BOOLEAN extends NativeColumnType(BooleanType, 4, 1) { override def extract(buffer: ByteBuffer) = { if (buffer.get() == 1) true else false } + + override def setField(row: MutableRow, ordinal: Int, value: Boolean) { + row.setBoolean(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getBoolean(ordinal) } private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) { @@ -126,6 +175,12 @@ private[sql] object BYTE extends NativeColumnType(ByteType, 5, 1) { override def extract(buffer: ByteBuffer) = { buffer.get() } + + override def setField(row: MutableRow, ordinal: Int, value: Byte) { + row.setByte(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getByte(ordinal) } private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { @@ -136,6 +191,12 @@ private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { override def extract(buffer: ByteBuffer) = { buffer.getShort() } + + override def setField(row: MutableRow, ordinal: Int, value: Short) { + row.setShort(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getShort(ordinal) } private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { @@ -152,6 +213,12 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { buffer.get(stringBytes, 0, length) new String(stringBytes) } + + override def setField(row: MutableRow, ordinal: Int, value: String) { + row.setString(ordinal, value) + } + + override def getField(row: Row, ordinal: Int) = row.getString(ordinal) } private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( @@ -173,15 +240,27 @@ private[sql] sealed abstract class ByteArrayColumnType[T <: DataType]( } } -private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) +private[sql] object BINARY extends ByteArrayColumnType[BinaryType.type](8, 16) { + override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { + row(ordinal) = value + } + + override def getField(row: Row, ordinal: Int) = row(ordinal).asInstanceOf[Array[Byte]] +} // Used to process generic objects (all types other than those listed above). Objects should be // serialized first before appending to the column `ByteBuffer`, and is also extracted as serialized // byte array. -private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) +private[sql] object GENERIC extends ByteArrayColumnType[DataType](9, 16) { + override def setField(row: MutableRow, ordinal: Int, value: Array[Byte]) { + row(ordinal) = SparkSqlSerializer.deserialize[Any](value) + } + + override def getField(row: Row, ordinal: Int) = SparkSqlSerializer.serialize(row(ordinal)) +} private[sql] object ColumnType { - implicit def dataTypeToColumnType(dataType: DataType): ColumnType[_, _] = { + def apply(dataType: DataType): ColumnType[_, _] = { dataType match { case IntegerType => INT case LongType => LONG diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala similarity index 93% rename from sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala rename to sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala index f853759e5a306..8a24733047423 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/InMemoryColumnarTableScan.scala @@ -21,9 +21,6 @@ import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute} import org.apache.spark.sql.execution.{SparkPlan, LeafNode} import org.apache.spark.sql.Row -/* Implicit conversions */ -import org.apache.spark.sql.columnar.ColumnType._ - private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan) extends LeafNode { @@ -32,8 +29,8 @@ private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], ch lazy val cachedColumnBuffers = { val output = child.output val cached = child.execute().mapPartitions { iterator => - val columnBuilders = output.map { a => - ColumnBuilder(a.dataType.typeId, 0, a.name) + val columnBuilders = output.map { attribute => + ColumnBuilder(ColumnType(attribute.dataType).typeId, 0, attribute.name) }.toArray var row: Row = null diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala index 2970c609b928d..7d49ab07f7a53 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala @@ -29,7 +29,7 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { private var nextNullIndex: Int = _ private var pos: Int = 0 - abstract override def initialize() { + abstract override protected def initialize() { nullsBuffer = underlyingBuffer.duplicate().order(ByteOrder.nativeOrder()) nullCount = nullsBuffer.getInt() nextNullIndex = if (nullCount > 0) nullsBuffer.getInt() else -1 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala index 048d1f05c7df2..2a3b6fc1e46d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala @@ -22,10 +22,18 @@ import java.nio.{ByteBuffer, ByteOrder} import org.apache.spark.sql.Row /** - * Builds a nullable column. The byte buffer of a nullable column contains: - * - 4 bytes for the null count (number of nulls) - * - positions for each null, in ascending order - * - the non-null data (column data type, compression type, data...) + * A stackable trait used for building byte buffer for a column containing null values. Memory + * layout of the final byte buffer is: + * {{{ + * .----------------------- Column type ID (4 bytes) + * | .------------------- Null count N (4 bytes) + * | | .--------------- Null positions (4 x N bytes, empty if null count is zero) + * | | | .--------- Non-null elements + * V V V V + * +---+---+-----+---------+ + * | | | ... | ... ... | + * +---+---+-----+---------+ + * }}} */ private[sql] trait NullableColumnBuilder extends ColumnBuilder { private var nulls: ByteBuffer = _ @@ -59,19 +67,8 @@ private[sql] trait NullableColumnBuilder extends ColumnBuilder { nulls.limit(nullDataLen) nulls.rewind() - // Column type ID is moved to the front, follows the null count, then non-null data - // - // +---------+ - // | 4 bytes | Column type ID - // +---------+ - // | 4 bytes | Null count - // +---------+ - // | ... | Null positions (if null count is not zero) - // +---------+ - // | ... | Non-null part (without column type ID) - // +---------+ val buffer = ByteBuffer - .allocate(4 + nullDataLen + nonNulls.limit) + .allocate(4 + 4 + nullDataLen + nonNulls.remaining()) .order(ByteOrder.nativeOrder()) .putInt(typeId) .putInt(nullCount) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.scala new file mode 100644 index 0000000000000..878cb84de106f --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnAccessor.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.columnar.compression + +import java.nio.ByteBuffer + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar.{ColumnAccessor, NativeColumnAccessor} + +private[sql] trait CompressibleColumnAccessor[T <: NativeType] extends ColumnAccessor { + this: NativeColumnAccessor[T] => + + private var decoder: Decoder[T] = _ + + abstract override protected def initialize() = { + super.initialize() + decoder = CompressionScheme(underlyingBuffer.getInt()).decoder(buffer, columnType) + } + + abstract override def extractSingle(buffer: ByteBuffer): T#JvmType = decoder.next() +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala new file mode 100644 index 0000000000000..3ac4b358ddf83 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -0,0 +1,95 @@ +/* + * 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.columnar.compression + +import java.nio.{ByteBuffer, ByteOrder} + +import org.apache.spark.sql.{Logging, Row} +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar.{ColumnBuilder, NativeColumnBuilder} + +/** + * A stackable trait that builds optionally compressed byte buffer for a column. Memory layout of + * the final byte buffer is: + * {{{ + * .--------------------------- Column type ID (4 bytes) + * | .----------------------- Null count N (4 bytes) + * | | .------------------- Null positions (4 x N bytes, empty if null count is zero) + * | | | .------------- Compression scheme ID (4 bytes) + * | | | | .--------- Compressed non-null elements + * V V V V V + * +---+---+-----+---+---------+ + * | | | ... | | ... ... | + * +---+---+-----+---+---------+ + * \-----------/ \-----------/ + * header body + * }}} + */ +private[sql] trait CompressibleColumnBuilder[T <: NativeType] + extends ColumnBuilder with Logging { + + this: NativeColumnBuilder[T] with WithCompressionSchemes => + + import CompressionScheme._ + + val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder) + + protected def isWorthCompressing(encoder: Encoder) = { + encoder.compressionRatio < 0.8 + } + + private def gatherCompressibilityStats(row: Row, ordinal: Int) { + val field = columnType.getField(row, ordinal) + + var i = 0 + while (i < compressionEncoders.length) { + compressionEncoders(i).gatherCompressibilityStats(field, columnType) + i += 1 + } + } + + abstract override def appendFrom(row: Row, ordinal: Int) { + super.appendFrom(row, ordinal) + gatherCompressibilityStats(row, ordinal) + } + + abstract override def build() = { + val rawBuffer = super.build() + val encoder = { + val candidate = compressionEncoders.minBy(_.compressionRatio) + if (isWorthCompressing(candidate)) candidate else PassThrough.encoder + } + + val headerSize = columnHeaderSize(rawBuffer) + val compressedSize = if (encoder.compressedSize == 0) { + rawBuffer.limit - headerSize + } else { + encoder.compressedSize + } + + // Reserves 4 bytes for compression scheme ID + val compressedBuffer = ByteBuffer + .allocate(headerSize + 4 + compressedSize) + .order(ByteOrder.nativeOrder) + + copyColumnHeader(rawBuffer, compressedBuffer) + + logger.info(s"Compressor for [$columnName]: $encoder, ratio: ${encoder.compressionRatio}") + encoder.compress(rawBuffer, compressedBuffer, columnType) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala new file mode 100644 index 0000000000000..d3a4ac8df926b --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -0,0 +1,94 @@ +/* + * 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.columnar.compression + +import java.nio.ByteBuffer + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} + +private[sql] trait Encoder { + def gatherCompressibilityStats[T <: NativeType]( + value: T#JvmType, + columnType: ColumnType[T, T#JvmType]) {} + + def compressedSize: Int + + def uncompressedSize: Int + + def compressionRatio: Double = { + if (uncompressedSize > 0) compressedSize.toDouble / uncompressedSize else 1.0 + } + + def compress[T <: NativeType]( + from: ByteBuffer, + to: ByteBuffer, + columnType: ColumnType[T, T#JvmType]): ByteBuffer +} + +private[sql] trait Decoder[T <: NativeType] extends Iterator[T#JvmType] + +private[sql] trait CompressionScheme { + def typeId: Int + + def supports(columnType: ColumnType[_, _]): Boolean + + def encoder: Encoder + + def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] +} + +private[sql] trait WithCompressionSchemes { + def schemes: Seq[CompressionScheme] +} + +private[sql] trait AllCompressionSchemes extends WithCompressionSchemes { + override val schemes: Seq[CompressionScheme] = { + Seq(PassThrough, RunLengthEncoding, DictionaryEncoding) + } +} + +private[sql] object CompressionScheme { + def apply(typeId: Int): CompressionScheme = typeId match { + case PassThrough.typeId => PassThrough + case _ => throw new UnsupportedOperationException() + } + + def copyColumnHeader(from: ByteBuffer, to: ByteBuffer) { + // Writes column type ID + to.putInt(from.getInt()) + + // Writes null count + val nullCount = from.getInt() + to.putInt(nullCount) + + // Writes null positions + var i = 0 + while (i < nullCount) { + to.putInt(from.getInt()) + i += 1 + } + } + + def columnHeaderSize(columnBuffer: ByteBuffer): Int = { + val header = columnBuffer.duplicate() + val nullCount = header.getInt(4) + // Column type ID + null count + null positions + 4 + 4 + 4 * nullCount + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala new file mode 100644 index 0000000000000..dc2c153faf8ad --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -0,0 +1,288 @@ +/* + * 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.columnar.compression + +import java.nio.ByteBuffer + +import scala.collection.mutable +import scala.reflect.ClassTag +import scala.reflect.runtime.universe.runtimeMirror + +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar._ + +private[sql] case object PassThrough extends CompressionScheme { + override val typeId = 0 + + override def supports(columnType: ColumnType[_, _]) = true + + override def encoder = new this.Encoder + + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder(buffer, columnType) + } + + class Encoder extends compression.Encoder { + override def uncompressedSize = 0 + + override def compressedSize = 0 + + override def compress[T <: NativeType]( + from: ByteBuffer, + to: ByteBuffer, + columnType: ColumnType[T, T#JvmType]) = { + + // Writes compression type ID and copies raw contents + to.putInt(PassThrough.typeId).put(from).rewind() + to + } + } + + class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + extends compression.Decoder[T] { + + override def next() = columnType.extract(buffer) + + override def hasNext = buffer.hasRemaining + } +} + +private[sql] case object RunLengthEncoding extends CompressionScheme { + override def typeId = 1 + + override def encoder = new this.Encoder + + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder(buffer, columnType) + } + + override def supports(columnType: ColumnType[_, _]) = columnType match { + case INT | LONG | SHORT | BYTE | STRING | BOOLEAN => true + case _ => false + } + + class Encoder extends compression.Encoder { + private var _uncompressedSize = 0 + private var _compressedSize = 0 + + // Using `MutableRow` to store the last value to avoid boxing/unboxing cost. + private val lastValue = new GenericMutableRow(1) + private var lastRun = 0 + + override def uncompressedSize = _uncompressedSize + + override def compressedSize = _compressedSize + + override def gatherCompressibilityStats[T <: NativeType]( + value: T#JvmType, + columnType: ColumnType[T, T#JvmType]) { + + val actualSize = columnType.actualSize(value) + _uncompressedSize += actualSize + + if (lastValue.isNullAt(0)) { + columnType.setField(lastValue, 0, value) + lastRun = 1 + _compressedSize += actualSize + 4 + } else { + if (columnType.getField(lastValue, 0) == value) { + lastRun += 1 + } else { + _compressedSize += actualSize + 4 + columnType.setField(lastValue, 0, value) + lastRun = 1 + } + } + } + + override def compress[T <: NativeType]( + from: ByteBuffer, + to: ByteBuffer, + columnType: ColumnType[T, T#JvmType]) = { + + to.putInt(RunLengthEncoding.typeId) + + if (from.hasRemaining) { + var currentValue = columnType.extract(from) + var currentRun = 1 + + while (from.hasRemaining) { + val value = columnType.extract(from) + + if (value == currentValue) { + currentRun += 1 + } else { + // Writes current run + columnType.append(currentValue, to) + to.putInt(currentRun) + + // Resets current run + currentValue = value + currentRun = 1 + } + } + + // Writes the last run + columnType.append(currentValue, to) + to.putInt(currentRun) + } + + to.rewind() + to + } + } + + class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + extends compression.Decoder[T] { + + private var run = 0 + private var valueCount = 0 + private var currentValue: T#JvmType = _ + + override def next() = { + if (valueCount == run) { + currentValue = columnType.extract(buffer) + run = buffer.getInt() + valueCount = 1 + } else { + valueCount += 1 + } + + currentValue + } + + override def hasNext = buffer.hasRemaining + } +} + +private[sql] case object DictionaryEncoding extends CompressionScheme { + override def typeId: Int = 2 + + // 32K unique values allowed + private val MAX_DICT_SIZE = Short.MaxValue - 1 + + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder[T](buffer, columnType) + } + + override def encoder = new this.Encoder + + override def supports(columnType: ColumnType[_, _]) = columnType match { + case INT | LONG | STRING => true + case _ => false + } + + class Encoder extends compression.Encoder{ + // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary + // overflows. + private var _uncompressedSize = 0 + + // If the number of distinct elements is too large, we discard the use of dictionary encoding + // and set the overflow flag to true. + private var overflow = false + + // Total number of elements. + private var count = 0 + + // The reverse mapping of _dictionary, i.e. mapping encoded integer to the value itself. + private var values = new mutable.ArrayBuffer[Any](1024) + + // The dictionary that maps a value to the encoded short integer. + private val dictionary = mutable.HashMap.empty[Any, Short] + + // Size of the serialized dictionary in bytes. Initialized to 4 since we need at least an `Int` + // to store dictionary element count. + private var dictionarySize = 4 + + override def gatherCompressibilityStats[T <: NativeType]( + value: T#JvmType, + columnType: ColumnType[T, T#JvmType]) { + + if (!overflow) { + val actualSize = columnType.actualSize(value) + count += 1 + _uncompressedSize += actualSize + + if (!dictionary.contains(value)) { + if (dictionary.size < MAX_DICT_SIZE) { + val clone = columnType.clone(value) + values += clone + dictionarySize += actualSize + dictionary(clone) = dictionary.size.toShort + } else { + overflow = true + values.clear() + dictionary.clear() + } + } + } + } + + override def compress[T <: NativeType]( + from: ByteBuffer, + to: ByteBuffer, + columnType: ColumnType[T, T#JvmType]) = { + + if (overflow) { + throw new IllegalStateException( + "Dictionary encoding should not be used because of dictionary overflow.") + } + + to.putInt(DictionaryEncoding.typeId) + .putInt(dictionary.size) + + var i = 0 + while (i < values.length) { + columnType.append(values(i).asInstanceOf[T#JvmType], to) + i += 1 + } + + while (from.hasRemaining) { + to.putShort(dictionary(columnType.extract(from))) + } + + to.rewind() + to + } + + override def uncompressedSize = _uncompressedSize + + override def compressedSize = if (overflow) Int.MaxValue else dictionarySize + count * 2 + } + + class Decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) + extends compression.Decoder[T] { + + private val dictionary = { + // TODO Can we clean up this mess? Maybe move this to `DataType`? + implicit val classTag = { + val mirror = runtimeMirror(getClass.getClassLoader) + ClassTag[T#JvmType](mirror.runtimeClass(columnType.scalaTag.tpe)) + } + + Array.fill(buffer.getInt()) { + columnType.extract(buffer) + } + } + + override def next() = dictionary(buffer.getShort()) + + override def hasNext = buffer.hasRemaining + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala new file mode 100644 index 0000000000000..78640b876d4aa --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnStatsSuite.scala @@ -0,0 +1,61 @@ +/* + * 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.columnar + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.types._ + +class ColumnStatsSuite extends FunSuite { + testColumnStats(classOf[BooleanColumnStats], BOOLEAN) + testColumnStats(classOf[ByteColumnStats], BYTE) + testColumnStats(classOf[ShortColumnStats], SHORT) + testColumnStats(classOf[IntColumnStats], INT) + testColumnStats(classOf[LongColumnStats], LONG) + testColumnStats(classOf[FloatColumnStats], FLOAT) + testColumnStats(classOf[DoubleColumnStats], DOUBLE) + testColumnStats(classOf[StringColumnStats], STRING) + + def testColumnStats[T <: NativeType, U <: NativeColumnStats[T]]( + columnStatsClass: Class[U], + columnType: NativeColumnType[T]) { + + val columnStatsName = columnStatsClass.getSimpleName + + test(s"$columnStatsName: empty") { + val columnStats = columnStatsClass.newInstance() + expectResult(columnStats.initialBounds, "Wrong initial bounds") { + (columnStats.lowerBound, columnStats.upperBound) + } + } + + test(s"$columnStatsName: non-empty") { + import ColumnarTestUtils._ + + val columnStats = columnStatsClass.newInstance() + val rows = Seq.fill(10)(makeRandomRow(columnType)) + rows.foreach(columnStats.gatherStats(_, 0)) + + val values = rows.map(_.head.asInstanceOf[T#JvmType]) + val ordering = columnType.dataType.ordering.asInstanceOf[Ordering[T#JvmType]] + + expectResult(values.min(ordering), "Wrong lower bound")(columnStats.lowerBound) + expectResult(values.max(ordering), "Wrong upper bound")(columnStats.upperBound) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 2d431affbcfcc..1d3608ed2d9ab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -19,46 +19,56 @@ package org.apache.spark.sql.columnar import java.nio.ByteBuffer -import scala.util.Random - import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.columnar.ColumnarTestUtils._ import org.apache.spark.sql.execution.SparkSqlSerializer class ColumnTypeSuite extends FunSuite { - val columnTypes = Seq(INT, SHORT, LONG, BYTE, DOUBLE, FLOAT, STRING, BINARY, GENERIC) + val DEFAULT_BUFFER_SIZE = 512 test("defaultSize") { - val defaultSize = Seq(4, 2, 8, 1, 8, 4, 8, 16, 16) + val checks = Map( + INT -> 4, SHORT -> 2, LONG -> 8, BYTE -> 1, DOUBLE -> 8, FLOAT -> 4, + BOOLEAN -> 1, STRING -> 8, BINARY -> 16, GENERIC -> 16) - columnTypes.zip(defaultSize).foreach { case (columnType, size) => - assert(columnType.defaultSize === size) + checks.foreach { case (columnType, expectedSize) => + expectResult(expectedSize, s"Wrong defaultSize for $columnType") { + columnType.defaultSize + } } } test("actualSize") { - val expectedSizes = Seq(4, 2, 8, 1, 8, 4, 4 + 5, 4 + 4, 4 + 11) - val actualSizes = Seq( - INT.actualSize(Int.MaxValue), - SHORT.actualSize(Short.MaxValue), - LONG.actualSize(Long.MaxValue), - BYTE.actualSize(Byte.MaxValue), - DOUBLE.actualSize(Double.MaxValue), - FLOAT.actualSize(Float.MaxValue), - STRING.actualSize("hello"), - BINARY.actualSize(new Array[Byte](4)), - GENERIC.actualSize(SparkSqlSerializer.serialize(Map(1 -> "a")))) - - expectedSizes.zip(actualSizes).foreach { case (expected, actual) => - assert(expected === actual) + def checkActualSize[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType], + value: JvmType, + expected: Int) { + + expectResult(expected, s"Wrong actualSize for $columnType") { + columnType.actualSize(value) + } } + + checkActualSize(INT, Int.MaxValue, 4) + checkActualSize(SHORT, Short.MaxValue, 2) + checkActualSize(LONG, Long.MaxValue, 8) + checkActualSize(BYTE, Byte.MaxValue, 1) + checkActualSize(DOUBLE, Double.MaxValue, 8) + checkActualSize(FLOAT, Float.MaxValue, 4) + checkActualSize(BOOLEAN, true, 1) + checkActualSize(STRING, "hello", 4 + 5) + + val binary = Array.fill[Byte](4)(0: Byte) + checkActualSize(BINARY, binary, 4 + 4) + + val generic = Map(1 -> "a") + checkActualSize(GENERIC, SparkSqlSerializer.serialize(generic), 4 + 11) } - testNumericColumnType[BooleanType.type, Boolean]( + testNativeColumnType[BooleanType.type]( BOOLEAN, - Array.fill(4)(Random.nextBoolean()), - ByteBuffer.allocate(32), (buffer: ByteBuffer, v: Boolean) => { buffer.put((if (v) 1 else 0).toByte) }, @@ -66,105 +76,42 @@ class ColumnTypeSuite extends FunSuite { buffer.get() == 1 }) - testNumericColumnType[IntegerType.type, Int]( - INT, - Array.fill(4)(Random.nextInt()), - ByteBuffer.allocate(32), - (_: ByteBuffer).putInt(_), - (_: ByteBuffer).getInt) - - testNumericColumnType[ShortType.type, Short]( - SHORT, - Array.fill(4)(Random.nextInt(Short.MaxValue).asInstanceOf[Short]), - ByteBuffer.allocate(32), - (_: ByteBuffer).putShort(_), - (_: ByteBuffer).getShort) - - testNumericColumnType[LongType.type, Long]( - LONG, - Array.fill(4)(Random.nextLong()), - ByteBuffer.allocate(64), - (_: ByteBuffer).putLong(_), - (_: ByteBuffer).getLong) - - testNumericColumnType[ByteType.type, Byte]( - BYTE, - Array.fill(4)(Random.nextInt(Byte.MaxValue).asInstanceOf[Byte]), - ByteBuffer.allocate(64), - (_: ByteBuffer).put(_), - (_: ByteBuffer).get) - - testNumericColumnType[DoubleType.type, Double]( - DOUBLE, - Array.fill(4)(Random.nextDouble()), - ByteBuffer.allocate(64), - (_: ByteBuffer).putDouble(_), - (_: ByteBuffer).getDouble) - - testNumericColumnType[FloatType.type, Float]( - FLOAT, - Array.fill(4)(Random.nextFloat()), - ByteBuffer.allocate(64), - (_: ByteBuffer).putFloat(_), - (_: ByteBuffer).getFloat) - - test("STRING") { - val buffer = ByteBuffer.allocate(128) - val seq = Array("hello", "world", "spark", "sql") - - seq.map(_.getBytes).foreach { bytes: Array[Byte] => - buffer.putInt(bytes.length).put(bytes) - } + testNativeColumnType[IntegerType.type](INT, _.putInt(_), _.getInt) - buffer.rewind() - seq.foreach { s => - assert(s === STRING.extract(buffer)) - } + testNativeColumnType[ShortType.type](SHORT, _.putShort(_), _.getShort) - buffer.rewind() - seq.foreach(STRING.append(_, buffer)) + testNativeColumnType[LongType.type](LONG, _.putLong(_), _.getLong) - buffer.rewind() - seq.foreach { s => - val length = buffer.getInt - assert(length === s.getBytes.length) + testNativeColumnType[ByteType.type](BYTE, _.put(_), _.get) + + testNativeColumnType[DoubleType.type](DOUBLE, _.putDouble(_), _.getDouble) + + testNativeColumnType[FloatType.type](FLOAT, _.putFloat(_), _.getFloat) + testNativeColumnType[StringType.type]( + STRING, + (buffer: ByteBuffer, string: String) => { + val bytes = string.getBytes() + buffer.putInt(bytes.length).put(string.getBytes) + }, + (buffer: ByteBuffer) => { + val length = buffer.getInt() val bytes = new Array[Byte](length) buffer.get(bytes, 0, length) - assert(s === new String(bytes)) - } - } - - test("BINARY") { - val buffer = ByteBuffer.allocate(128) - val seq = Array.fill(4) { - val bytes = new Array[Byte](4) - Random.nextBytes(bytes) - bytes - } + new String(bytes) + }) - seq.foreach { bytes => + testColumnType[BinaryType.type, Array[Byte]]( + BINARY, + (buffer: ByteBuffer, bytes: Array[Byte]) => { buffer.putInt(bytes.length).put(bytes) - } - - buffer.rewind() - seq.foreach { b => - assert(b === BINARY.extract(buffer)) - } - - buffer.rewind() - seq.foreach(BINARY.append(_, buffer)) - - buffer.rewind() - seq.foreach { b => - val length = buffer.getInt - assert(length === b.length) - + }, + (buffer: ByteBuffer) => { + val length = buffer.getInt() val bytes = new Array[Byte](length) buffer.get(bytes, 0, length) - assert(b === bytes) - } - } + bytes + }) test("GENERIC") { val buffer = ByteBuffer.allocate(512) @@ -177,43 +124,58 @@ class ColumnTypeSuite extends FunSuite { val length = buffer.getInt() assert(length === serializedObj.length) - val bytes = new Array[Byte](length) - buffer.get(bytes, 0, length) - assert(obj === SparkSqlSerializer.deserialize(bytes)) + expectResult(obj, "Deserialized object didn't equal to the original object") { + val bytes = new Array[Byte](length) + buffer.get(bytes, 0, length) + SparkSqlSerializer.deserialize(bytes) + } buffer.rewind() buffer.putInt(serializedObj.length).put(serializedObj) - buffer.rewind() - assert(obj === SparkSqlSerializer.deserialize(GENERIC.extract(buffer))) + expectResult(obj, "Deserialized object didn't equal to the original object") { + buffer.rewind() + SparkSqlSerializer.deserialize(GENERIC.extract(buffer)) + } + } + + def testNativeColumnType[T <: NativeType]( + columnType: NativeColumnType[T], + putter: (ByteBuffer, T#JvmType) => Unit, + getter: (ByteBuffer) => T#JvmType) { + + testColumnType[T, T#JvmType](columnType, putter, getter) } - def testNumericColumnType[T <: DataType, JvmType]( + def testColumnType[T <: DataType, JvmType]( columnType: ColumnType[T, JvmType], - seq: Seq[JvmType], - buffer: ByteBuffer, putter: (ByteBuffer, JvmType) => Unit, getter: (ByteBuffer) => JvmType) { - val columnTypeName = columnType.getClass.getSimpleName.stripSuffix("$") + val buffer = ByteBuffer.allocate(DEFAULT_BUFFER_SIZE) + val seq = (0 until 4).map(_ => makeRandomValue(columnType)) - test(s"$columnTypeName.extract") { + test(s"$columnType.extract") { buffer.rewind() seq.foreach(putter(buffer, _)) buffer.rewind() - seq.foreach { i => - assert(i === columnType.extract(buffer)) + seq.foreach { expected => + assert( + expected === columnType.extract(buffer), + "Extracted value didn't equal to the original one") } } - test(s"$columnTypeName.append") { + test(s"$columnType.append") { buffer.rewind() seq.foreach(columnType.append(_, buffer)) buffer.rewind() - seq.foreach { i => - assert(i === getter(buffer)) + seq.foreach { expected => + assert( + expected === getter(buffer), + "Extracted value didn't equal to the original one") } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala index 928851a385d41..70b2e851737f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala @@ -17,11 +17,11 @@ package org.apache.spark.sql.columnar +import org.apache.spark.sql.{QueryTest, TestData} import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.{TestData, DslQuerySuite} -class ColumnarQuerySuite extends DslQuerySuite { +class ColumnarQuerySuite extends QueryTest { import TestData._ import TestSQLContext._ diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala deleted file mode 100644 index ddcdede8d1a4a..0000000000000 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestData.scala +++ /dev/null @@ -1,55 +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.columnar - -import scala.util.Random - -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow - -// TODO Enrich test data -object ColumnarTestData { - object GenericMutableRow { - def apply(values: Any*) = { - val row = new GenericMutableRow(values.length) - row.indices.foreach { i => - row(i) = values(i) - } - row - } - } - - def randomBytes(length: Int) = { - val bytes = new Array[Byte](length) - Random.nextBytes(bytes) - bytes - } - - val nonNullRandomRow = GenericMutableRow( - Random.nextInt(), - Random.nextLong(), - Random.nextFloat(), - Random.nextDouble(), - Random.nextBoolean(), - Random.nextInt(Byte.MaxValue).asInstanceOf[Byte], - Random.nextInt(Short.MaxValue).asInstanceOf[Short], - Random.nextString(Random.nextInt(64)), - randomBytes(Random.nextInt(64)), - Map(Random.nextInt() -> Random.nextString(4))) - - val nullRow = GenericMutableRow(Seq.fill(10)(null): _*) -} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala new file mode 100644 index 0000000000000..04bdc43d95328 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarTestUtils.scala @@ -0,0 +1,100 @@ +/* + * 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.columnar + +import scala.collection.immutable.HashSet +import scala.util.Random + +import org.apache.spark.sql.Row +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.types.{DataType, NativeType} + +object ColumnarTestUtils { + def makeNullRow(length: Int) = { + val row = new GenericMutableRow(length) + (0 until length).foreach(row.setNullAt) + row + } + + def makeRandomValue[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]): JvmType = { + def randomBytes(length: Int) = { + val bytes = new Array[Byte](length) + Random.nextBytes(bytes) + bytes + } + + (columnType match { + case BYTE => (Random.nextInt(Byte.MaxValue * 2) - Byte.MaxValue).toByte + case SHORT => (Random.nextInt(Short.MaxValue * 2) - Short.MaxValue).toShort + case INT => Random.nextInt() + case LONG => Random.nextLong() + case FLOAT => Random.nextFloat() + case DOUBLE => Random.nextDouble() + case STRING => Random.nextString(Random.nextInt(32)) + case BOOLEAN => Random.nextBoolean() + case BINARY => randomBytes(Random.nextInt(32)) + case _ => + // Using a random one-element map instead of an arbitrary object + Map(Random.nextInt() -> Random.nextString(Random.nextInt(32))) + }).asInstanceOf[JvmType] + } + + def makeRandomValues( + head: ColumnType[_ <: DataType, _], + tail: ColumnType[_ <: DataType, _]*): Seq[Any] = makeRandomValues(Seq(head) ++ tail) + + def makeRandomValues(columnTypes: Seq[ColumnType[_ <: DataType, _]]): Seq[Any] = { + columnTypes.map(makeRandomValue(_)) + } + + def makeUniqueRandomValues[T <: DataType, JvmType]( + columnType: ColumnType[T, JvmType], + count: Int): Seq[JvmType] = { + + Iterator.iterate(HashSet.empty[JvmType]) { set => + set + Iterator.continually(makeRandomValue(columnType)).filterNot(set.contains).next() + }.drop(count).next().toSeq + } + + def makeRandomRow( + head: ColumnType[_ <: DataType, _], + tail: ColumnType[_ <: DataType, _]*): Row = makeRandomRow(Seq(head) ++ tail) + + def makeRandomRow(columnTypes: Seq[ColumnType[_ <: DataType, _]]): Row = { + val row = new GenericMutableRow(columnTypes.length) + makeRandomValues(columnTypes).zipWithIndex.foreach { case (value, index) => + row(index) = value + } + row + } + + def makeUniqueValuesAndSingleValueRows[T <: NativeType]( + columnType: NativeColumnType[T], + count: Int) = { + + val values = makeUniqueRandomValues(columnType, count) + val rows = values.map { value => + val row = new GenericMutableRow(1) + row(0) = value + row + } + + (values, rows) + } + +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index d413d483f4e7e..4a21eb6201a69 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -17,12 +17,29 @@ package org.apache.spark.sql.columnar +import java.nio.ByteBuffer + import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.types.DataType + import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.types.DataType + +class TestNullableColumnAccessor[T <: DataType, JvmType]( + buffer: ByteBuffer, + columnType: ColumnType[T, JvmType]) + extends BasicColumnAccessor(buffer, columnType) + with NullableColumnAccessor + +object TestNullableColumnAccessor { + def apply[T <: DataType, JvmType](buffer: ByteBuffer, columnType: ColumnType[T, JvmType]) = { + // Skips the column type ID + buffer.getInt() + new TestNullableColumnAccessor(buffer, columnType) + } +} class NullableColumnAccessorSuite extends FunSuite { - import ColumnarTestData._ + import ColumnarTestUtils._ Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { testNullableColumnAccessor(_) @@ -30,30 +47,32 @@ class NullableColumnAccessorSuite extends FunSuite { def testNullableColumnAccessor[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + val nullRow = makeNullRow(1) - test(s"$typeName accessor: empty column") { - val builder = ColumnBuilder(columnType.typeId, 4) - val accessor = ColumnAccessor(builder.build()) + test(s"Nullable $typeName column accessor: empty column") { + val builder = TestNullableColumnBuilder(columnType) + val accessor = TestNullableColumnAccessor(builder.build(), columnType) assert(!accessor.hasNext) } - test(s"$typeName accessor: access null values") { - val builder = ColumnBuilder(columnType.typeId, 4) + test(s"Nullable $typeName column accessor: access null values") { + val builder = TestNullableColumnBuilder(columnType) + val randomRow = makeRandomRow(columnType) (0 until 4).foreach { _ => - builder.appendFrom(nonNullRandomRow, columnType.typeId) - builder.appendFrom(nullRow, columnType.typeId) + builder.appendFrom(randomRow, 0) + builder.appendFrom(nullRow, 0) } - val accessor = ColumnAccessor(builder.build()) + val accessor = TestNullableColumnAccessor(builder.build(), columnType) val row = new GenericMutableRow(1) (0 until 4).foreach { _ => accessor.extractTo(row, 0) - assert(row(0) === nonNullRandomRow(columnType.typeId)) + assert(row(0) === randomRow(0)) accessor.extractTo(row, 0) - assert(row(0) === null) + assert(row.isNullAt(0)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala index 5222a47e1ab87..d9d1e1bfddb75 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala @@ -19,63 +19,71 @@ package org.apache.spark.sql.columnar import org.scalatest.FunSuite -import org.apache.spark.sql.catalyst.types.DataType +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.execution.SparkSqlSerializer +class TestNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) + extends BasicColumnBuilder[T, JvmType](new NoopColumnStats[T, JvmType], columnType) + with NullableColumnBuilder + +object TestNullableColumnBuilder { + def apply[T <: DataType, JvmType](columnType: ColumnType[T, JvmType], initialSize: Int = 0) = { + val builder = new TestNullableColumnBuilder(columnType) + builder.initialize(initialSize) + builder + } +} + class NullableColumnBuilderSuite extends FunSuite { - import ColumnarTestData._ + import ColumnarTestUtils._ Seq(INT, LONG, SHORT, BOOLEAN, BYTE, STRING, DOUBLE, FLOAT, BINARY, GENERIC).foreach { testNullableColumnBuilder(_) } def testNullableColumnBuilder[T <: DataType, JvmType](columnType: ColumnType[T, JvmType]) { - val columnBuilder = ColumnBuilder(columnType.typeId) val typeName = columnType.getClass.getSimpleName.stripSuffix("$") test(s"$typeName column builder: empty column") { - columnBuilder.initialize(4) - + val columnBuilder = TestNullableColumnBuilder(columnType) val buffer = columnBuilder.build() - // For column type ID - assert(buffer.getInt() === columnType.typeId) - // For null count - assert(buffer.getInt === 0) + expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + expectResult(0, "Wrong null count")(buffer.getInt()) assert(!buffer.hasRemaining) } test(s"$typeName column builder: buffer size auto growth") { - columnBuilder.initialize(4) + val columnBuilder = TestNullableColumnBuilder(columnType) + val randomRow = makeRandomRow(columnType) - (0 until 4) foreach { _ => - columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId) + (0 until 4).foreach { _ => + columnBuilder.appendFrom(randomRow, 0) } val buffer = columnBuilder.build() - // For column type ID - assert(buffer.getInt() === columnType.typeId) - // For null count - assert(buffer.getInt() === 0) + expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + expectResult(0, "Wrong null count")(buffer.getInt()) } test(s"$typeName column builder: null values") { - columnBuilder.initialize(4) + val columnBuilder = TestNullableColumnBuilder(columnType) + val randomRow = makeRandomRow(columnType) + val nullRow = makeNullRow(1) - (0 until 4) foreach { _ => - columnBuilder.appendFrom(nonNullRandomRow, columnType.typeId) - columnBuilder.appendFrom(nullRow, columnType.typeId) + (0 until 4).foreach { _ => + columnBuilder.appendFrom(randomRow, 0) + columnBuilder.appendFrom(nullRow, 0) } val buffer = columnBuilder.build() - // For column type ID - assert(buffer.getInt() === columnType.typeId) - // For null count - assert(buffer.getInt() === 4) + expectResult(columnType.typeId, "Wrong column type ID")(buffer.getInt()) + expectResult(4, "Wrong null count")(buffer.getInt()) + // For null positions - (1 to 7 by 2).foreach(i => assert(buffer.getInt() === i)) + (1 to 7 by 2).foreach(expectResult(_, "Wrong null position")(buffer.getInt())) // For non-null values (0 until 4).foreach { _ => @@ -84,7 +92,8 @@ class NullableColumnBuilderSuite extends FunSuite { } else { columnType.extract(buffer) } - assert(actual === nonNullRandomRow(columnType.typeId)) + + assert(actual === randomRow(0), "Extracted value didn't equal to the original one") } assert(!buffer.hasRemaining) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala new file mode 100644 index 0000000000000..184691ab5b46a --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -0,0 +1,113 @@ +/* + * 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.columnar.compression + +import java.nio.ByteBuffer + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.columnar.ColumnarTestUtils._ +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow + +class DictionaryEncodingSuite extends FunSuite { + testDictionaryEncoding(new IntColumnStats, INT) + testDictionaryEncoding(new LongColumnStats, LONG) + testDictionaryEncoding(new StringColumnStats, STRING) + + def testDictionaryEncoding[T <: NativeType]( + columnStats: NativeColumnStats[T], + columnType: NativeColumnType[T]) { + + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + + def buildDictionary(buffer: ByteBuffer) = { + (0 until buffer.getInt()).map(columnType.extract(buffer) -> _.toShort).toMap + } + + test(s"$DictionaryEncoding with $typeName: simple case") { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, DictionaryEncoding) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) + + builder.initialize(0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(1), 0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(1), 0) + + val buffer = builder.build() + val headerSize = CompressionScheme.columnHeaderSize(buffer) + // 4 extra bytes for dictionary size + val dictionarySize = 4 + values.map(columnType.actualSize).sum + // 4 `Short`s, 2 bytes each + val compressedSize = dictionarySize + 2 * 4 + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + // Skips column header + buffer.position(headerSize) + expectResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + val dictionary = buildDictionary(buffer) + Array[Short](0, 1).foreach { i => + expectResult(i, "Wrong dictionary entry")(dictionary(values(i))) + } + + Array[Short](0, 1, 0, 1).foreach { + expectResult(_, "Wrong column element value")(buffer.getShort()) + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = new DictionaryEncoding.Decoder[T](buffer, columnType) + + Array[Short](0, 1, 0, 1).foreach { i => + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } + + assert(!decoder.hasNext) + } + } + + test(s"$DictionaryEncoding: overflow") { + val builder = TestCompressibleColumnBuilder(new IntColumnStats, INT, DictionaryEncoding) + builder.initialize(0) + + (0 to Short.MaxValue).foreach { n => + val row = new GenericMutableRow(1) + row.setInt(0, n) + builder.appendFrom(row, 0) + } + + withClue("Dictionary overflowed, encoding should fail") { + intercept[Throwable] { + builder.build() + } + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala new file mode 100644 index 0000000000000..2089ad120d4f2 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -0,0 +1,130 @@ +/* + * 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.columnar.compression + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.columnar.ColumnarTestUtils._ + +class RunLengthEncodingSuite extends FunSuite { + testRunLengthEncoding(new BooleanColumnStats, BOOLEAN) + testRunLengthEncoding(new ByteColumnStats, BYTE) + testRunLengthEncoding(new ShortColumnStats, SHORT) + testRunLengthEncoding(new IntColumnStats, INT) + testRunLengthEncoding(new LongColumnStats, LONG) + testRunLengthEncoding(new StringColumnStats, STRING) + + def testRunLengthEncoding[T <: NativeType]( + columnStats: NativeColumnStats[T], + columnType: NativeColumnType[T]) { + + val typeName = columnType.getClass.getSimpleName.stripSuffix("$") + + test(s"$RunLengthEncoding with $typeName: simple case") { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) + + builder.initialize(0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(1), 0) + builder.appendFrom(rows(1), 0) + + val buffer = builder.build() + val headerSize = CompressionScheme.columnHeaderSize(buffer) + // 4 extra bytes each run for run length + val compressedSize = values.map(columnType.actualSize(_) + 4).sum + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + // Skips column header + buffer.position(headerSize) + expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + Array(0, 1).foreach { i => + expectResult(values(i), "Wrong column element value")(columnType.extract(buffer)) + expectResult(2, "Wrong run length")(buffer.getInt()) + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) + + Array(0, 0, 1, 1).foreach { i => + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } + + assert(!decoder.hasNext) + } + + test(s"$RunLengthEncoding with $typeName: run length == 1") { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) + + builder.initialize(0) + builder.appendFrom(rows(0), 0) + builder.appendFrom(rows(1), 0) + + val buffer = builder.build() + val headerSize = CompressionScheme.columnHeaderSize(buffer) + // 4 bytes each run for run length + val compressedSize = values.map(columnType.actualSize(_) + 4).sum + // 4 bytes for compression scheme type ID + expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + // Skips column header + buffer.position(headerSize) + expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + Array(0, 1).foreach { i => + expectResult(values(i), "Wrong column element value")(columnType.extract(buffer)) + expectResult(1, "Wrong run length")(buffer.getInt()) + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) + + Array(0, 1).foreach { i => + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } + + assert(!decoder.hasNext) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala new file mode 100644 index 0000000000000..e0ec812863dcf --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -0,0 +1,43 @@ +/* + * 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.columnar.compression + +import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.columnar._ + +class TestCompressibleColumnBuilder[T <: NativeType]( + override val columnStats: NativeColumnStats[T], + override val columnType: NativeColumnType[T], + override val schemes: Seq[CompressionScheme]) + extends NativeColumnBuilder(columnStats, columnType) + with NullableColumnBuilder + with CompressibleColumnBuilder[T] { + + override protected def isWorthCompressing(encoder: Encoder) = true +} + +object TestCompressibleColumnBuilder { + def apply[T <: NativeType]( + columnStats: NativeColumnStats[T], + columnType: NativeColumnType[T], + scheme: CompressionScheme) = { + + new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) + } +} + From ed730c95026d322f4b24d3d9fe92050ffa74cf4a Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Wed, 2 Apr 2014 12:48:04 -0700 Subject: [PATCH 180/397] StopAfter / TopK related changes 1. Renamed StopAfter to Limit to be more consistent with naming in other relational databases. 2. Renamed TopK to TakeOrdered to be more consistent with Spark RDD API. 3. Avoid breaking lineage in Limit. 4. Added a bunch of override's to execution/basicOperators.scala. @marmbrus @liancheng Author: Reynold Xin Author: Michael Armbrust Closes #233 from rxin/limit and squashes the following commits: 13eb12a [Reynold Xin] Merge pull request #1 from marmbrus/limit 92b9727 [Michael Armbrust] More hacks to make Maps serialize with Kryo. 4fc8b4e [Reynold Xin] Merge branch 'master' of github.com:apache/spark into limit 87b7d37 [Reynold Xin] Use the proper serializer in limit. 9b79246 [Reynold Xin] Updated doc for Limit. 47d3327 [Reynold Xin] Copy tuples in Limit before shuffle. 231af3a [Reynold Xin] Limit/TakeOrdered: 1. Renamed StopAfter to Limit to be more consistent with naming in other relational databases. 2. Renamed TopK to TakeOrdered to be more consistent with Spark RDD API. 3. Avoid breaking lineage in Limit. 4. Added a bunch of override's to execution/basicOperators.scala. --- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../plans/logical/basicOperators.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 2 +- .../sql/execution/SparkSqlSerializer.scala | 6 ++ .../spark/sql/execution/SparkStrategies.scala | 10 +-- .../spark/sql/execution/basicOperators.scala | 71 ++++++++++++------- .../apache/spark/sql/hive/HiveContext.scala | 2 +- .../org/apache/spark/sql/hive/HiveQl.scala | 4 +- 8 files changed, 64 insertions(+), 35 deletions(-) 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 index 0c851c2ee2183..8de87594c8ab9 100644 --- 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 @@ -181,7 +181,7 @@ class SqlParser extends StandardTokenParsers { 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) + val withLimit = l.map { l => Limit(l, withOrder) }.getOrElse(withOrder) withLimit } 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 9d16189deedfe..b39c2b32cc42c 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 @@ -130,7 +130,7 @@ case class Aggregate( def references = child.references } -case class StopAfter(limit: Expression, child: LogicalPlan) extends UnaryNode { +case class Limit(limit: Expression, child: LogicalPlan) extends UnaryNode { def output = child.output def references = limit.references } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 69bbbdc8943fa..f4bf00f4cffa6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -145,7 +145,7 @@ class SQLContext(@transient val sparkContext: SparkContext) val sparkContext = self.sparkContext val strategies: Seq[Strategy] = - TopK :: + TakeOrdered :: PartialAggregation :: HashJoin :: ParquetOperations :: diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index 915f551fb2f01..d8e1b970c1d88 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -32,7 +32,13 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { kryo.setRegistrationRequired(false) kryo.register(classOf[MutablePair[_, _]]) kryo.register(classOf[Array[Any]]) + // This is kinda hacky... kryo.register(classOf[scala.collection.immutable.Map$Map1], new MapSerializer) + kryo.register(classOf[scala.collection.immutable.Map$Map2], new MapSerializer) + kryo.register(classOf[scala.collection.immutable.Map$Map3], new MapSerializer) + kryo.register(classOf[scala.collection.immutable.Map$Map4], new MapSerializer) + kryo.register(classOf[scala.collection.immutable.Map[_,_]], new MapSerializer) + kryo.register(classOf[scala.collection.Map[_,_]], new MapSerializer) 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[_]]) 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 e35ac0b6ca95a..b3e51fdf75270 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 @@ -158,10 +158,10 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { case other => other } - object TopK extends Strategy { + object TakeOrdered extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { - case logical.StopAfter(IntegerLiteral(limit), logical.Sort(order, child)) => - execution.TopK(limit, order, planLater(child))(sparkContext) :: Nil + case logical.Limit(IntegerLiteral(limit), logical.Sort(order, child)) => + execution.TakeOrdered(limit, order, planLater(child))(sparkContext) :: Nil case _ => Nil } } @@ -213,8 +213,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(IntegerLiteral(limit), child) => - execution.StopAfter(limit, planLater(child))(sparkContext) :: Nil + case logical.Limit(IntegerLiteral(limit), child) => + execution.Limit(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/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 65cb8f8becefa..524e5022ee14b 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 @@ -19,27 +19,28 @@ package org.apache.spark.sql.execution import scala.reflect.runtime.universe.TypeTag -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext - +import org.apache.spark.{HashPartitioner, SparkConf, SparkContext} +import org.apache.spark.rdd.{RDD, ShuffledRDD} +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{OrderedDistribution, UnspecifiedDistribution} -import org.apache.spark.sql.catalyst.ScalaReflection +import org.apache.spark.util.MutablePair + case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { - def output = projectList.map(_.toAttribute) + override def output = projectList.map(_.toAttribute) - def execute() = child.execute().mapPartitions { iter => + override def execute() = child.execute().mapPartitions { iter => @transient val reusableProjection = new MutableProjection(projectList) iter.map(reusableProjection) } } case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { - def output = child.output + override def output = child.output - def execute() = child.execute().mapPartitions { iter => + override def execute() = child.execute().mapPartitions { iter => iter.filter(condition.apply(_).asInstanceOf[Boolean]) } } @@ -47,37 +48,59 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan) extends UnaryNode { - def output = child.output + override def output = child.output // TODO: How to pick seed? - def execute() = child.execute().sample(withReplacement, fraction, seed) + override def execute() = child.execute().sample(withReplacement, fraction, seed) } 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())) + override def output = children.head.output + override def execute() = sc.union(children.map(_.execute())) override def otherCopyArgs = sc :: Nil } -case class StopAfter(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { +/** + * Take the first limit elements. Note that the implementation is different depending on whether + * this is a terminal operator or not. If it is terminal and is invoked using executeCollect, + * this operator uses Spark's take method on the Spark driver. If it is not terminal or is + * invoked using execute, we first take the limit on each partition, and then repartition all the + * data to a single partition to compute the global limit. + */ +case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { + // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan: + // partition local limit -> exchange into one partition -> partition local limit again + override def otherCopyArgs = sc :: Nil - def output = child.output + override def output = child.output 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|. - def execute() = sc.makeRDD(executeCollect(), 1) + override def execute() = { + val rdd = child.execute().mapPartitions { iter => + val mutablePair = new MutablePair[Boolean, Row]() + iter.take(limit).map(row => mutablePair.update(false, row)) + } + val part = new HashPartitioner(1) + val shuffled = new ShuffledRDD[Boolean, Row, MutablePair[Boolean, Row]](rdd, part) + shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.mapPartitions(_.take(limit).map(_._2)) + } } -case class TopK(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) - (@transient sc: SparkContext) extends UnaryNode { +/** + * Take the first limit elements as defined by the sortOrder. This is logically equivalent to + * having a [[Limit]] operator after a [[Sort]] operator. This could have been named TopK, but + * Spark's top operator does the opposite in ordering so we name it TakeOrdered to avoid confusion. + */ +case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) + (@transient sc: SparkContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil - def output = child.output + override def output = child.output @transient lazy val ordering = new RowOrdering(sortOrder) @@ -86,7 +109,7 @@ case class TopK(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) // TODO: Terminal split should be implemented differently from non-terminal split. // TODO: Pick num splits based on |limit|. - def execute() = sc.makeRDD(executeCollect(), 1) + override def execute() = sc.makeRDD(executeCollect(), 1) } @@ -101,7 +124,7 @@ case class Sort( @transient lazy val ordering = new RowOrdering(sortOrder) - def execute() = attachTree(this, "sort") { + override def execute() = attachTree(this, "sort") { // TODO: Optimize sorting operation? child.execute() .mapPartitions( @@ -109,7 +132,7 @@ case class Sort( preservesPartitioning = true) } - def output = child.output + override def output = child.output } object ExistingRdd { @@ -130,6 +153,6 @@ object ExistingRdd { } case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { - def execute() = rdd + override def execute() = rdd } 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 197b557cba5f4..46febbfad037d 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 @@ -188,7 +188,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val hiveContext = self override val strategies: Seq[Strategy] = Seq( - TopK, + TakeOrdered, ParquetOperations, HiveTableScans, DataSinks, 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 490a592a588d0..b2b03bc790fcc 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 @@ -529,7 +529,7 @@ object HiveQl { val withLimit = limitClause.map(l => nodeToExpr(l.getChildren.head)) - .map(StopAfter(_, withSort)) + .map(Limit(_, withSort)) .getOrElse(withSort) // TOK_INSERT_INTO means to add files to the table. @@ -602,7 +602,7 @@ object HiveQl { case Token("TOK_TABLESPLITSAMPLE", Token("TOK_ROWCOUNT", Nil) :: Token(count, Nil) :: Nil) => - StopAfter(Literal(count.toInt), relation) + Limit(Literal(count.toInt), relation) case Token("TOK_TABLESPLITSAMPLE", Token("TOK_PERCENT", Nil) :: Token(fraction, Nil) :: Nil) => From 9c65fa76f9d413e311a80f29d35d3ff7722e9476 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 2 Apr 2014 14:01:12 -0700 Subject: [PATCH 181/397] [SPARK-1212, Part II] Support sparse data in MLlib In PR https://github.com/apache/spark/pull/117, we added dense/sparse vector data model and updated KMeans to support sparse input. This PR is to replace all other `Array[Double]` usage by `Vector` in generalized linear models (GLMs) and Naive Bayes. Major changes: 1. `LabeledPoint` becomes `LabeledPoint(Double, Vector)`. 2. Methods that accept `RDD[Array[Double]]` now accept `RDD[Vector]`. We cannot support both in an elegant way because of type erasure. 3. Mark 'createModel' and 'predictPoint' protected because they are not for end users. 4. Add libSVMFile to MLContext. 5. NaiveBayes can accept arbitrary labels (introducing a breaking change to Python's `NaiveBayesModel`). 6. Gradient computation no longer creates temp vectors. 7. Column normalization and centering are removed from Lasso and Ridge because the operation will densify the data. Simple feature transformation can be done before training. TODO: 1. ~~Use axpy when possible.~~ 2. ~~Optimize Naive Bayes.~~ Author: Xiangrui Meng Closes #245 from mengxr/vector and squashes the following commits: eb6e793 [Xiangrui Meng] move libSVMFile to MLUtils and rename to loadLibSVMData c26c4fc [Xiangrui Meng] update DecisionTree to use RDD[Vector] 11999c7 [Xiangrui Meng] Merge branch 'master' into vector f7da54b [Xiangrui Meng] add minSplits to libSVMFile da25e24 [Xiangrui Meng] revert the change to default addIntercept because it might change the behavior of existing code without warning 493f26f [Xiangrui Meng] Merge branch 'master' into vector 7c1bc01 [Xiangrui Meng] add a TODO to NB b9b7ef7 [Xiangrui Meng] change default value of addIntercept to false b01df54 [Xiangrui Meng] allow to change or clear threshold in LR and SVM 4addc50 [Xiangrui Meng] merge master 4ca5b1b [Xiangrui Meng] remove normalization from Lasso and update tests f04fe8a [Xiangrui Meng] remove normalization from RidgeRegression and update tests d088552 [Xiangrui Meng] use static constructor for MLContext 6f59eed [Xiangrui Meng] update libSVMFile to determine number of features automatically 3432e84 [Xiangrui Meng] update NaiveBayes to support sparse data 0f8759b [Xiangrui Meng] minor updates to NB b11659c [Xiangrui Meng] style update 78c4671 [Xiangrui Meng] add libSVMFile to MLContext f0fe616 [Xiangrui Meng] add a test for sparse linear regression 44733e1 [Xiangrui Meng] use in-place gradient computation e981396 [Xiangrui Meng] use axpy in Updater db808a1 [Xiangrui Meng] update JavaLR example befa592 [Xiangrui Meng] passed scala/java tests 75c83a4 [Xiangrui Meng] passed test compile 1859701 [Xiangrui Meng] passed compile 834ada2 [Xiangrui Meng] optimized MLUtils.computeStats update some ml algorithms to use Vector (cont.) 135ab72 [Xiangrui Meng] merge glm 0e57aa4 [Xiangrui Meng] update Lasso and RidgeRegression to parse the weights correctly from GLM mark createModel protected mark predictPoint protected d7f629f [Xiangrui Meng] fix a bug in GLM when intercept is not used 3f346ba [Xiangrui Meng] update some ml algorithms to use Vector --- .../apache/spark/mllib/examples/JavaLR.java | 14 +- .../mllib/api/python/PythonMLLibAPI.scala | 154 ++++++++++---- .../classification/ClassificationModel.scala | 13 +- .../classification/LogisticRegression.scala | 84 ++++---- .../mllib/classification/NaiveBayes.scala | 116 ++++++----- .../spark/mllib/classification/SVM.scala | 69 ++++--- .../spark/mllib/clustering/KMeans.scala | 3 +- .../apache/spark/mllib/linalg/Vectors.scala | 8 + .../spark/mllib/optimization/Gradient.scala | 117 ++++++++--- .../mllib/optimization/GradientDescent.scala | 58 +++--- .../spark/mllib/optimization/Optimizer.scala | 7 +- .../spark/mllib/optimization/Updater.scala | 69 +++++-- .../GeneralizedLinearAlgorithm.scala | 69 ++++--- .../spark/mllib/regression/LabeledPoint.scala | 6 +- .../apache/spark/mllib/regression/Lasso.scala | 79 ++----- .../mllib/regression/LinearRegression.scala | 37 ++-- .../mllib/regression/RegressionModel.scala | 5 +- .../mllib/regression/RidgeRegression.scala | 75 ++----- .../spark/mllib/tree/DecisionTree.scala | 9 +- .../mllib/tree/model/DecisionTreeModel.scala | 5 +- .../apache/spark/mllib/tree/model/Node.scala | 7 +- .../mllib/util/LinearDataGenerator.scala | 3 +- .../LogisticRegressionDataGenerator.scala | 3 +- .../org/apache/spark/mllib/util/MLUtils.scala | 193 ++++++++++++++---- .../spark/mllib/util/SVMDataGenerator.scala | 3 +- .../classification/JavaNaiveBayesSuite.java | 13 +- .../mllib/classification/JavaSVMSuite.java | 3 - .../spark/mllib/linalg/JavaVectorsSuite.java | 6 +- .../mllib/regression/JavaLassoSuite.java | 4 +- .../regression/JavaRidgeRegressionSuite.java | 38 ++-- .../LogisticRegressionSuite.scala | 7 +- .../classification/NaiveBayesSuite.scala | 4 +- .../spark/mllib/classification/SVMSuite.scala | 10 +- .../optimization/GradientDescentSuite.scala | 14 +- .../spark/mllib/regression/LassoSuite.scala | 51 +++-- .../regression/LinearRegressionSuite.scala | 54 ++++- .../regression/RidgeRegressionSuite.scala | 27 +-- .../spark/mllib/tree/DecisionTreeSuite.scala | 9 +- .../spark/mllib/util/MLUtilsSuite.scala | 59 +++++- python/pyspark/mllib/classification.py | 12 +- 40 files changed, 926 insertions(+), 591 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index 667c72f379e71..cd8879ff886e2 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -17,6 +17,7 @@ package org.apache.spark.mllib.examples; +import java.util.regex.Pattern; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -24,11 +25,9 @@ import org.apache.spark.mllib.classification.LogisticRegressionWithSGD; import org.apache.spark.mllib.classification.LogisticRegressionModel; +import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; -import java.util.Arrays; -import java.util.regex.Pattern; - /** * Logistic regression based classification using ML Lib. */ @@ -47,14 +46,10 @@ public LabeledPoint call(String line) { for (int i = 0; i < tok.length; ++i) { x[i] = Double.parseDouble(tok[i]); } - return new LabeledPoint(y, x); + return new LabeledPoint(y, Vectors.dense(x)); } } - public static void printWeights(double[] a) { - System.out.println(Arrays.toString(a)); - } - public static void main(String[] args) { if (args.length != 4) { System.err.println("Usage: JavaLR "); @@ -80,8 +75,7 @@ public static void main(String[] args) { LogisticRegressionModel model = LogisticRegressionWithSGD.train(points.rdd(), iterations, stepSize); - System.out.print("Final w: "); - printWeights(model.weights()); + System.out.print("Final w: " + model.weights()); System.exit(0); } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 3449c698da60b..2df5b0d02b699 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -110,16 +110,16 @@ class PythonMLLibAPI extends Serializable { private def trainRegressionModel( trainFunc: (RDD[LabeledPoint], Array[Double]) => GeneralizedLinearModel, - dataBytesJRDD: JavaRDD[Array[Byte]], initialWeightsBA: Array[Byte]): - java.util.LinkedList[java.lang.Object] = { + dataBytesJRDD: JavaRDD[Array[Byte]], + initialWeightsBA: Array[Byte]): java.util.LinkedList[java.lang.Object] = { val data = dataBytesJRDD.rdd.map(xBytes => { val x = deserializeDoubleVector(xBytes) - LabeledPoint(x(0), x.slice(1, x.length)) + LabeledPoint(x(0), Vectors.dense(x.slice(1, x.length))) }) val initialWeights = deserializeDoubleVector(initialWeightsBA) val model = trainFunc(data, initialWeights) val ret = new java.util.LinkedList[java.lang.Object]() - ret.add(serializeDoubleVector(model.weights)) + ret.add(serializeDoubleVector(model.weights.toArray)) ret.add(model.intercept: java.lang.Double) ret } @@ -127,75 +127,127 @@ class PythonMLLibAPI extends Serializable { /** * Java stub for Python mllib LinearRegressionWithSGD.train() */ - def trainLinearRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], - numIterations: Int, stepSize: Double, miniBatchFraction: Double, + def trainLinearRegressionModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - LinearRegressionWithSGD.train(data, numIterations, stepSize, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + LinearRegressionWithSGD.train( + data, + numIterations, + stepSize, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** * Java stub for Python mllib LassoWithSGD.train() */ - def trainLassoModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int, - stepSize: Double, regParam: Double, miniBatchFraction: Double, + def trainLassoModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - LassoWithSGD.train(data, numIterations, stepSize, regParam, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + LassoWithSGD.train( + data, + numIterations, + stepSize, + regParam, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** * Java stub for Python mllib RidgeRegressionWithSGD.train() */ - def trainRidgeModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int, - stepSize: Double, regParam: Double, miniBatchFraction: Double, + def trainRidgeModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - RidgeRegressionWithSGD.train(data, numIterations, stepSize, regParam, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + RidgeRegressionWithSGD.train( + data, + numIterations, + stepSize, + regParam, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** * Java stub for Python mllib SVMWithSGD.train() */ - def trainSVMModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], numIterations: Int, - stepSize: Double, regParam: Double, miniBatchFraction: Double, + def trainSVMModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + regParam: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - SVMWithSGD.train(data, numIterations, stepSize, regParam, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + SVMWithSGD.train( + data, + numIterations, + stepSize, + regParam, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** * Java stub for Python mllib LogisticRegressionWithSGD.train() */ - def trainLogisticRegressionModelWithSGD(dataBytesJRDD: JavaRDD[Array[Byte]], - numIterations: Int, stepSize: Double, miniBatchFraction: Double, + def trainLogisticRegressionModelWithSGD( + dataBytesJRDD: JavaRDD[Array[Byte]], + numIterations: Int, + stepSize: Double, + miniBatchFraction: Double, initialWeightsBA: Array[Byte]): java.util.List[java.lang.Object] = { - trainRegressionModel((data, initialWeights) => - LogisticRegressionWithSGD.train(data, numIterations, stepSize, - miniBatchFraction, initialWeights), - dataBytesJRDD, initialWeightsBA) + trainRegressionModel( + (data, initialWeights) => + LogisticRegressionWithSGD.train( + data, + numIterations, + stepSize, + miniBatchFraction, + Vectors.dense(initialWeights)), + dataBytesJRDD, + initialWeightsBA) } /** * Java stub for NaiveBayes.train() */ - def trainNaiveBayes(dataBytesJRDD: JavaRDD[Array[Byte]], lambda: Double) - : java.util.List[java.lang.Object] = - { + def trainNaiveBayes( + dataBytesJRDD: JavaRDD[Array[Byte]], + lambda: Double): java.util.List[java.lang.Object] = { val data = dataBytesJRDD.rdd.map(xBytes => { val x = deserializeDoubleVector(xBytes) - LabeledPoint(x(0), x.slice(1, x.length)) + LabeledPoint(x(0), Vectors.dense(x.slice(1, x.length))) }) val model = NaiveBayes.train(data, lambda) val ret = new java.util.LinkedList[java.lang.Object]() + ret.add(serializeDoubleVector(model.labels)) ret.add(serializeDoubleVector(model.pi)) ret.add(serializeDoubleMatrix(model.theta)) ret @@ -204,9 +256,12 @@ class PythonMLLibAPI extends Serializable { /** * Java stub for Python mllib KMeans.train() */ - def trainKMeansModel(dataBytesJRDD: JavaRDD[Array[Byte]], k: Int, - maxIterations: Int, runs: Int, initializationMode: String): - java.util.List[java.lang.Object] = { + def trainKMeansModel( + dataBytesJRDD: JavaRDD[Array[Byte]], + k: Int, + maxIterations: Int, + runs: Int, + initializationMode: String): java.util.List[java.lang.Object] = { val data = dataBytesJRDD.rdd.map(xBytes => Vectors.dense(deserializeDoubleVector(xBytes))) val model = KMeans.train(data, k, maxIterations, runs, initializationMode) val ret = new java.util.LinkedList[java.lang.Object]() @@ -259,8 +314,12 @@ class PythonMLLibAPI extends Serializable { * needs to be taken in the Python code to ensure it gets freed on exit; see * the Py4J documentation. */ - def trainALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int, - iterations: Int, lambda: Double, blocks: Int): MatrixFactorizationModel = { + def trainALSModel( + ratingsBytesJRDD: JavaRDD[Array[Byte]], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int): MatrixFactorizationModel = { val ratings = ratingsBytesJRDD.rdd.map(unpackRating) ALS.train(ratings, rank, iterations, lambda, blocks) } @@ -271,8 +330,13 @@ class PythonMLLibAPI extends Serializable { * Extra care needs to be taken in the Python code to ensure it gets freed on * exit; see the Py4J documentation. */ - def trainImplicitALSModel(ratingsBytesJRDD: JavaRDD[Array[Byte]], rank: Int, - iterations: Int, lambda: Double, blocks: Int, alpha: Double): MatrixFactorizationModel = { + def trainImplicitALSModel( + ratingsBytesJRDD: JavaRDD[Array[Byte]], + rank: Int, + iterations: Int, + lambda: Double, + blocks: Int, + alpha: Double): MatrixFactorizationModel = { val ratings = ratingsBytesJRDD.rdd.map(unpackRating) ALS.trainImplicit(ratings, rank, iterations, lambda, blocks, alpha) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala index 391f5b9b7a7de..bd10e2e9e10e2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/ClassificationModel.scala @@ -17,22 +17,27 @@ package org.apache.spark.mllib.classification +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.rdd.RDD +/** + * Represents a classification model that predicts to which of a set of categories an example + * belongs. The categories are represented by double values: 0.0, 1.0, 2.0, etc. + */ trait ClassificationModel extends Serializable { /** * Predict values for the given data set using the model trained. * * @param testData RDD representing data points to be predicted - * @return RDD[Int] where each entry contains the corresponding prediction + * @return an RDD[Double] where each entry contains the corresponding prediction */ - def predict(testData: RDD[Array[Double]]): RDD[Double] + def predict(testData: RDD[Vector]): RDD[Double] /** * Predict values for a single data point using the model trained. * * @param testData array representing a single data point - * @return Int prediction from the trained model + * @return predicted category from the trained model */ - def predict(testData: Array[Double]): Double + def predict(testData: Vector): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index a481f522761e2..798f3a5c94740 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -17,16 +17,12 @@ package org.apache.spark.mllib.classification -import scala.math.round - import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.util.DataValidators - -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.rdd.RDD /** * Classification model trained using Logistic Regression. @@ -35,15 +31,38 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class LogisticRegressionModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) - with ClassificationModel with Serializable { + extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { + + private var threshold: Option[Double] = Some(0.5) + + /** + * Sets the threshold that separates positive predictions from negative predictions. An example + * with prediction score greater than or equal to this threshold is identified as an positive, + * and negative otherwise. The default value is 0.5. + */ + def setThreshold(threshold: Double): this.type = { + this.threshold = Some(threshold) + this + } - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, + /** + * Clears the threshold so that `predict` will output raw prediction scores. + */ + def clearThreshold(): this.type = { + threshold = None + this + } + + override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { - val margin = dataMatrix.mmul(weightMatrix).get(0) + intercept - round(1.0/ (1.0 + math.exp(margin * -1))) + val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + val score = 1.0/ (1.0 + math.exp(-margin)) + threshold match { + case Some(t) => if (score < t) 0.0 else 1.0 + case None => score + } } } @@ -56,16 +75,15 @@ class LogisticRegressionWithSGD private ( var numIterations: Int, var regParam: Double, var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[LogisticRegressionModel] - with Serializable { + extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { val gradient = new LogisticGradient() val updater = new SimpleUpdater() override val optimizer = new GradientDescent(gradient, updater) - .setStepSize(stepSize) - .setNumIterations(numIterations) - .setRegParam(regParam) - .setMiniBatchFraction(miniBatchFraction) + .setStepSize(stepSize) + .setNumIterations(numIterations) + .setRegParam(regParam) + .setMiniBatchFraction(miniBatchFraction) override val validators = List(DataValidators.classificationLabels) /** @@ -73,7 +91,7 @@ class LogisticRegressionWithSGD private ( */ def this() = this(1.0, 100, 0.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { + def createModel(weights: Vector, intercept: Double) = { new LogisticRegressionModel(weights, intercept) } } @@ -105,11 +123,9 @@ object LogisticRegressionWithSGD { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) - : LogisticRegressionModel = - { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( - input, initialWeights) + initialWeights: Vector): LogisticRegressionModel = { + new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction) + .run(input, initialWeights) } /** @@ -128,11 +144,9 @@ object LogisticRegressionWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - miniBatchFraction: Double) - : LogisticRegressionModel = - { - new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction).run( - input) + miniBatchFraction: Double): LogisticRegressionModel = { + new LogisticRegressionWithSGD(stepSize, numIterations, 0.0, miniBatchFraction) + .run(input) } /** @@ -150,9 +164,7 @@ object LogisticRegressionWithSGD { def train( input: RDD[LabeledPoint], numIterations: Int, - stepSize: Double) - : LogisticRegressionModel = - { + stepSize: Double): LogisticRegressionModel = { train(input, numIterations, stepSize, 1.0) } @@ -168,9 +180,7 @@ object LogisticRegressionWithSGD { */ def train( input: RDD[LabeledPoint], - numIterations: Int) - : LogisticRegressionModel = - { + numIterations: Int): LogisticRegressionModel = { train(input, numIterations, 1.0, 1.0) } @@ -183,7 +193,7 @@ object LogisticRegressionWithSGD { val sc = new SparkContext(args(0), "LogisticRegression") val data = MLUtils.loadLabeledData(sc, args(1)) val model = LogisticRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 6539b2f339465..e956185319a69 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -17,14 +17,14 @@ package org.apache.spark.mllib.classification -import scala.collection.mutable +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} -import org.jblas.DoubleMatrix - -import org.apache.spark.{SparkContext, Logging} +import org.apache.spark.{Logging, SparkContext} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.rdd.RDD import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD /** * Model for Naive Bayes Classifiers. @@ -32,19 +32,28 @@ import org.apache.spark.mllib.util.MLUtils * @param pi Log of class priors, whose dimension is C. * @param theta Log of class conditional probabilities, whose dimension is CxD. */ -class NaiveBayesModel(val pi: Array[Double], val theta: Array[Array[Double]]) - extends ClassificationModel with Serializable { - - // Create a column vector that can be used for predictions - private val _pi = new DoubleMatrix(pi.length, 1, pi: _*) - private val _theta = new DoubleMatrix(theta) +class NaiveBayesModel( + val labels: Array[Double], + val pi: Array[Double], + val theta: Array[Array[Double]]) extends ClassificationModel with Serializable { + + private val brzPi = new BDV[Double](pi) + private val brzTheta = new BDM[Double](theta.length, theta(0).length) + + var i = 0 + while (i < theta.length) { + var j = 0 + while (j < theta(i).length) { + brzTheta(i, j) = theta(i)(j) + j += 1 + } + i += 1 + } - def predict(testData: RDD[Array[Double]]): RDD[Double] = testData.map(predict) + override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) - def predict(testData: Array[Double]): Double = { - val dataMatrix = new DoubleMatrix(testData.length, 1, testData: _*) - val result = _pi.add(_theta.mmul(dataMatrix)) - result.argmax() + override def predict(testData: Vector): Double = { + labels(brzArgmax(brzPi + brzTheta * testData.toBreeze)) } } @@ -56,9 +65,8 @@ class NaiveBayesModel(val pi: Array[Double], val theta: Array[Array[Double]]) * document classification. By making every vector a 0-1 vector, it can also be used as * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). */ -class NaiveBayes private (var lambda: Double) - extends Serializable with Logging -{ +class NaiveBayes private (var lambda: Double) extends Serializable with Logging { + def this() = this(1.0) /** Set the smoothing parameter. Default: 1.0. */ @@ -70,45 +78,42 @@ class NaiveBayes private (var lambda: Double) /** * Run the algorithm with the configured parameters on an input RDD of LabeledPoint entries. * - * @param data RDD of (label, array of features) pairs. + * @param data RDD of [[org.apache.spark.mllib.regression.LabeledPoint]]. */ def run(data: RDD[LabeledPoint]) = { - // Aggregates all sample points to driver side to get sample count and summed feature vector - // for each label. The shape of `zeroCombiner` & `aggregated` is: - // - // label: Int -> (count: Int, featuresSum: DoubleMatrix) - val zeroCombiner = mutable.Map.empty[Int, (Int, DoubleMatrix)] - val aggregated = data.aggregate(zeroCombiner)({ (combiner, point) => - point match { - case LabeledPoint(label, features) => - val (count, featuresSum) = combiner.getOrElse(label.toInt, (0, DoubleMatrix.zeros(1))) - val fs = new DoubleMatrix(features.length, 1, features: _*) - combiner += label.toInt -> (count + 1, featuresSum.addi(fs)) - } - }, { (lhs, rhs) => - for ((label, (c, fs)) <- rhs) { - val (count, featuresSum) = lhs.getOrElse(label, (0, DoubleMatrix.zeros(1))) - lhs(label) = (count + c, featuresSum.addi(fs)) + // Aggregates term frequencies per label. + // TODO: Calling combineByKey and collect creates two stages, we can implement something + // TODO: similar to reduceByKeyLocally to save one stage. + val aggregated = data.map(p => (p.label, p.features)).combineByKey[(Long, BDV[Double])]( + createCombiner = (v: Vector) => (1L, v.toBreeze.toDenseVector), + mergeValue = (c: (Long, BDV[Double]), v: Vector) => (c._1 + 1L, c._2 += v.toBreeze), + mergeCombiners = (c1: (Long, BDV[Double]), c2: (Long, BDV[Double])) => + (c1._1 + c2._1, c1._2 += c2._2) + ).collect() + val numLabels = aggregated.length + var numDocuments = 0L + aggregated.foreach { case (_, (n, _)) => + numDocuments += n + } + val numFeatures = aggregated.head match { case (_, (_, v)) => v.size } + val labels = new Array[Double](numLabels) + val pi = new Array[Double](numLabels) + val theta = Array.fill(numLabels)(new Array[Double](numFeatures)) + val piLogDenom = math.log(numDocuments + numLabels * lambda) + var i = 0 + aggregated.foreach { case (label, (n, sumTermFreqs)) => + labels(i) = label + val thetaLogDenom = math.log(brzSum(sumTermFreqs) + numFeatures * lambda) + pi(i) = math.log(n + lambda) - piLogDenom + var j = 0 + while (j < numFeatures) { + theta(i)(j) = math.log(sumTermFreqs(j) + lambda) - thetaLogDenom + j += 1 } - lhs - }) - - // Kinds of label - val C = aggregated.size - // Total sample count - val N = aggregated.values.map(_._1).sum - - val pi = new Array[Double](C) - val theta = new Array[Array[Double]](C) - val piLogDenom = math.log(N + C * lambda) - - for ((label, (count, fs)) <- aggregated) { - val thetaLogDenom = math.log(fs.sum() + fs.length * lambda) - pi(label) = math.log(count + lambda) - piLogDenom - theta(label) = fs.toArray.map(f => math.log(f + lambda) - thetaLogDenom) + i += 1 } - new NaiveBayesModel(pi, theta) + new NaiveBayesModel(labels, pi, theta) } } @@ -158,8 +163,9 @@ object NaiveBayes { } else { NaiveBayes.train(data, args(2).toDouble) } - println("Pi: " + model.pi.mkString("[", ", ", "]")) - println("Theta:\n" + model.theta.map(_.mkString("[", ", ", "]")).mkString("[", "\n ", "]")) + + println("Pi\n: " + model.pi) + println("Theta:\n" + model.theta) sc.stop() } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index 6dff29dfb45cc..e31a08899f8bc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -18,13 +18,11 @@ package org.apache.spark.mllib.classification import org.apache.spark.SparkContext -import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.regression._ -import org.apache.spark.mllib.util.MLUtils -import org.apache.spark.mllib.util.DataValidators - -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.util.{DataValidators, MLUtils} +import org.apache.spark.rdd.RDD /** * Model for Support Vector Machines (SVMs). @@ -33,15 +31,37 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class SVMModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) - extends GeneralizedLinearModel(weights, intercept) - with ClassificationModel with Serializable { + extends GeneralizedLinearModel(weights, intercept) with ClassificationModel with Serializable { + + private var threshold: Option[Double] = Some(0.0) + + /** + * Sets the threshold that separates positive predictions from negative predictions. An example + * with prediction score greater than or equal to this threshold is identified as an positive, + * and negative otherwise. The default value is 0.0. + */ + def setThreshold(threshold: Double): this.type = { + this.threshold = Some(threshold) + this + } - override def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, + /** + * Clears the threshold so that `predict` will output raw prediction scores. + */ + def clearThreshold(): this.type = { + threshold = None + this + } + + override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { - val margin = dataMatrix.dot(weightMatrix) + intercept - if (margin < 0) 0.0 else 1.0 + val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept + threshold match { + case Some(t) => if (margin < 0) 0.0 else 1.0 + case None => margin + } } } @@ -71,7 +91,7 @@ class SVMWithSGD private ( */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Array[Double], intercept: Double) = { + def createModel(weights: Vector, intercept: Double) = { new SVMModel(weights, intercept) } } @@ -103,11 +123,9 @@ object SVMWithSGD { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) - : SVMModel = - { - new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, - initialWeights) + initialWeights: Vector): SVMModel = { + new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction) + .run(input, initialWeights) } /** @@ -127,9 +145,7 @@ object SVMWithSGD { numIterations: Int, stepSize: Double, regParam: Double, - miniBatchFraction: Double) - : SVMModel = - { + miniBatchFraction: Double): SVMModel = { new SVMWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) } @@ -149,9 +165,7 @@ object SVMWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - regParam: Double) - : SVMModel = - { + regParam: Double): SVMModel = { train(input, numIterations, stepSize, regParam, 1.0) } @@ -165,11 +179,7 @@ object SVMWithSGD { * @param numIterations Number of iterations of gradient descent to run. * @return a SVMModel which has the weights and offset from training. */ - def train( - input: RDD[LabeledPoint], - numIterations: Int) - : SVMModel = - { + def train(input: RDD[LabeledPoint], numIterations: Int): SVMModel = { train(input, numIterations, 1.0, 1.0, 1.0) } @@ -181,7 +191,8 @@ object SVMWithSGD { val sc = new SparkContext(args(0), "SVM") val data = MLUtils.loadLabeledData(sc, args(1)) val model = SVMWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index b412738e3f00a..a78503df3134d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -42,8 +42,7 @@ class KMeans private ( var runs: Int, var initializationMode: String, var initializationSteps: Int, - var epsilon: Double) - extends Serializable with Logging { + var epsilon: Double) extends Serializable with Logging { def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) /** Set the number of clusters to create (k). Default: 2. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 01c1501548f87..2cea58cd3fd22 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -54,6 +54,12 @@ trait Vector extends Serializable { * Converts the instance to a breeze vector. */ private[mllib] def toBreeze: BV[Double] + + /** + * Gets the value of the ith element. + * @param i index + */ + private[mllib] def apply(i: Int): Double = toBreeze(i) } /** @@ -145,6 +151,8 @@ class DenseVector(val values: Array[Double]) extends Vector { override def toArray: Array[Double] = values private[mllib] override def toBreeze: BV[Double] = new BDV[Double](values) + + override def apply(i: Int) = values(i) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 82124703da6cd..20654284965ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -17,7 +17,9 @@ package org.apache.spark.mllib.optimization -import org.jblas.DoubleMatrix +import breeze.linalg.{axpy => brzAxpy} + +import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * Class used to compute the gradient for a loss function, given a single data point. @@ -26,17 +28,26 @@ abstract class Gradient extends Serializable { /** * Compute the gradient and loss given the features of a single data point. * - * @param data - Feature values for one data point. Column matrix of size dx1 - * where d is the number of features. - * @param label - Label for this data item. - * @param weights - Column matrix containing weights for every feature. + * @param data features for one data point + * @param label label for this data point + * @param weights weights/coefficients corresponding to features * - * @return A tuple of 2 elements. The first element is a column matrix containing the computed - * gradient and the second element is the loss computed at this data point. + * @return (gradient: Vector, loss: Double) + */ + def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) + + /** + * Compute the gradient and loss given the features of a single data point, + * add the gradient to a provided vector to avoid creating new objects, and return loss. * + * @param data features for one data point + * @param label label for this data point + * @param weights weights/coefficients corresponding to features + * @param cumGradient the computed gradient will be added to this vector + * + * @return loss */ - def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) + def compute(data: Vector, label: Double, weights: Vector, cumGradient: Vector): Double } /** @@ -44,12 +55,12 @@ abstract class Gradient extends Serializable { * See also the documentation for the precise formulation. */ class LogisticGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { - val margin: Double = -1.0 * data.dot(weights) + override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val margin: Double = -1.0 * brzWeights.dot(brzData) val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label - - val gradient = data.mul(gradientMultiplier) + val gradient = brzData * gradientMultiplier val loss = if (label > 0) { math.log(1 + math.exp(margin)) @@ -57,7 +68,26 @@ class LogisticGradient extends Gradient { math.log(1 + math.exp(margin)) - margin } - (gradient, loss) + (Vectors.fromBreeze(gradient), loss) + } + + override def compute( + data: Vector, + label: Double, + weights: Vector, + cumGradient: Vector): Double = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val margin: Double = -1.0 * brzWeights.dot(brzData) + val gradientMultiplier = (1.0 / (1.0 + math.exp(margin))) - label + + brzAxpy(gradientMultiplier, brzData, cumGradient.toBreeze) + + if (label > 0) { + math.log(1 + math.exp(margin)) + } else { + math.log(1 + math.exp(margin)) - margin + } } } @@ -68,14 +98,28 @@ class LogisticGradient extends Gradient { * See also the documentation for the precise formulation. */ class LeastSquaresGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { - val diff: Double = data.dot(weights) - label - + override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val diff = brzWeights.dot(brzData) - label val loss = diff * diff - val gradient = data.mul(2.0 * diff) + val gradient = brzData * (2.0 * diff) - (gradient, loss) + (Vectors.fromBreeze(gradient), loss) + } + + override def compute( + data: Vector, + label: Double, + weights: Vector, + cumGradient: Vector): Double = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val diff = brzWeights.dot(brzData) - label + + brzAxpy(2.0 * diff, brzData, cumGradient.toBreeze) + + diff * diff } } @@ -85,19 +129,40 @@ class LeastSquaresGradient extends Gradient { * NOTE: This assumes that the labels are {0,1} */ class HingeGradient extends Gradient { - override def compute(data: DoubleMatrix, label: Double, weights: DoubleMatrix): - (DoubleMatrix, Double) = { + override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val dotProduct = brzWeights.dot(brzData) + + // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) + // Therefore the gradient is -(2y - 1)*x + val labelScaled = 2 * label - 1.0 + + if (1.0 > labelScaled * dotProduct) { + (Vectors.fromBreeze(brzData * (-labelScaled)), 1.0 - labelScaled * dotProduct) + } else { + (Vectors.dense(new Array[Double](weights.size)), 0.0) + } + } - val dotProduct = data.dot(weights) + override def compute( + data: Vector, + label: Double, + weights: Vector, + cumGradient: Vector): Double = { + val brzData = data.toBreeze + val brzWeights = weights.toBreeze + val dotProduct = brzWeights.dot(brzData) // Our loss function with {0, 1} labels is max(0, 1 - (2y – 1) (f_w(x))) // Therefore the gradient is -(2y - 1)*x val labelScaled = 2 * label - 1.0 if (1.0 > labelScaled * dotProduct) { - (data.mul(-labelScaled), 1.0 - labelScaled * dotProduct) + brzAxpy(-labelScaled, brzData, cumGradient.toBreeze) + 1.0 - labelScaled * dotProduct } else { - (DoubleMatrix.zeros(1, weights.length), 0.0) + 0.0 } } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index b967b22e818d3..d0777ffd63ff8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -17,12 +17,13 @@ package org.apache.spark.mllib.optimization -import org.apache.spark.Logging -import org.apache.spark.rdd.RDD +import scala.collection.mutable.ArrayBuffer -import org.jblas.DoubleMatrix +import breeze.linalg.{Vector => BV, DenseVector => BDV} -import scala.collection.mutable.ArrayBuffer +import org.apache.spark.Logging +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * Class used to solve an optimization problem using Gradient Descent. @@ -91,18 +92,16 @@ class GradientDescent(var gradient: Gradient, var updater: Updater) this } - def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double]) - : Array[Double] = { - - val (weights, stochasticLossHistory) = GradientDescent.runMiniBatchSGD( - data, - gradient, - updater, - stepSize, - numIterations, - regParam, - miniBatchFraction, - initialWeights) + def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector = { + val (weights, _) = GradientDescent.runMiniBatchSGD( + data, + gradient, + updater, + stepSize, + numIterations, + regParam, + miniBatchFraction, + initialWeights) weights } @@ -133,14 +132,14 @@ object GradientDescent extends Logging { * stochastic loss computed for every iteration. */ def runMiniBatchSGD( - data: RDD[(Double, Array[Double])], + data: RDD[(Double, Vector)], gradient: Gradient, updater: Updater, stepSize: Double, numIterations: Int, regParam: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) : (Array[Double], Array[Double]) = { + initialWeights: Vector): (Vector, Array[Double]) = { val stochasticLossHistory = new ArrayBuffer[Double](numIterations) @@ -148,24 +147,27 @@ object GradientDescent extends Logging { val miniBatchSize = nexamples * miniBatchFraction // Initialize weights as a column vector - var weights = new DoubleMatrix(initialWeights.length, 1, initialWeights:_*) + var weights = Vectors.dense(initialWeights.toArray) /** * For the first iteration, the regVal will be initialized as sum of sqrt of * weights if it's L2 update; for L1 update; the same logic is followed. */ var regVal = updater.compute( - weights, new DoubleMatrix(initialWeights.length, 1), 0, 1, regParam)._2 + weights, Vectors.dense(new Array[Double](weights.size)), 0, 1, regParam)._2 for (i <- 1 to numIterations) { // Sample a subset (fraction miniBatchFraction) of the total data // compute and sum up the subgradients on this subset (this is one map-reduce) - val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i).map { - case (y, features) => - val featuresCol = new DoubleMatrix(features.length, 1, features:_*) - val (grad, loss) = gradient.compute(featuresCol, y, weights) - (grad, loss) - }.reduce((a, b) => (a._1.addi(b._1), a._2 + b._2)) + val (gradientSum, lossSum) = data.sample(false, miniBatchFraction, 42 + i) + .aggregate((BDV.zeros[Double](weights.size), 0.0))( + seqOp = (c, v) => (c, v) match { case ((grad, loss), (label, features)) => + val l = gradient.compute(features, label, weights, Vectors.fromBreeze(grad)) + (grad, loss + l) + }, + combOp = (c1, c2) => (c1, c2) match { case ((grad1, loss1), (grad2, loss2)) => + (grad1 += grad2, loss1 + loss2) + }) /** * NOTE(Xinghao): lossSum is computed using the weights from the previous iteration @@ -173,7 +175,7 @@ object GradientDescent extends Logging { */ stochasticLossHistory.append(lossSum / miniBatchSize + regVal) val update = updater.compute( - weights, gradientSum.div(miniBatchSize), stepSize, i, regParam) + weights, Vectors.fromBreeze(gradientSum / miniBatchSize), stepSize, i, regParam) weights = update._1 regVal = update._2 } @@ -181,6 +183,6 @@ object GradientDescent extends Logging { logInfo("GradientDescent.runMiniBatchSGD finished. Last 10 stochastic losses %s".format( stochasticLossHistory.takeRight(10).mkString(", "))) - (weights.toArray, stochasticLossHistory.toArray) + (weights, stochasticLossHistory.toArray) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index 94d30b56f212b..f9ce908a5f3b0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -19,11 +19,12 @@ package org.apache.spark.mllib.optimization import org.apache.spark.rdd.RDD -trait Optimizer { +import org.apache.spark.mllib.linalg.Vector + +trait Optimizer extends Serializable { /** * Solve the provided convex optimization problem. */ - def optimize(data: RDD[(Double, Array[Double])], initialWeights: Array[Double]): Array[Double] - + def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index bf8f731459e99..3b7754cd7ac28 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -18,7 +18,10 @@ package org.apache.spark.mllib.optimization import scala.math._ -import org.jblas.DoubleMatrix + +import breeze.linalg.{norm => brzNorm, axpy => brzAxpy, Vector => BV} + +import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * Class used to perform steps (weight update) using Gradient Descent methods. @@ -47,8 +50,12 @@ abstract class Updater extends Serializable { * @return A tuple of 2 elements. The first element is a column matrix containing updated weights, * and the second element is the regularization value computed using updated weights. */ - def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, stepSize: Double, iter: Int, - regParam: Double): (DoubleMatrix, Double) + def compute( + weightsOld: Vector, + gradient: Vector, + stepSize: Double, + iter: Int, + regParam: Double): (Vector, Double) } /** @@ -56,11 +63,17 @@ abstract class Updater extends Serializable { * Uses a step-size decreasing with the square root of the number of iterations. */ class SimpleUpdater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + override def compute( + weightsOld: Vector, + gradient: Vector, + stepSize: Double, + iter: Int, + regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) - val step = gradient.mul(thisIterStepSize) - (weightsOld.sub(step), 0) + val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector + brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + + (Vectors.fromBreeze(brzWeights), 0) } } @@ -83,19 +96,26 @@ class SimpleUpdater extends Updater { * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) */ class L1Updater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { + override def compute( + weightsOld: Vector, + gradient: Vector, + stepSize: Double, + iter: Int, + regParam: Double): (Vector, Double) = { val thisIterStepSize = stepSize / math.sqrt(iter) - val step = gradient.mul(thisIterStepSize) // Take gradient step - val newWeights = weightsOld.sub(step) + val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector + brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) // Apply proximal operator (soft thresholding) val shrinkageVal = regParam * thisIterStepSize - (0 until newWeights.length).foreach { i => - val wi = newWeights.get(i) - newWeights.put(i, signum(wi) * max(0.0, abs(wi) - shrinkageVal)) + var i = 0 + while (i < brzWeights.length) { + val wi = brzWeights(i) + brzWeights(i) = signum(wi) * max(0.0, abs(wi) - shrinkageVal) + i += 1 } - (newWeights, newWeights.norm1 * regParam) + + (Vectors.fromBreeze(brzWeights), brzNorm(brzWeights, 1.0) * regParam) } } @@ -105,16 +125,23 @@ class L1Updater extends Updater { * Uses a step-size decreasing with the square root of the number of iterations. */ class SquaredL2Updater extends Updater { - override def compute(weightsOld: DoubleMatrix, gradient: DoubleMatrix, - stepSize: Double, iter: Int, regParam: Double): (DoubleMatrix, Double) = { - val thisIterStepSize = stepSize / math.sqrt(iter) - val step = gradient.mul(thisIterStepSize) + override def compute( + weightsOld: Vector, + gradient: Vector, + stepSize: Double, + iter: Int, + regParam: Double): (Vector, Double) = { // add up both updates from the gradient of the loss (= step) as well as // the gradient of the regularizer (= regParam * weightsOld) // w' = w - thisIterStepSize * (gradient + regParam * w) // w' = (1 - thisIterStepSize * regParam) * w - thisIterStepSize * gradient - val newWeights = weightsOld.mul(1.0 - thisIterStepSize * regParam).sub(step) - (newWeights, 0.5 * pow(newWeights.norm2, 2.0) * regParam) + val thisIterStepSize = stepSize / math.sqrt(iter) + val brzWeights: BV[Double] = weightsOld.toBreeze.toDenseVector + brzWeights :*= (1.0 - thisIterStepSize * regParam) + brzAxpy(-thisIterStepSize, gradient.toBreeze, brzWeights) + val norm = brzNorm(brzWeights, 2.0) + + (Vectors.fromBreeze(brzWeights), 0.5 * regParam * norm * norm) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 3e1ed91bf6729..80dc0f12ff84f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -17,11 +17,12 @@ package org.apache.spark.mllib.regression +import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} + import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ - -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * GeneralizedLinearModel (GLM) represents a model trained using @@ -31,12 +32,9 @@ import org.jblas.DoubleMatrix * @param weights Weights computed for every feature. * @param intercept Intercept computed for this model. */ -abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: Double) +abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double) extends Serializable { - // Create a column vector that can be used for predictions - private val weightsMatrix = new DoubleMatrix(weights.length, 1, weights:_*) - /** * Predict the result given a data point and the weights learned. * @@ -44,8 +42,7 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: * @param weightMatrix Column vector containing the weights of the model * @param intercept Intercept of the model. */ - def predictPoint(dataMatrix: DoubleMatrix, weightMatrix: DoubleMatrix, - intercept: Double): Double + protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double): Double /** * Predict values for the given data set using the model trained. @@ -53,16 +50,13 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: * @param testData RDD representing data points to be predicted * @return RDD[Double] where each entry contains the corresponding prediction */ - def predict(testData: RDD[Array[Double]]): RDD[Double] = { + def predict(testData: RDD[Vector]): RDD[Double] = { // A small optimization to avoid serializing the entire model. Only the weightsMatrix // and intercept is needed. - val localWeights = weightsMatrix + val localWeights = weights val localIntercept = intercept - testData.map { x => - val dataMatrix = new DoubleMatrix(1, x.length, x:_*) - predictPoint(dataMatrix, localWeights, localIntercept) - } + testData.map(v => predictPoint(v, localWeights, localIntercept)) } /** @@ -71,14 +65,13 @@ abstract class GeneralizedLinearModel(val weights: Array[Double], val intercept: * @param testData array representing a single data point * @return Double prediction from the trained model */ - def predict(testData: Array[Double]): Double = { - val dataMat = new DoubleMatrix(1, testData.length, testData:_*) - predictPoint(dataMat, weightsMatrix, intercept) + def predict(testData: Vector): Double = { + predictPoint(testData, weights, intercept) } } /** - * GeneralizedLinearAlgorithm implements methods to train a Genearalized Linear Model (GLM). + * GeneralizedLinearAlgorithm implements methods to train a Generalized Linear Model (GLM). * This class should be extended with an Optimizer to create a new GLM. */ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] @@ -88,6 +81,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] val optimizer: Optimizer + /** Whether to add intercept (default: true). */ protected var addIntercept: Boolean = true protected var validateData: Boolean = true @@ -95,7 +89,7 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * Create a model given the weights and intercept */ - protected def createModel(weights: Array[Double], intercept: Double): M + protected def createModel(weights: Vector, intercept: Double): M /** * Set if the algorithm should add an intercept. Default true. @@ -117,17 +111,27 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] * Run the algorithm with the configured parameters on an input * RDD of LabeledPoint entries. */ - def run(input: RDD[LabeledPoint]) : M = { - val nfeatures: Int = input.first().features.length - val initialWeights = new Array[Double](nfeatures) + def run(input: RDD[LabeledPoint]): M = { + val numFeatures: Int = input.first().features.size + val initialWeights = Vectors.dense(new Array[Double](numFeatures)) run(input, initialWeights) } + /** Prepends one to the input vector. */ + private def prependOne(vector: Vector): Vector = { + val vector1 = vector.toBreeze match { + case dv: BDV[Double] => BDV.vertcat(BDV.ones[Double](1), dv) + case sv: BSV[Double] => BSV.vertcat(new BSV[Double](Array(0), Array(1.0), 1), sv) + case v: Any => throw new IllegalArgumentException("Do not support vector type " + v.getClass) + } + Vectors.fromBreeze(vector1) + } + /** * Run the algorithm with the configured parameters on an input RDD * of LabeledPoint entries starting from the initial weights provided. */ - def run(input: RDD[LabeledPoint], initialWeights: Array[Double]) : M = { + def run(input: RDD[LabeledPoint], initialWeights: Vector): M = { // Check the data properties before running the optimizer if (validateData && !validators.forall(func => func(input))) { @@ -136,27 +140,26 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] // Prepend an extra variable consisting of all 1.0's for the intercept. val data = if (addIntercept) { - input.map(labeledPoint => (labeledPoint.label, 1.0 +: labeledPoint.features)) + input.map(labeledPoint => (labeledPoint.label, prependOne(labeledPoint.features))) } else { input.map(labeledPoint => (labeledPoint.label, labeledPoint.features)) } val initialWeightsWithIntercept = if (addIntercept) { - 0.0 +: initialWeights + prependOne(initialWeights) } else { initialWeights } val weightsWithIntercept = optimizer.optimize(data, initialWeightsWithIntercept) - val (intercept, weights) = if (addIntercept) { - (weightsWithIntercept(0), weightsWithIntercept.tail) - } else { - (0.0, weightsWithIntercept) - } - - logInfo("Final weights " + weights.mkString(",")) - logInfo("Final intercept " + intercept) + val intercept = if (addIntercept) weightsWithIntercept(0) else 0.0 + val weights = + if (addIntercept) { + Vectors.dense(weightsWithIntercept.toArray.slice(1, weightsWithIntercept.size)) + } else { + weightsWithIntercept + } createModel(weights, intercept) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala index 1a18292fe3f3b..3deab1ab785b9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LabeledPoint.scala @@ -17,14 +17,16 @@ package org.apache.spark.mllib.regression +import org.apache.spark.mllib.linalg.Vector + /** * Class that represents the features and labels of a data point. * * @param label Label for this data point. * @param features List of features for this data point. */ -case class LabeledPoint(label: Double, features: Array[Double]) { +case class LabeledPoint(label: Double, features: Vector) { override def toString: String = { - "LabeledPoint(%s, %s)".format(label, features.mkString("[", ", ", "]")) + "LabeledPoint(%s, %s)".format(label, features) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index be63ce8538fef..25920d0dc976e 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -17,12 +17,11 @@ package org.apache.spark.mllib.regression -import org.apache.spark.{Logging, SparkContext} -import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix +import org.apache.spark.rdd.RDD /** * Regression model trained using Lasso. @@ -31,16 +30,16 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class LassoModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { - override def predictPoint( - dataMatrix: DoubleMatrix, - weightMatrix: DoubleMatrix, + override protected def predictPoint( + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double): Double = { - dataMatrix.dot(weightMatrix) + intercept + weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } } @@ -57,8 +56,7 @@ class LassoWithSGD private ( var numIterations: Int, var regParam: Double, var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[LassoModel] - with Serializable { + extends GeneralizedLinearAlgorithm[LassoModel] with Serializable { val gradient = new LeastSquaresGradient() val updater = new L1Updater() @@ -70,10 +68,6 @@ class LassoWithSGD private ( // We don't want to penalize the intercept, so set this to false. super.setIntercept(false) - var yMean = 0.0 - var xColMean: DoubleMatrix = _ - var xColSd: DoubleMatrix = _ - /** * Construct a Lasso object with default parameters */ @@ -85,36 +79,8 @@ class LassoWithSGD private ( this } - override def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) - val weightsScaled = weightsMat.div(xColSd) - val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) - - new LassoModel(weightsScaled.data, interceptScaled) - } - - override def run( - input: RDD[LabeledPoint], - initialWeights: Array[Double]) - : LassoModel = - { - val nfeatures: Int = input.first.features.length - val nexamples: Long = input.count() - - // To avoid penalizing the intercept, we center and scale the data. - val stats = MLUtils.computeStats(input, nfeatures, nexamples) - yMean = stats._1 - xColMean = stats._2 - xColSd = stats._3 - - val normalizedData = input.map { point => - val yNormalized = point.label - yMean - val featuresMat = new DoubleMatrix(nfeatures, 1, point.features:_*) - val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) - LabeledPoint(yNormalized, featuresNormalized.toArray) - } - - super.run(normalizedData, initialWeights) + override protected def createModel(weights: Vector, intercept: Double) = { + new LassoModel(weights, intercept) } } @@ -144,11 +110,9 @@ object LassoWithSGD { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) - : LassoModel = - { - new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input, - initialWeights) + initialWeights: Vector): LassoModel = { + new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction) + .run(input, initialWeights) } /** @@ -168,9 +132,7 @@ object LassoWithSGD { numIterations: Int, stepSize: Double, regParam: Double, - miniBatchFraction: Double) - : LassoModel = - { + miniBatchFraction: Double): LassoModel = { new LassoWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) } @@ -190,9 +152,7 @@ object LassoWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - regParam: Double) - : LassoModel = - { + regParam: Double): LassoModel = { train(input, numIterations, stepSize, regParam, 1.0) } @@ -208,9 +168,7 @@ object LassoWithSGD { */ def train( input: RDD[LabeledPoint], - numIterations: Int) - : LassoModel = - { + numIterations: Int): LassoModel = { train(input, numIterations, 1.0, 1.0, 1.0) } @@ -222,7 +180,8 @@ object LassoWithSGD { val sc = new SparkContext(args(0), "Lasso") val data = MLUtils.loadLabeledData(sc, args(1)) val model = LassoWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index f5f15d1a33f4d..9ed927994e795 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -19,11 +19,10 @@ package org.apache.spark.mllib.regression import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils -import org.jblas.DoubleMatrix - /** * Regression model trained using LinearRegression. * @@ -31,15 +30,15 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class LinearRegressionModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { - override def predictPoint( - dataMatrix: DoubleMatrix, - weightMatrix: DoubleMatrix, + override protected def predictPoint( + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double): Double = { - dataMatrix.dot(weightMatrix) + intercept + weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } } @@ -69,7 +68,7 @@ class LinearRegressionWithSGD private ( */ def this() = this(1.0, 100, 1.0) - override def createModel(weights: Array[Double], intercept: Double) = { + override protected def createModel(weights: Vector, intercept: Double) = { new LinearRegressionModel(weights, intercept) } } @@ -98,11 +97,9 @@ object LinearRegressionWithSGD { numIterations: Int, stepSize: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) - : LinearRegressionModel = - { - new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input, - initialWeights) + initialWeights: Vector): LinearRegressionModel = { + new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction) + .run(input, initialWeights) } /** @@ -120,9 +117,7 @@ object LinearRegressionWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - miniBatchFraction: Double) - : LinearRegressionModel = - { + miniBatchFraction: Double): LinearRegressionModel = { new LinearRegressionWithSGD(stepSize, numIterations, miniBatchFraction).run(input) } @@ -140,9 +135,7 @@ object LinearRegressionWithSGD { def train( input: RDD[LabeledPoint], numIterations: Int, - stepSize: Double) - : LinearRegressionModel = - { + stepSize: Double): LinearRegressionModel = { train(input, numIterations, stepSize, 1.0) } @@ -158,9 +151,7 @@ object LinearRegressionWithSGD { */ def train( input: RDD[LabeledPoint], - numIterations: Int) - : LinearRegressionModel = - { + numIterations: Int): LinearRegressionModel = { train(input, numIterations, 1.0, 1.0) } @@ -172,7 +163,7 @@ object LinearRegressionWithSGD { val sc = new SparkContext(args(0), "LinearRegression") val data = MLUtils.loadLabeledData(sc, args(1)) val model = LinearRegressionWithSGD.train(data, args(3).toInt, args(2).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index 423afc32d665c..5e4b8a345b1c5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -18,6 +18,7 @@ package org.apache.spark.mllib.regression import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector trait RegressionModel extends Serializable { /** @@ -26,7 +27,7 @@ trait RegressionModel extends Serializable { * @param testData RDD representing data points to be predicted * @return RDD[Double] where each entry contains the corresponding prediction */ - def predict(testData: RDD[Array[Double]]): RDD[Double] + def predict(testData: RDD[Vector]): RDD[Double] /** * Predict values for a single data point using the model trained. @@ -34,5 +35,5 @@ trait RegressionModel extends Serializable { * @param testData array representing a single data point * @return Double prediction from the trained model */ - def predict(testData: Array[Double]): Double + def predict(testData: Vector): Double } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index feb100f21888f..1f17d2107f940 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -21,8 +21,7 @@ import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.util.MLUtils - -import org.jblas.DoubleMatrix +import org.apache.spark.mllib.linalg.Vector /** * Regression model trained using RidgeRegression. @@ -31,16 +30,16 @@ import org.jblas.DoubleMatrix * @param intercept Intercept computed for this model. */ class RidgeRegressionModel( - override val weights: Array[Double], + override val weights: Vector, override val intercept: Double) extends GeneralizedLinearModel(weights, intercept) with RegressionModel with Serializable { - override def predictPoint( - dataMatrix: DoubleMatrix, - weightMatrix: DoubleMatrix, + override protected def predictPoint( + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double): Double = { - dataMatrix.dot(weightMatrix) + intercept + weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept } } @@ -57,8 +56,7 @@ class RidgeRegressionWithSGD private ( var numIterations: Int, var regParam: Double, var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[RidgeRegressionModel] - with Serializable { + extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable { val gradient = new LeastSquaresGradient() val updater = new SquaredL2Updater() @@ -71,10 +69,6 @@ class RidgeRegressionWithSGD private ( // We don't want to penalize the intercept in RidgeRegression, so set this to false. super.setIntercept(false) - var yMean = 0.0 - var xColMean: DoubleMatrix = _ - var xColSd: DoubleMatrix = _ - /** * Construct a RidgeRegression object with default parameters */ @@ -86,36 +80,8 @@ class RidgeRegressionWithSGD private ( this } - override def createModel(weights: Array[Double], intercept: Double) = { - val weightsMat = new DoubleMatrix(weights.length, 1, weights: _*) - val weightsScaled = weightsMat.div(xColSd) - val interceptScaled = yMean - weightsMat.transpose().mmul(xColMean.div(xColSd)).get(0) - - new RidgeRegressionModel(weightsScaled.data, interceptScaled) - } - - override def run( - input: RDD[LabeledPoint], - initialWeights: Array[Double]) - : RidgeRegressionModel = - { - val nfeatures: Int = input.first().features.length - val nexamples: Long = input.count() - - // To avoid penalizing the intercept, we center and scale the data. - val stats = MLUtils.computeStats(input, nfeatures, nexamples) - yMean = stats._1 - xColMean = stats._2 - xColSd = stats._3 - - val normalizedData = input.map { point => - val yNormalized = point.label - yMean - val featuresMat = new DoubleMatrix(nfeatures, 1, point.features:_*) - val featuresNormalized = featuresMat.sub(xColMean).divi(xColSd) - LabeledPoint(yNormalized, featuresNormalized.toArray) - } - - super.run(normalizedData, initialWeights) + override protected def createModel(weights: Vector, intercept: Double) = { + new RidgeRegressionModel(weights, intercept) } } @@ -144,9 +110,7 @@ object RidgeRegressionWithSGD { stepSize: Double, regParam: Double, miniBatchFraction: Double, - initialWeights: Array[Double]) - : RidgeRegressionModel = - { + initialWeights: Vector): RidgeRegressionModel = { new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run( input, initialWeights) } @@ -167,9 +131,7 @@ object RidgeRegressionWithSGD { numIterations: Int, stepSize: Double, regParam: Double, - miniBatchFraction: Double) - : RidgeRegressionModel = - { + miniBatchFraction: Double): RidgeRegressionModel = { new RidgeRegressionWithSGD(stepSize, numIterations, regParam, miniBatchFraction).run(input) } @@ -188,9 +150,7 @@ object RidgeRegressionWithSGD { input: RDD[LabeledPoint], numIterations: Int, stepSize: Double, - regParam: Double) - : RidgeRegressionModel = - { + regParam: Double): RidgeRegressionModel = { train(input, numIterations, stepSize, regParam, 1.0) } @@ -205,23 +165,22 @@ object RidgeRegressionWithSGD { */ def train( input: RDD[LabeledPoint], - numIterations: Int) - : RidgeRegressionModel = - { + numIterations: Int): RidgeRegressionModel = { train(input, numIterations, 1.0, 1.0, 1.0) } def main(args: Array[String]) { if (args.length != 5) { - println("Usage: RidgeRegression " + - " ") + println("Usage: RidgeRegression " + + " ") System.exit(1) } val sc = new SparkContext(args(0), "RidgeRegression") val data = MLUtils.loadLabeledData(sc, args(1)) val model = RidgeRegressionWithSGD.train(data, args(4).toInt, args(2).toDouble, args(3).toDouble) - println("Weights: " + model.weights.mkString("[", ", ", "]")) + + println("Weights: " + model.weights) println("Intercept: " + model.intercept) sc.stop() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 33205b919db8f..dee9594a9dd79 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -30,6 +30,7 @@ import org.apache.spark.mllib.tree.impurity.{Entropy, Gini, Impurity, Variance} import org.apache.spark.mllib.tree.model._ import org.apache.spark.rdd.RDD import org.apache.spark.util.random.XORShiftRandom +import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * A class that implements a decision tree algorithm for classification and regression. It @@ -295,7 +296,7 @@ object DecisionTree extends Serializable with Logging { val numNodes = scala.math.pow(2, level).toInt logDebug("numNodes = " + numNodes) // Find the number of features by looking at the first sample. - val numFeatures = input.first().features.length + val numFeatures = input.first().features.size logDebug("numFeatures = " + numFeatures) val numBins = bins(0).length logDebug("numBins = " + numBins) @@ -902,7 +903,7 @@ object DecisionTree extends Serializable with Logging { val count = input.count() // Find the number of features by looking at the first sample - val numFeatures = input.take(1)(0).features.length + val numFeatures = input.take(1)(0).features.size val maxBins = strategy.maxBins val numBins = if (maxBins <= count) maxBins else count.toInt @@ -1116,7 +1117,7 @@ object DecisionTree extends Serializable with Logging { sc.textFile(dir).map { line => val parts = line.trim().split(",") val label = parts(0).toDouble - val features = parts.slice(1,parts.length).map(_.toDouble) + val features = Vectors.dense(parts.slice(1,parts.length).map(_.toDouble)) LabeledPoint(label, features) } } @@ -1127,7 +1128,7 @@ object DecisionTree extends Serializable with Logging { */ private def accuracyScore(model: DecisionTreeModel, data: RDD[LabeledPoint], threshold: Double = 0.5): Double = { - def predictedValue(features: Array[Double]) = { + def predictedValue(features: Vector) = { if (model.predict(features) < threshold) 0.0 else 1.0 } val correctCount = data.filter(y => predictedValue(y.features) == y.label).count() diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index a8bbf21daec01..a6dca84a2ce09 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vector /** * Model to store the decision tree parameters @@ -33,7 +34,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * @param features array representing a single data point * @return Double prediction from the trained model */ - def predict(features: Array[Double]): Double = { + def predict(features: Vector): Double = { topNode.predictIfLeaf(features) } @@ -43,7 +44,7 @@ class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable * @param features RDD representing data points to be predicted * @return RDD[Int] where each entry contains the corresponding prediction */ - def predict(features: RDD[Array[Double]]): RDD[Double] = { + def predict(features: RDD[Vector]): RDD[Double] = { features.map(x => predict(x)) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index ea4693c5c2f4e..aac3f9ce308f7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.tree.model import org.apache.spark.Logging import org.apache.spark.mllib.tree.configuration.FeatureType._ +import org.apache.spark.mllib.linalg.Vector /** * Node in a decision tree @@ -54,8 +55,8 @@ class Node ( logDebug("stats = " + stats) logDebug("predict = " + predict) if (!isLeaf) { - val leftNodeIndex = id*2 + 1 - val rightNodeIndex = id*2 + 2 + val leftNodeIndex = id * 2 + 1 + val rightNodeIndex = id * 2 + 2 leftNode = Some(nodes(leftNodeIndex)) rightNode = Some(nodes(rightNodeIndex)) leftNode.get.build(nodes) @@ -68,7 +69,7 @@ class Node ( * @param feature feature value * @return predicted value */ - def predictIfLeaf(feature: Array[Double]) : Double = { + def predictIfLeaf(feature: Vector) : Double = { if (isLeaf) { predict } else{ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 2e03684e62861..81e4eda2a68c4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -24,6 +24,7 @@ import org.jblas.DoubleMatrix import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** @@ -74,7 +75,7 @@ object LinearDataGenerator { val y = x.map { xi => new DoubleMatrix(1, xi.length, xi: _*).dot(weightsMat) + intercept + eps * rnd.nextGaussian() } - y.zip(x).map(p => LabeledPoint(p._1, p._2)) + y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) } /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index 52c4a71d621a1..61498dcc2be00 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -22,6 +22,7 @@ import scala.util.Random import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors /** * Generate test data for LogisticRegression. This class chooses positive labels @@ -54,7 +55,7 @@ object LogisticRegressionDataGenerator { val x = Array.fill[Double](nfeatures) { rnd.nextGaussian() + (y * eps) } - LabeledPoint(y, x) + LabeledPoint(y, Vectors.dense(x)) } data } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 08cd9ab05547b..cb85e433bfc73 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -17,15 +17,13 @@ package org.apache.spark.mllib.util +import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, + squaredDistance => breezeSquaredDistance} + import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD -import org.apache.spark.SparkContext._ - -import org.jblas.DoubleMatrix - import org.apache.spark.mllib.regression.LabeledPoint - -import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} +import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * Helper methods to load, save and pre-process data used in ML Lib. @@ -40,6 +38,107 @@ object MLUtils { eps } + /** + * Multiclass label parser, which parses a string into double. + */ + val multiclassLabelParser: String => Double = _.toDouble + + /** + * Binary label parser, which outputs 1.0 (positive) if the value is greater than 0.5, + * or 0.0 (negative) otherwise. + */ + val binaryLabelParser: String => Double = label => if (label.toDouble > 0.5) 1.0 else 0.0 + + /** + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint]. + * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. + * Each line represents a labeled sparse feature vector using the following format: + * {{{label index1:value1 index2:value2 ...}}} + * where the indices are one-based and in ascending order. + * This method parses each line into a [[org.apache.spark.mllib.regression.LabeledPoint]], + * where the feature indices are converted to zero-based. + * + * @param sc Spark context + * @param path file or directory path in any Hadoop-supported file system URI + * @param labelParser parser for labels, default: 1.0 if label > 0.5 or 0.0 otherwise + * @param numFeatures number of features, which will be determined from the input data if a + * negative value is given. The default value is -1. + * @param minSplits min number of partitions, default: sc.defaultMinSplits + * @return labeled data stored as an RDD[LabeledPoint] + */ + def loadLibSVMData( + sc: SparkContext, + path: String, + labelParser: String => Double, + numFeatures: Int, + minSplits: Int): RDD[LabeledPoint] = { + val parsed = sc.textFile(path, minSplits) + .map(_.trim) + .filter(!_.isEmpty) + .map(_.split(' ')) + // Determine number of features. + val d = if (numFeatures >= 0) { + numFeatures + } else { + parsed.map { items => + if (items.length > 1) { + items.last.split(':')(0).toInt + } else { + 0 + } + }.reduce(math.max) + } + parsed.map { items => + val label = labelParser(items.head) + val (indices, values) = items.tail.map { item => + val indexAndValue = item.split(':') + val index = indexAndValue(0).toInt - 1 + val value = indexAndValue(1).toDouble + (index, value) + }.unzip + LabeledPoint(label, Vectors.sparse(d, indices.toArray, values.toArray)) + } + } + + // Convenient methods for calling from Java. + + /** + * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], + * with number of features determined automatically and the default number of partitions. + */ + def loadLibSVMData(sc: SparkContext, path: String): RDD[LabeledPoint] = + loadLibSVMData(sc, path, binaryLabelParser, -1, sc.defaultMinSplits) + + /** + * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], + * with number of features specified explicitly and the default number of partitions. + */ + def loadLibSVMData(sc: SparkContext, path: String, numFeatures: Int): RDD[LabeledPoint] = + loadLibSVMData(sc, path, binaryLabelParser, numFeatures, sc.defaultMinSplits) + + /** + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], + * with the given label parser, number of features determined automatically, + * and the default number of partitions. + */ + def loadLibSVMData( + sc: SparkContext, + path: String, + labelParser: String => Double): RDD[LabeledPoint] = + loadLibSVMData(sc, path, labelParser, -1, sc.defaultMinSplits) + + /** + * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], + * with the given label parser, number of features specified explicitly, + * and the default number of partitions. + */ + def loadLibSVMData( + sc: SparkContext, + path: String, + labelParser: String => Double, + numFeatures: Int): RDD[LabeledPoint] = + loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinSplits) + /** * Load labeled data from a file. The data format used here is * , ... @@ -54,7 +153,7 @@ object MLUtils { sc.textFile(dir).map { line => val parts = line.split(',') val label = parts(0).toDouble - val features = parts(1).trim().split(' ').map(_.toDouble) + val features = Vectors.dense(parts(1).trim().split(' ').map(_.toDouble)) LabeledPoint(label, features) } } @@ -68,7 +167,7 @@ object MLUtils { * @param dir Directory to save the data. */ def saveLabeledData(data: RDD[LabeledPoint], dir: String) { - val dataStr = data.map(x => x.label + "," + x.features.mkString(" ")) + val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" ")) dataStr.saveAsTextFile(dir) } @@ -76,44 +175,52 @@ object MLUtils { * Utility function to compute mean and standard deviation on a given dataset. * * @param data - input data set whose statistics are computed - * @param nfeatures - number of features - * @param nexamples - number of examples in input dataset + * @param numFeatures - number of features + * @param numExamples - number of examples in input dataset * * @return (yMean, xColMean, xColSd) - Tuple consisting of * yMean - mean of the labels * xColMean - Row vector with mean for every column (or feature) of the input data * xColSd - Row vector standard deviation for every column (or feature) of the input data. */ - def computeStats(data: RDD[LabeledPoint], nfeatures: Int, nexamples: Long): - (Double, DoubleMatrix, DoubleMatrix) = { - val yMean: Double = data.map { labeledPoint => labeledPoint.label }.reduce(_ + _) / nexamples - - // NOTE: We shuffle X by column here to compute column sum and sum of squares. - val xColSumSq: RDD[(Int, (Double, Double))] = data.flatMap { labeledPoint => - val nCols = labeledPoint.features.length - // Traverse over every column and emit (col, value, value^2) - Iterator.tabulate(nCols) { i => - (i, (labeledPoint.features(i), labeledPoint.features(i)*labeledPoint.features(i))) - } - }.reduceByKey { case(x1, x2) => - (x1._1 + x2._1, x1._2 + x2._2) + def computeStats( + data: RDD[LabeledPoint], + numFeatures: Int, + numExamples: Long): (Double, Vector, Vector) = { + val brzData = data.map { case LabeledPoint(label, features) => + (label, features.toBreeze) } - val xColSumsMap = xColSumSq.collectAsMap() - - val xColMean = DoubleMatrix.zeros(nfeatures, 1) - val xColSd = DoubleMatrix.zeros(nfeatures, 1) - - // Compute mean and unbiased variance using column sums - var col = 0 - while (col < nfeatures) { - xColMean.put(col, xColSumsMap(col)._1 / nexamples) - val variance = - (xColSumsMap(col)._2 - (math.pow(xColSumsMap(col)._1, 2) / nexamples)) / nexamples - xColSd.put(col, math.sqrt(variance)) - col += 1 + val aggStats = brzData.aggregate( + (0L, 0.0, BDV.zeros[Double](numFeatures), BDV.zeros[Double](numFeatures)) + )( + seqOp = (c, v) => (c, v) match { + case ((n, sumLabel, sum, sumSq), (label, features)) => + features.activeIterator.foreach { case (i, x) => + sumSq(i) += x * x + } + (n + 1L, sumLabel + label, sum += features, sumSq) + }, + combOp = (c1, c2) => (c1, c2) match { + case ((n1, sumLabel1, sum1, sumSq1), (n2, sumLabel2, sum2, sumSq2)) => + (n1 + n2, sumLabel1 + sumLabel2, sum1 += sum2, sumSq1 += sumSq2) + } + ) + val (nl, sumLabel, sum, sumSq) = aggStats + + require(nl > 0, "Input data is empty.") + require(nl == numExamples) + + val n = nl.toDouble + val yMean = sumLabel / n + val mean = sum / n + val std = new Array[Double](sum.length) + var i = 0 + while (i < numFeatures) { + std(i) = sumSq(i) / n - mean(i) * mean(i) + i += 1 } - (yMean, xColMean, xColSd) + (yMean, Vectors.fromBreeze(mean), Vectors.dense(std)) } /** @@ -144,6 +251,18 @@ object MLUtils { val sumSquaredNorm = norm1 * norm1 + norm2 * norm2 val normDiff = norm1 - norm2 var sqDist = 0.0 + /* + * The relative error is + *
    +     * EPSILON * ( \|a\|_2^2 + \|b\\_2^2 + 2 |a^T b|) / ( \|a - b\|_2^2 ),
    +     * 
    + * which is bounded by + *
    +     * 2.0 * EPSILON * ( \|a\|_2^2 + \|b\|_2^2 ) / ( (\|a\|_2 - \|b\|_2)^2 ).
    +     * 
    + * The bound doesn't need the inner product, so we can use it as a sufficient condition to + * check quickly whether the inner product approach is accurate. + */ val precisionBound1 = 2.0 * EPSILON * sumSquaredNorm / (normDiff * normDiff + EPSILON) if (precisionBound1 < precision) { sqDist = sumSquaredNorm - 2.0 * v1.dot(v2) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index c96c94f70eef7..e300c3dbe1fe0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -23,6 +23,7 @@ import org.jblas.DoubleMatrix import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** @@ -58,7 +59,7 @@ object SVMDataGenerator { } val yD = new DoubleMatrix(1, x.length, x: _*).dot(trueWeights) + rnd.nextGaussian() * 0.1 val y = if (yD < 0) 0.0 else 1.0 - LabeledPoint(y, x) + LabeledPoint(y, Vectors.dense(x)) } MLUtils.saveLabeledData(data, outputPath) diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java index 073ded6f36933..c80b1134ed1b2 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaNaiveBayesSuite.java @@ -19,6 +19,7 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.mllib.linalg.Vectors; import org.apache.spark.mllib.regression.LabeledPoint; import org.junit.After; import org.junit.Assert; @@ -45,12 +46,12 @@ public void tearDown() { } private static final List POINTS = Arrays.asList( - new LabeledPoint(0, new double[] {1.0, 0.0, 0.0}), - new LabeledPoint(0, new double[] {2.0, 0.0, 0.0}), - new LabeledPoint(1, new double[] {0.0, 1.0, 0.0}), - new LabeledPoint(1, new double[] {0.0, 2.0, 0.0}), - new LabeledPoint(2, new double[] {0.0, 0.0, 1.0}), - new LabeledPoint(2, new double[] {0.0, 0.0, 2.0}) + new LabeledPoint(0, Vectors.dense(1.0, 0.0, 0.0)), + new LabeledPoint(0, Vectors.dense(2.0, 0.0, 0.0)), + new LabeledPoint(1, Vectors.dense(0.0, 1.0, 0.0)), + new LabeledPoint(1, Vectors.dense(0.0, 2.0, 0.0)), + new LabeledPoint(2, Vectors.dense(0.0, 0.0, 1.0)), + new LabeledPoint(2, Vectors.dense(0.0, 0.0, 2.0)) ); private int validatePrediction(List points, NaiveBayesModel model) { diff --git a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java index 117e5eaa8b78e..4701a5e545020 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/classification/JavaSVMSuite.java @@ -17,7 +17,6 @@ package org.apache.spark.mllib.classification; - import java.io.Serializable; import java.util.List; @@ -28,7 +27,6 @@ import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; - import org.apache.spark.mllib.regression.LabeledPoint; public class JavaSVMSuite implements Serializable { @@ -94,5 +92,4 @@ public void runSVMUsingStaticMethods() { int numAccurate = validatePrediction(validationData, model); Assert.assertTrue(numAccurate > nPoints * 4.0 / 5.0); } - } diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java index 2c4d795f96e4e..c6d8425ffc38d 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -19,10 +19,10 @@ import java.io.Serializable; -import com.google.common.collect.Lists; - import scala.Tuple2; +import com.google.common.collect.Lists; + import org.junit.Test; import static org.junit.Assert.*; @@ -36,7 +36,7 @@ public void denseArrayConstruction() { @Test public void sparseArrayConstruction() { - Vector v = Vectors.sparse(3, Lists.newArrayList( + Vector v = Vectors.sparse(3, Lists.>newArrayList( new Tuple2(0, 2.0), new Tuple2(2, 3.0))); assertArrayEquals(new double[]{2.0, 0.0, 3.0}, v.toArray(), 0.0); diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java index f44b25cd44d19..f725924a2d971 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaLassoSuite.java @@ -59,7 +59,7 @@ int validatePrediction(List validationData, LassoModel model) { @Test public void runLassoUsingConstructor() { int nPoints = 10000; - double A = 2.0; + double A = 0.0; double[] weights = {-1.5, 1.0e-2}; JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, @@ -80,7 +80,7 @@ public void runLassoUsingConstructor() { @Test public void runLassoUsingStaticMethods() { int nPoints = 10000; - double A = 2.0; + double A = 0.0; double[] weights = {-1.5, 1.0e-2}; JavaRDD testRDD = sc.parallelize(LinearDataGenerator.generateLinearInputAsList(A, diff --git a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java index 2fdd5fc8fdca6..03714ae7e4d00 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/regression/JavaRidgeRegressionSuite.java @@ -55,30 +55,27 @@ public void tearDown() { return errorSum / validationData.size(); } - List generateRidgeData(int numPoints, int nfeatures, double eps) { + List generateRidgeData(int numPoints, int numFeatures, double std) { org.jblas.util.Random.seed(42); // Pick weights as random values distributed uniformly in [-0.5, 0.5] - DoubleMatrix w = DoubleMatrix.rand(nfeatures, 1).subi(0.5); - // Set first two weights to eps - w.put(0, 0, eps); - w.put(1, 0, eps); - return LinearDataGenerator.generateLinearInputAsList(0.0, w.data, numPoints, 42, eps); + DoubleMatrix w = DoubleMatrix.rand(numFeatures, 1).subi(0.5); + return LinearDataGenerator.generateLinearInputAsList(0.0, w.data, numPoints, 42, std); } @Test public void runRidgeRegressionUsingConstructor() { - int nexamples = 200; - int nfeatures = 20; - double eps = 10.0; - List data = generateRidgeData(2*nexamples, nfeatures, eps); + int numExamples = 50; + int numFeatures = 20; + List data = generateRidgeData(2*numExamples, numFeatures, 10.0); - JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); - List validationData = data.subList(nexamples, 2*nexamples); + JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples)); + List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionWithSGD ridgeSGDImpl = new RidgeRegressionWithSGD(); - ridgeSGDImpl.optimizer().setStepSize(1.0) - .setRegParam(0.0) - .setNumIterations(200); + ridgeSGDImpl.optimizer() + .setStepSize(1.0) + .setRegParam(0.0) + .setNumIterations(200); RidgeRegressionModel model = ridgeSGDImpl.run(testRDD.rdd()); double unRegularizedErr = predictionError(validationData, model); @@ -91,13 +88,12 @@ public void runRidgeRegressionUsingConstructor() { @Test public void runRidgeRegressionUsingStaticMethods() { - int nexamples = 200; - int nfeatures = 20; - double eps = 10.0; - List data = generateRidgeData(2*nexamples, nfeatures, eps); + int numExamples = 50; + int numFeatures = 20; + List data = generateRidgeData(2 * numExamples, numFeatures, 10.0); - JavaRDD testRDD = sc.parallelize(data.subList(0, nexamples)); - List validationData = data.subList(nexamples, 2*nexamples); + JavaRDD testRDD = sc.parallelize(data.subList(0, numExamples)); + List validationData = data.subList(numExamples, 2 * numExamples); RidgeRegressionModel model = RidgeRegressionWithSGD.train(testRDD.rdd(), 200, 1.0, 0.0); double unRegularizedErr = predictionError(validationData, model); diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala index 05322b024d5f6..1e03c9df820b0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/LogisticRegressionSuite.scala @@ -20,11 +20,10 @@ package org.apache.spark.mllib.classification import scala.util.Random import scala.collection.JavaConversions._ -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.SparkContext +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.LocalSparkContext @@ -61,7 +60,7 @@ object LogisticRegressionSuite { if (yVal > 0) 1 else 0 } - val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i)))) + val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(Array(x1(i))))) testData } @@ -113,7 +112,7 @@ class LogisticRegressionSuite extends FunSuite with LocalSparkContext with Shoul val testData = LogisticRegressionSuite.generateLogisticInput(A, B, nPoints, 42) val initialB = -1.0 - val initialWeights = Array(initialB) + val initialWeights = Vectors.dense(initialB) val testRDD = sc.parallelize(testData, 2) testRDD.cache() diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala index 9dd6c79ee6ad8..516895d04222d 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/NaiveBayesSuite.scala @@ -19,9 +19,9 @@ package org.apache.spark.mllib.classification import scala.util.Random -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.LocalSparkContext @@ -54,7 +54,7 @@ object NaiveBayesSuite { if (rnd.nextDouble() < _theta(y)(j)) 1 else 0 } - LabeledPoint(y, xi) + LabeledPoint(y, Vectors.dense(xi)) } } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala index bc7abb568a172..dfacbfeee6fb4 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/classification/SVMSuite.scala @@ -20,7 +20,6 @@ package org.apache.spark.mllib.classification import scala.util.Random import scala.collection.JavaConversions._ -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.jblas.DoubleMatrix @@ -28,6 +27,7 @@ import org.jblas.DoubleMatrix import org.apache.spark.SparkException import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.Vectors object SVMSuite { @@ -54,7 +54,7 @@ object SVMSuite { intercept + 0.01 * rnd.nextGaussian() if (yD < 0) 0.0 else 1.0 } - y.zip(x).map(p => LabeledPoint(p._1, p._2)) + y.zip(x).map(p => LabeledPoint(p._1, Vectors.dense(p._2))) } } @@ -110,7 +110,7 @@ class SVMSuite extends FunSuite with LocalSparkContext { val initialB = -1.0 val initialC = -1.0 - val initialWeights = Array(initialB,initialC) + val initialWeights = Vectors.dense(initialB, initialC) val testRDD = sc.parallelize(testData, 2) testRDD.cache() @@ -150,10 +150,10 @@ class SVMSuite extends FunSuite with LocalSparkContext { } intercept[SparkException] { - val model = SVMWithSGD.train(testRDDInvalid, 100) + SVMWithSGD.train(testRDDInvalid, 100) } // Turning off data validation should not throw an exception - val noValidationModel = new SVMWithSGD().setValidateData(false).run(testRDDInvalid) + new SVMWithSGD().setValidateData(false).run(testRDDInvalid) } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala index 631d0e2ad9cdb..c4b433499a091 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/optimization/GradientDescentSuite.scala @@ -20,13 +20,12 @@ package org.apache.spark.mllib.optimization import scala.util.Random import scala.collection.JavaConversions._ -import org.scalatest.BeforeAndAfterAll import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.SparkContext import org.apache.spark.mllib.regression._ import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.Vectors object GradientDescentSuite { @@ -58,8 +57,7 @@ object GradientDescentSuite { if (yVal > 0) 1 else 0 } - val testData = (0 until nPoints).map(i => LabeledPoint(y(i), Array(x1(i)))) - testData + (0 until nPoints).map(i => LabeledPoint(y(i), Vectors.dense(x1(i)))) } } @@ -83,11 +81,11 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa // Add a extra variable consisting of all 1.0's for the intercept. val testData = GradientDescentSuite.generateGDInput(A, B, nPoints, 42) val data = testData.map { case LabeledPoint(label, features) => - label -> Array(1.0, features: _*) + label -> Vectors.dense(1.0, features.toArray: _*) } val dataRDD = sc.parallelize(data, 2).cache() - val initialWeightsWithIntercept = Array(1.0, initialWeights: _*) + val initialWeightsWithIntercept = Vectors.dense(1.0, initialWeights: _*) val (_, loss) = GradientDescent.runMiniBatchSGD( dataRDD, @@ -113,13 +111,13 @@ class GradientDescentSuite extends FunSuite with LocalSparkContext with ShouldMa // Add a extra variable consisting of all 1.0's for the intercept. val testData = GradientDescentSuite.generateGDInput(2.0, -1.5, 10000, 42) val data = testData.map { case LabeledPoint(label, features) => - label -> Array(1.0, features: _*) + label -> Vectors.dense(1.0, features.toArray: _*) } val dataRDD = sc.parallelize(data, 2).cache() // Prepare non-zero weights - val initialWeightsWithIntercept = Array(1.0, 0.5) + val initialWeightsWithIntercept = Vectors.dense(1.0, 0.5) val regParam0 = 0 val (newWeights0, loss0) = GradientDescent.runMiniBatchSGD( diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala index 2cebac943e15f..6aad9eb84e13c 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LassoSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.regression import org.scalatest.FunSuite +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} class LassoSuite extends FunSuite with LocalSparkContext { @@ -33,29 +34,33 @@ class LassoSuite extends FunSuite with LocalSparkContext { } test("Lasso local random SGD") { - val nPoints = 10000 + val nPoints = 1000 val A = 2.0 val B = -1.5 val C = 1.0e-2 - val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 42) - - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() + val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B, C), nPoints, 42) + .map { case LabeledPoint(label, features) => + LabeledPoint(label, Vectors.dense(1.0 +: features.toArray)) + } + val testRDD = sc.parallelize(testData, 2).cache() val ls = new LassoWithSGD() - ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(20) + ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40) val model = ls.run(testRDD) - val weight0 = model.weights(0) val weight1 = model.weights(1) - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + val weight2 = model.weights(2) + assert(weight0 >= 1.9 && weight0 <= 2.1, weight0 + " not in [1.9, 2.1]") + assert(weight1 >= -1.60 && weight1 <= -1.40, weight1 + " not in [-1.6, -1.4]") + assert(weight2 >= -1.0e-3 && weight2 <= 1.0e-3, weight2 + " not in [-0.001, 0.001]") val validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) + .map { case LabeledPoint(label, features) => + LabeledPoint(label, Vectors.dense(1.0 +: features.toArray)) + } val validationRDD = sc.parallelize(validationData, 2) // Test prediction on RDD. @@ -66,33 +71,39 @@ class LassoSuite extends FunSuite with LocalSparkContext { } test("Lasso local random SGD with initial weights") { - val nPoints = 10000 + val nPoints = 1000 val A = 2.0 val B = -1.5 val C = 1.0e-2 - val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 42) + val testData = LinearDataGenerator.generateLinearInput(A, Array[Double](B, C), nPoints, 42) + .map { case LabeledPoint(label, features) => + LabeledPoint(label, Vectors.dense(1.0 +: features.toArray)) + } + val initialA = -1.0 val initialB = -1.0 val initialC = -1.0 - val initialWeights = Array(initialB,initialC) + val initialWeights = Vectors.dense(initialA, initialB, initialC) - val testRDD = sc.parallelize(testData, 2) - testRDD.cache() + val testRDD = sc.parallelize(testData, 2).cache() val ls = new LassoWithSGD() - ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(20) + ls.optimizer.setStepSize(1.0).setRegParam(0.01).setNumIterations(40) val model = ls.run(testRDD, initialWeights) - val weight0 = model.weights(0) val weight1 = model.weights(1) - assert(model.intercept >= 1.9 && model.intercept <= 2.1, model.intercept + " not in [1.9, 2.1]") - assert(weight0 >= -1.60 && weight0 <= -1.40, weight0 + " not in [-1.6, -1.4]") - assert(weight1 >= -1.0e-3 && weight1 <= 1.0e-3, weight1 + " not in [-0.001, 0.001]") + val weight2 = model.weights(2) + assert(weight0 >= 1.9 && weight0 <= 2.1, weight0 + " not in [1.9, 2.1]") + assert(weight1 >= -1.60 && weight1 <= -1.40, weight1 + " not in [-1.6, -1.4]") + assert(weight2 >= -1.0e-3 && weight2 <= 1.0e-3, weight2 + " not in [-0.001, 0.001]") val validationData = LinearDataGenerator.generateLinearInput(A, Array[Double](B,C), nPoints, 17) + .map { case LabeledPoint(label, features) => + LabeledPoint(label, Vectors.dense(1.0 +: features.toArray)) + } val validationRDD = sc.parallelize(validationData,2) // Test prediction on RDD. diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala index 5d251bcbf35db..2f7d30708ce17 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/LinearRegressionSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.regression import org.scalatest.FunSuite +import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} class LinearRegressionSuite extends FunSuite with LocalSparkContext { @@ -40,11 +41,12 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { linReg.optimizer.setNumIterations(1000).setStepSize(1.0) val model = linReg.run(testRDD) - assert(model.intercept >= 2.5 && model.intercept <= 3.5) - assert(model.weights.length === 2) - assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) - assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + + val weights = model.weights + assert(weights.size === 2) + assert(weights(0) >= 9.0 && weights(0) <= 11.0) + assert(weights(1) >= 9.0 && weights(1) <= 11.0) val validationData = LinearDataGenerator.generateLinearInput( 3.0, Array(10.0, 10.0), 100, 17) @@ -67,9 +69,11 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { val model = linReg.run(testRDD) assert(model.intercept === 0.0) - assert(model.weights.length === 2) - assert(model.weights(0) >= 9.0 && model.weights(0) <= 11.0) - assert(model.weights(1) >= 9.0 && model.weights(1) <= 11.0) + + val weights = model.weights + assert(weights.size === 2) + assert(weights(0) >= 9.0 && weights(0) <= 11.0) + assert(weights(1) >= 9.0 && weights(1) <= 11.0) val validationData = LinearDataGenerator.generateLinearInput( 0.0, Array(10.0, 10.0), 100, 17) @@ -81,4 +85,40 @@ class LinearRegressionSuite extends FunSuite with LocalSparkContext { // Test prediction on Array. validatePrediction(validationData.map(row => model.predict(row.features)), validationData) } + + // Test if we can correctly learn Y = 10*X1 + 10*X10000 + test("sparse linear regression without intercept") { + val denseRDD = sc.parallelize( + LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 42), 2) + val sparseRDD = denseRDD.map { case LabeledPoint(label, v) => + val sv = Vectors.sparse(10000, Seq((0, v(0)), (9999, v(1)))) + LabeledPoint(label, sv) + }.cache() + val linReg = new LinearRegressionWithSGD().setIntercept(false) + linReg.optimizer.setNumIterations(1000).setStepSize(1.0) + + val model = linReg.run(sparseRDD) + + assert(model.intercept === 0.0) + + val weights = model.weights + assert(weights.size === 10000) + assert(weights(0) >= 9.0 && weights(0) <= 11.0) + assert(weights(9999) >= 9.0 && weights(9999) <= 11.0) + + val validationData = LinearDataGenerator.generateLinearInput(0.0, Array(10.0, 10.0), 100, 17) + val sparseValidationData = validationData.map { case LabeledPoint(label, v) => + val sv = Vectors.sparse(10000, Seq((0, v(0)), (9999, v(1)))) + LabeledPoint(label, sv) + } + val sparseValidationRDD = sc.parallelize(sparseValidationData, 2) + + // Test prediction on RDD. + validatePrediction( + model.predict(sparseValidationRDD.map(_.features)).collect(), sparseValidationData) + + // Test prediction on Array. + validatePrediction( + sparseValidationData.map(row => model.predict(row.features)), sparseValidationData) + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala index b2044ed0d8066..f66fc6ea6c1ec 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/regression/RidgeRegressionSuite.scala @@ -17,9 +17,10 @@ package org.apache.spark.mllib.regression -import org.jblas.DoubleMatrix import org.scalatest.FunSuite +import org.jblas.DoubleMatrix + import org.apache.spark.mllib.util.{LinearDataGenerator, LocalSparkContext} class RidgeRegressionSuite extends FunSuite with LocalSparkContext { @@ -30,22 +31,22 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { }.reduceLeft(_ + _) / predictions.size } - test("regularization with skewed weights") { - val nexamples = 200 - val nfeatures = 20 - val eps = 10 + test("ridge regression can help avoid overfitting") { + + // For small number of examples and large variance of error distribution, + // ridge regression should give smaller generalization error that linear regression. + + val numExamples = 50 + val numFeatures = 20 org.jblas.util.Random.seed(42) // Pick weights as random values distributed uniformly in [-0.5, 0.5] - val w = DoubleMatrix.rand(nfeatures, 1).subi(0.5) - // Set first two weights to eps - w.put(0, 0, eps) - w.put(1, 0, eps) + val w = DoubleMatrix.rand(numFeatures, 1).subi(0.5) // Use half of data for training and other half for validation - val data = LinearDataGenerator.generateLinearInput(3.0, w.toArray, 2*nexamples, 42, eps) - val testData = data.take(nexamples) - val validationData = data.takeRight(nexamples) + val data = LinearDataGenerator.generateLinearInput(3.0, w.toArray, 2 * numExamples, 42, 10.0) + val testData = data.take(numExamples) + val validationData = data.takeRight(numExamples) val testRDD = sc.parallelize(testData, 2).cache() val validationRDD = sc.parallelize(validationData, 2).cache() @@ -67,7 +68,7 @@ class RidgeRegressionSuite extends FunSuite with LocalSparkContext { val ridgeErr = predictionError( ridgeModel.predict(validationRDD.map(_.features)).collect(), validationData) - // Ridge CV-error should be lower than linear regression + // Ridge validation error should be lower than linear regression. assert(ridgeErr < linearErr, "ridgeError (" + ridgeErr + ") was not less than linearError(" + linearErr + ")") } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala index 4349c7000a0ae..350130c914f26 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/tree/DecisionTreeSuite.scala @@ -27,6 +27,7 @@ import org.apache.spark.mllib.tree.model.Filter import org.apache.spark.mllib.tree.configuration.Strategy import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.FeatureType._ +import org.apache.spark.mllib.linalg.Vectors class DecisionTreeSuite extends FunSuite with BeforeAndAfterAll { @@ -396,7 +397,7 @@ object DecisionTreeSuite { def generateOrderedLabeledPointsWithLabel0(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000){ - val lp = new LabeledPoint(0.0,Array(i.toDouble,1000.0-i)) + val lp = new LabeledPoint(0.0, Vectors.dense(i.toDouble, 1000.0 - i)) arr(i) = lp } arr @@ -405,7 +406,7 @@ object DecisionTreeSuite { def generateOrderedLabeledPointsWithLabel1(): Array[LabeledPoint] = { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000){ - val lp = new LabeledPoint(1.0,Array(i.toDouble,999.0-i)) + val lp = new LabeledPoint(1.0, Vectors.dense(i.toDouble, 999.0 - i)) arr(i) = lp } arr @@ -415,9 +416,9 @@ object DecisionTreeSuite { val arr = new Array[LabeledPoint](1000) for (i <- 0 until 1000){ if (i < 600){ - arr(i) = new LabeledPoint(1.0,Array(0.0,1.0)) + arr(i) = new LabeledPoint(1.0, Vectors.dense(0.0, 1.0)) } else { - arr(i) = new LabeledPoint(0.0,Array(1.0,0.0)) + arr(i) = new LabeledPoint(0.0, Vectors.dense(1.0, 0.0)) } } arr diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 60f053b381305..27d41c7869aa0 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -17,14 +17,20 @@ package org.apache.spark.mllib.util +import java.io.File + import org.scalatest.FunSuite import breeze.linalg.{DenseVector => BDV, SparseVector => BSV, norm => breezeNorm, squaredDistance => breezeSquaredDistance} +import com.google.common.base.Charsets +import com.google.common.io.Files +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ -class MLUtilsSuite extends FunSuite { +class MLUtilsSuite extends FunSuite with LocalSparkContext { test("epsilon computation") { assert(1.0 + EPSILON > 1.0, s"EPSILON is too small: $EPSILON.") @@ -49,4 +55,55 @@ class MLUtilsSuite extends FunSuite { assert((fastSquaredDist2 - squaredDist) <= precision * squaredDist, s"failed with m = $m") } } + + test("compute stats") { + val data = Seq.fill(3)(Seq( + LabeledPoint(1.0, Vectors.dense(1.0, 2.0, 3.0)), + LabeledPoint(0.0, Vectors.dense(3.0, 4.0, 5.0)) + )).flatten + val rdd = sc.parallelize(data, 2) + val (meanLabel, mean, std) = MLUtils.computeStats(rdd, 3, 6) + assert(meanLabel === 0.5) + assert(mean === Vectors.dense(2.0, 3.0, 4.0)) + assert(std === Vectors.dense(1.0, 1.0, 1.0)) + } + + test("loadLibSVMData") { + val lines = + """ + |+1 1:1.0 3:2.0 5:3.0 + |-1 + |-1 2:4.0 4:5.0 6:6.0 + """.stripMargin + val tempDir = Files.createTempDir() + val file = new File(tempDir.getPath, "part-00000") + Files.write(lines, file, Charsets.US_ASCII) + val path = tempDir.toURI.toString + + val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, 6).collect() + val pointsWithoutNumFeatures = MLUtils.loadLibSVMData(sc, path).collect() + + for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { + assert(points.length === 3) + assert(points(0).label === 1.0) + assert(points(0).features === Vectors.sparse(6, Seq((0, 1.0), (2, 2.0), (4, 3.0)))) + assert(points(1).label == 0.0) + assert(points(1).features == Vectors.sparse(6, Seq())) + assert(points(2).label === 0.0) + assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) + } + + val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MLUtils.multiclassLabelParser).collect() + assert(multiclassPoints.length === 3) + assert(multiclassPoints(0).label === 1.0) + assert(multiclassPoints(1).label === -1.0) + assert(multiclassPoints(2).label === -1.0) + + try { + file.delete() + tempDir.delete() + } catch { + case t: Throwable => + } + } } diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 19b90dfd6e167..d2f9cdb3f4298 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -87,18 +87,19 @@ class NaiveBayesModel(object): >>> data = array([0.0, 0.0, 1.0, 0.0, 0.0, 2.0, 1.0, 1.0, 0.0]).reshape(3,3) >>> model = NaiveBayes.train(sc.parallelize(data)) >>> model.predict(array([0.0, 1.0])) - 0 + 0.0 >>> model.predict(array([1.0, 0.0])) - 1 + 1.0 """ - def __init__(self, pi, theta): + def __init__(self, labels, pi, theta): + self.labels = labels self.pi = pi self.theta = theta def predict(self, x): """Return the most likely class for a data vector x""" - return numpy.argmax(self.pi + dot(x, self.theta)) + return self.labels[numpy.argmax(self.pi + dot(x, self.theta))] class NaiveBayes(object): @classmethod @@ -122,7 +123,8 @@ def train(cls, data, lambda_=1.0): ans = sc._jvm.PythonMLLibAPI().trainNaiveBayes(dataBytes._jrdd, lambda_) return NaiveBayesModel( _deserialize_double_vector(ans[0]), - _deserialize_double_matrix(ans[1])) + _deserialize_double_vector(ans[1]), + _deserialize_double_matrix(ans[2])) def _test(): From 47ebea5468df2e4f94ef493c5403fcdcda8c5eb2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Wed, 2 Apr 2014 18:14:31 -0700 Subject: [PATCH 182/397] [SQL] SPARK-1364 Improve datatype and test coverage for ScalaReflection schema inference. Author: Michael Armbrust Closes #293 from marmbrus/reflectTypes and squashes the following commits: f54e8e8 [Michael Armbrust] Improve datatype and test coverage for ScalaReflection schema inference. --- .../spark/sql/catalyst/ScalaReflection.scala | 10 ++++ .../sql/ScalaReflectionRelationSuite.scala | 56 +++++++++++++++++++ 2 files changed, 66 insertions(+) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 976dda8d7e59a..5aaa63bf3b4b4 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -43,15 +43,25 @@ object ScalaReflection { val params = t.member("": TermName).asMethod.paramss StructType( params.head.map(p => StructField(p.name.toString, schemaFor(p.typeSignature), true))) + // Need to decide if we actually need a special type here. + case t if t <:< typeOf[Array[Byte]] => BinaryType + case t if t <:< typeOf[Array[_]] => + sys.error(s"Only Array[Byte] supported now, use Seq instead of $t") case t if t <:< typeOf[Seq[_]] => val TypeRef(_, _, Seq(elementType)) = t ArrayType(schemaFor(elementType)) + case t if t <:< typeOf[Map[_,_]] => + val TypeRef(_, _, Seq(keyType, valueType)) = t + MapType(schemaFor(keyType), schemaFor(valueType)) 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.FloatTpe => FloatType case t if t <:< definitions.DoubleTpe => DoubleType case t if t <:< definitions.ShortTpe => ShortType case t if t <:< definitions.ByteTpe => ByteType + case t if t <:< definitions.BooleanTpe => BooleanType + case t if t <:< typeOf[BigDecimal] => DecimalType } implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala new file mode 100644 index 0000000000000..70033a050c78c --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -0,0 +1,56 @@ +/* + * 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.FunSuite + +import org.apache.spark.sql.test.TestSQLContext._ + +case class ReflectData( + stringField: String, + intField: Int, + longField: Long, + floatField: Float, + doubleField: Double, + shortField: Short, + byteField: Byte, + booleanField: Boolean, + decimalField: BigDecimal, + seqInt: Seq[Int]) + +case class ReflectBinary(data: Array[Byte]) + +class ScalaReflectionRelationSuite extends FunSuite { + test("query case class RDD") { + val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, + BigDecimal(1), Seq(1,2,3)) + val rdd = sparkContext.parallelize(data :: Nil) + rdd.registerAsTable("reflectData") + + assert(sql("SELECT * FROM reflectData").collect().head === data.productIterator.toSeq) + } + + // Equality is broken for Arrays, so we test that separately. + test("query binary data") { + val rdd = sparkContext.parallelize(ReflectBinary(Array[Byte](1)) :: Nil) + rdd.registerAsTable("reflectBinary") + + val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] + assert(result.toSeq === Seq[Byte](1)) + } +} \ No newline at end of file From 92a86b285f8a4af1bdf577dd4c4ea0fd5ca8d682 Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Thu, 3 Apr 2014 14:08:47 -0700 Subject: [PATCH 183/397] [SPARK-1398] Removed findbugs jsr305 dependency Should be a painless upgrade, and does offer some significant advantages should we want to leverage FindBugs more during the 1.0 lifecycle. http://findbugs.sourceforge.net/findbugs2.html Author: Mark Hamstra Closes #307 from markhamstra/findbugs and squashes the following commits: 99f2d09 [Mark Hamstra] Removed unnecessary findbugs jsr305 dependency --- core/pom.xml | 4 ---- pom.xml | 5 ----- project/SparkBuild.scala | 1 - 3 files changed, 10 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index e4c32eff0cd77..273aa69659336 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -82,10 +82,6 @@ com.google.guava guava
    - - com.google.code.findbugs - jsr305 - org.slf4j slf4j-api diff --git a/pom.xml b/pom.xml index 7d58060cba606..b91b14d2f84d0 100644 --- a/pom.xml +++ b/pom.xml @@ -214,11 +214,6 @@ guava 14.0.1 - - com.google.code.findbugs - jsr305 - 1.3.9 - org.slf4j slf4j-api diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c5c697e8e2427..a2a21d9763548 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -296,7 +296,6 @@ object SparkBuild extends Build { name := "spark-core", libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", - "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, From fbebaedf26286ee8a75065822a3af1148351f828 Mon Sep 17 00:00:00 2001 From: Andre Schumacher Date: Thu, 3 Apr 2014 15:31:47 -0700 Subject: [PATCH 184/397] Spark parquet improvements A few improvements to the Parquet support for SQL queries: - Instead of files a ParquetRelation is now backed by a directory, which simplifies importing data from other sources - InsertIntoParquetTable operation now supports switching between overwriting or appending (at least in HiveQL) - tests now use the new API - Parquet logging can be set to WARNING level (Default) - Default compression for Parquet files (GZIP, as in parquet-mr) Author: Andre Schumacher Closes #195 from AndreSchumacher/spark_parquet_improvements and squashes the following commits: 54df314 [Andre Schumacher] SPARK-1383 [SQL] Improvements to ParquetRelation --- .../apache/spark/sql/catalyst/SqlParser.scala | 14 +- .../spark/sql/catalyst/analysis/Catalog.scala | 26 ++- .../org/apache/spark/sql/SQLContext.scala | 4 +- .../spark/sql/execution/SparkStrategies.scala | 6 +- .../spark/sql/parquet/ParquetRelation.scala | 129 ++++++++----- .../sql/parquet/ParquetTableOperations.scala | 139 +++++++++++--- .../sql/parquet/ParquetTableSupport.scala | 35 ++-- .../spark/sql/parquet/ParquetTestData.scala | 10 +- sql/core/src/test/resources/log4j.properties | 8 +- .../spark/sql/parquet/ParquetQuerySuite.scala | 118 ++++++++++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 2 + .../org/apache/spark/sql/hive/TestHive.scala | 2 + .../spark/sql/hive/CachedTableSuite.scala | 4 +- .../hive/execution/HiveComparisonTest.scala | 6 +- .../spark/sql/parquet/HiveParquetSuite.scala | 169 +++++++++--------- 15 files changed, 460 insertions(+), 212 deletions(-) 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 index 8de87594c8ab9..4ea80fee23e1e 100644 --- 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 @@ -106,6 +106,8 @@ class SqlParser extends StandardTokenParsers { protected val IF = Keyword("IF") protected val IN = Keyword("IN") protected val INNER = Keyword("INNER") + protected val INSERT = Keyword("INSERT") + protected val INTO = Keyword("INTO") protected val IS = Keyword("IS") protected val JOIN = Keyword("JOIN") protected val LEFT = Keyword("LEFT") @@ -114,6 +116,7 @@ class SqlParser extends StandardTokenParsers { protected val NULL = Keyword("NULL") protected val ON = Keyword("ON") protected val OR = Keyword("OR") + protected val OVERWRITE = Keyword("OVERWRITE") protected val LIKE = Keyword("LIKE") protected val RLIKE = Keyword("RLIKE") protected val REGEXP = Keyword("REGEXP") @@ -162,7 +165,7 @@ class SqlParser extends StandardTokenParsers { select * ( UNION ~ ALL ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Union(q1, q2) } | UNION ~ opt(DISTINCT) ^^^ { (q1: LogicalPlan, q2: LogicalPlan) => Distinct(Union(q1, q2)) } - ) + ) | insert protected lazy val select: Parser[LogicalPlan] = SELECT ~> opt(DISTINCT) ~ projections ~ @@ -185,6 +188,13 @@ class SqlParser extends StandardTokenParsers { withLimit } + protected lazy val insert: Parser[LogicalPlan] = + INSERT ~> opt(OVERWRITE) ~ inTo ~ select <~ opt(";") ^^ { + case o ~ r ~ s => + val overwrite: Boolean = o.getOrElse("") == "OVERWRITE" + InsertIntoTable(r, Map[String, Option[String]](), s, overwrite) + } + protected lazy val projections: Parser[Seq[Expression]] = repsep(projection, ",") protected lazy val projection: Parser[Expression] = @@ -195,6 +205,8 @@ class SqlParser extends StandardTokenParsers { protected lazy val from: Parser[LogicalPlan] = FROM ~> relations + protected lazy val inTo: Parser[LogicalPlan] = INTO ~> relation + // Based very loosely on the MySQL Grammar. // http://dev.mysql.com/doc/refman/5.0/en/join.html protected lazy val relations: Parser[LogicalPlan] = 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 6b58b9322c4bf..f30b5d816703a 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 @@ -31,19 +31,33 @@ trait Catalog { alias: Option[String] = None): LogicalPlan def registerTable(databaseName: Option[String], tableName: String, plan: LogicalPlan): Unit + def unregisterTable(databaseName: Option[String], tableName: String): Unit + + def unregisterAllTables(): Unit } class SimpleCatalog extends Catalog { val tables = new mutable.HashMap[String, LogicalPlan]() - def registerTable(databaseName: Option[String],tableName: String, plan: LogicalPlan): Unit = { + override def registerTable( + databaseName: Option[String], + tableName: String, + plan: LogicalPlan): Unit = { tables += ((tableName, plan)) } - def unregisterTable(databaseName: Option[String], tableName: String) = { tables -= tableName } + override def unregisterTable( + databaseName: Option[String], + tableName: String) = { + tables -= tableName + } + + override def unregisterAllTables() = { + tables.clear() + } - def lookupRelation( + override def lookupRelation( databaseName: Option[String], tableName: String, alias: Option[String] = None): LogicalPlan = { @@ -92,6 +106,10 @@ trait OverrideCatalog extends Catalog { override def unregisterTable(databaseName: Option[String], tableName: String): Unit = { overrides.remove((databaseName, tableName)) } + + override def unregisterAllTables(): Unit = { + overrides.clear() + } } /** @@ -113,4 +131,6 @@ object EmptyCatalog extends Catalog { def unregisterTable(databaseName: Option[String], tableName: String): Unit = { throw new UnsupportedOperationException } + + override def unregisterAllTables(): Unit = {} } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index f4bf00f4cffa6..36059c6630aa4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -80,12 +80,12 @@ class SQLContext(@transient val sparkContext: SparkContext) new SchemaRDD(this, SparkLogicalPlan(ExistingRdd.fromProductRdd(rdd))) /** - * Loads a parequet file, returning the result as a [[SchemaRDD]]. + * Loads a Parquet file, returning the result as a [[SchemaRDD]]. * * @group userf */ def parquetFile(path: String): SchemaRDD = - new SchemaRDD(this, parquet.ParquetRelation("ParquetFile", path)) + new SchemaRDD(this, parquet.ParquetRelation(path)) /** 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 b3e51fdf75270..fe8bd5a508820 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 @@ -171,10 +171,10 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] { // TODO: need to support writing to other types of files. Unify the below code paths. case logical.WriteToFile(path, child) => val relation = - ParquetRelation.create(path, child, sparkContext.hadoopConfiguration, None) - InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil + ParquetRelation.create(path, child, sparkContext.hadoopConfiguration) + InsertIntoParquetTable(relation, planLater(child), overwrite=true)(sparkContext) :: Nil case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) => - InsertIntoParquetTable(table, planLater(child))(sparkContext) :: Nil + InsertIntoParquetTable(table, planLater(child), overwrite)(sparkContext) :: Nil case PhysicalOperation(projectList, filters, relation: ParquetRelation) => // TODO: Should be pushing down filters as well. pruneFilterProject( 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 4ab755c096bd8..114bfbb719ee9 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 @@ -17,30 +17,29 @@ package org.apache.spark.sql.parquet -import java.io.{IOException, FileNotFoundException} - -import scala.collection.JavaConversions._ +import java.io.IOException import org.apache.hadoop.conf.Configuration -import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.permission.FsAction import org.apache.hadoop.mapreduce.Job -import parquet.hadoop.metadata.{FileMetaData, ParquetMetadata} import parquet.hadoop.util.ContextUtil -import parquet.hadoop.{Footer, ParquetFileReader, ParquetFileWriter} +import parquet.hadoop.{ParquetOutputFormat, Footer, ParquetFileWriter, ParquetFileReader} +import parquet.hadoop.metadata.{CompressionCodecName, FileMetaData, ParquetMetadata} import parquet.io.api.{Binary, RecordConsumer} +import parquet.schema.{Type => ParquetType, PrimitiveType => ParquetPrimitiveType, MessageType, MessageTypeParser} import parquet.schema.PrimitiveType.{PrimitiveTypeName => ParquetPrimitiveTypeName} import parquet.schema.Type.Repetition -import parquet.schema.{MessageType, MessageTypeParser} -import parquet.schema.{PrimitiveType => ParquetPrimitiveType} -import parquet.schema.{Type => ParquetType} import org.apache.spark.sql.catalyst.analysis.{MultiInstanceRelation, UnresolvedException} import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Row} -import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan} +import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LeafNode} import org.apache.spark.sql.catalyst.types._ +// Implicits +import scala.collection.JavaConversions._ + /** * Relation that consists of data stored in a Parquet columnar format. * @@ -48,14 +47,14 @@ import org.apache.spark.sql.catalyst.types._ * of using this class directly. * * {{{ - * val parquetRDD = sqlContext.parquetFile("path/to/parequet.file") + * val parquetRDD = sqlContext.parquetFile("path/to/parquet.file") * }}} * - * @param tableName The name of the relation that can be used in queries. * @param path The path to the Parquet file. */ -case class ParquetRelation(tableName: String, path: String) - extends BaseRelation with MultiInstanceRelation { +private[sql] case class ParquetRelation(val path: String) + extends LeafNode with MultiInstanceRelation { + self: Product => /** Schema derived from ParquetFile */ def parquetSchema: MessageType = @@ -65,33 +64,59 @@ case class ParquetRelation(tableName: String, path: String) .getSchema /** Attributes */ - val attributes = + override val output = ParquetTypesConverter - .convertToAttributes(parquetSchema) + .convertToAttributes(parquetSchema) - /** Output */ - override val output = attributes - - // 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 - - override def newInstance = ParquetRelation(tableName, path).asInstanceOf[this.type] + override def newInstance = ParquetRelation(path).asInstanceOf[this.type] // Equals must also take into account the output attributes so that we can distinguish between // different instances of the same relation, override def equals(other: Any) = other match { case p: ParquetRelation => - p.tableName == tableName && p.path == path && p.output == output + p.path == path && p.output == output case _ => false } } -object ParquetRelation { +private[sql] object ParquetRelation { + + def enableLogForwarding() { + // Note: Parquet does not use forwarding to parent loggers which + // is required for the JUL-SLF4J bridge to work. Also there is + // a default logger that appends to Console which needs to be + // reset. + import org.slf4j.bridge.SLF4JBridgeHandler + import java.util.logging.Logger + import java.util.logging.LogManager + + val loggerNames = Seq( + "parquet.hadoop.ColumnChunkPageWriteStore", + "parquet.hadoop.InternalParquetRecordWriter", + "parquet.hadoop.ParquetRecordReader", + "parquet.hadoop.ParquetInputFormat", + "parquet.hadoop.ParquetOutputFormat", + "parquet.hadoop.ParquetFileReader", + "parquet.hadoop.InternalParquetRecordReader", + "parquet.hadoop.codec.CodecConfig") + LogManager.getLogManager.reset() + SLF4JBridgeHandler.install() + for(name <- loggerNames) { + val logger = Logger.getLogger(name) + logger.setParent(Logger.getGlobal) + logger.setUseParentHandlers(true) + } + } // The element type for the RDDs that this relation maps to. type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow + // The compression type + type CompressionType = parquet.hadoop.metadata.CompressionCodecName + + // The default compression + val defaultCompression = CompressionCodecName.GZIP + /** * Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that * this is used inside [[org.apache.spark.sql.execution.SparkStrategies SparkStrategies]] to @@ -100,24 +125,39 @@ object ParquetRelation { * * @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. + * @param conf A configuration to be used. + * @return An empty ParquetRelation with inferred metadata. */ def create(pathString: String, child: LogicalPlan, - conf: Configuration, - tableName: Option[String]): ParquetRelation = { + conf: Configuration): ParquetRelation = { if (!child.resolved) { throw new UnresolvedException[LogicalPlan]( child, "Attempt to create Parquet table from unresolved child (when schema is not available)") } + createEmpty(pathString, child.output, conf) + } - val name = s"${tableName.getOrElse(child.nodeName)}_parquet" + /** + * Creates an empty ParquetRelation and underlying Parquetfile that only + * consists of the Metadata for the given schema. + * + * @param pathString The directory the Parquetfile will be stored in. + * @param attributes The schema of the relation. + * @param conf A configuration to be used. + * @return An empty ParquetRelation. + */ + def createEmpty(pathString: String, + attributes: Seq[Attribute], + conf: Configuration): ParquetRelation = { val path = checkPath(pathString, conf) - ParquetTypesConverter.writeMetaData(child.output, path, conf) - new ParquetRelation(name, path.toString) + if (conf.get(ParquetOutputFormat.COMPRESSION) == null) { + conf.set(ParquetOutputFormat.COMPRESSION, ParquetRelation.defaultCompression.name()) + } + ParquetRelation.enableLogForwarding() + ParquetTypesConverter.writeMetaData(attributes, path, conf) + new ParquetRelation(path.toString) } private def checkPath(pathStr: String, conf: Configuration): Path = { @@ -143,7 +183,7 @@ object ParquetRelation { } } -object ParquetTypesConverter { +private[parquet] 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 @@ -242,6 +282,7 @@ object ParquetTypesConverter { extraMetadata, "Spark") + ParquetRelation.enableLogForwarding() ParquetFileWriter.writeMetadataFile( conf, path, @@ -268,16 +309,24 @@ object ParquetTypesConverter { throw new IllegalArgumentException(s"Incorrectly formatted Parquet metadata path $origPath") } val path = origPath.makeQualified(fs) + if (!fs.getFileStatus(path).isDir) { + throw new IllegalArgumentException( + s"Expected $path for be a directory with Parquet files/metadata") + } + ParquetRelation.enableLogForwarding() val metadataPath = new Path(path, ParquetFileWriter.PARQUET_METADATA_FILE) + // if this is a new table that was just created we will find only the 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") + // there may be one or more Parquet files in the given directory + val footers = ParquetFileReader.readFooters(conf, fs.getFileStatus(path)) + // TODO: for now we assume that all footers (if there is more than one) have identical + // metadata; we may want to add a check here at some point + if (footers.size() == 0) { + throw new IllegalArgumentException(s"Could not find Parquet metadata at path $path") } - ParquetFileReader.readFooter(conf, path) + footers(0).getParquetMetadata } } } 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 7285f5b88b9bf..d5846baa72ada 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 @@ -24,26 +24,29 @@ import java.util.Date import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce._ -import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} +import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} +import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat, FileOutputCommitter} -import parquet.hadoop.util.ContextUtil import parquet.hadoop.{ParquetInputFormat, ParquetOutputFormat} +import parquet.hadoop.util.ContextUtil import parquet.io.InvalidRecordException import parquet.schema.MessageType +import org.apache.spark.{SerializableWritable, SparkContext, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, Row} import org.apache.spark.sql.execution.{LeafNode, SparkPlan, UnaryNode} -import org.apache.spark.{SerializableWritable, SparkContext, TaskContext} /** * Parquet table scan operator. Imports the file that backs the given * [[ParquetRelation]] as a RDD[Row]. */ case class ParquetTableScan( - @transient output: Seq[Attribute], - @transient relation: ParquetRelation, - @transient columnPruningPred: Option[Expression])( + // note: output cannot be transient, see + // https://issues.apache.org/jira/browse/SPARK-1367 + output: Seq[Attribute], + relation: ParquetRelation, + columnPruningPred: Option[Expression])( @transient val sc: SparkContext) extends LeafNode { @@ -53,6 +56,12 @@ case class ParquetTableScan( job, classOf[org.apache.spark.sql.parquet.RowReadSupport]) val conf: Configuration = ContextUtil.getConfiguration(job) + val fileList = FileSystemHelper.listFiles(relation.path, conf) + // add all paths in the directory but skip "hidden" ones such + // as "_SUCCESS" and "_metadata" + for (path <- fileList if !path.getName.startsWith("_")) { + NewFileInputFormat.addInputPath(job, path) + } conf.set( RowReadSupport.PARQUET_ROW_REQUESTED_SCHEMA, ParquetTypesConverter.convertFromAttributes(output).toString) @@ -63,14 +72,12 @@ case class ParquetTableScan( ``FilteredRecordReader`` (via Configuration, for example). Simple filter-rows-by-column-values however should be supported. */ - sc.newAPIHadoopFile( - relation.path, - classOf[ParquetInputFormat[Row]], - classOf[Void], classOf[Row], - conf) + sc.newAPIHadoopRDD(conf, classOf[ParquetInputFormat[Row]], classOf[Void], classOf[Row]) .map(_._2) } + override def otherCopyArgs = sc :: Nil + /** * Applies a (candidate) projection. * @@ -108,15 +115,31 @@ case class ParquetTableScan( } } +/** + * Operator that acts as a sink for queries on RDDs and can be used to + * store the output inside a directory of Parquet files. This operator + * is similar to Hive's INSERT INTO TABLE operation in the sense that + * one can choose to either overwrite or append to a directory. Note + * that consecutive insertions to the same table must have compatible + * (source) schemas. + * + * WARNING: EXPERIMENTAL! InsertIntoParquetTable with overwrite=false may + * cause data corruption in the case that multiple users try to append to + * the same table simultaneously. Inserting into a table that was + * previously generated by other means (e.g., by creating an HDFS + * directory and importing Parquet files generated by other tools) may + * cause unpredicted behaviour and therefore results in a RuntimeException + * (only detected via filename pattern so will not catch all cases). + */ case class InsertIntoParquetTable( - @transient relation: ParquetRelation, - @transient child: SparkPlan)( + relation: ParquetRelation, + child: SparkPlan, + overwrite: Boolean = false)( @transient val sc: SparkContext) extends UnaryNode with SparkHadoopMapReduceUtil { /** - * Inserts all the rows in the Parquet file. Note that OVERWRITE is implicit, since - * Parquet files are write-once. + * Inserts all rows into the Parquet file. */ override def execute() = { // TODO: currently we do not check whether the "schema"s are compatible @@ -135,19 +158,21 @@ case class InsertIntoParquetTable( classOf[org.apache.spark.sql.parquet.RowWriteSupport]) // TODO: move that to function in object - val conf = job.getConfiguration + val conf = ContextUtil.getConfiguration(job) conf.set(RowWriteSupport.PARQUET_ROW_SCHEMA, relation.parquetSchema.toString) val fspath = new Path(relation.path) val fs = fspath.getFileSystem(conf) - try { - fs.delete(fspath, true) - } catch { - case e: IOException => - throw new IOException( - s"Unable to clear output directory ${fspath.toString} prior" - + s" to InsertIntoParquetTable:\n${e.toString}") + if (overwrite) { + try { + fs.delete(fspath, true) + } catch { + case e: IOException => + throw new IOException( + s"Unable to clear output directory ${fspath.toString} prior" + + s" to InsertIntoParquetTable:\n${e.toString}") + } } saveAsHadoopFile(childRdd, relation.path.toString, conf) @@ -157,6 +182,8 @@ case class InsertIntoParquetTable( override def output = child.output + override def otherCopyArgs = sc :: Nil + // based on ``saveAsNewAPIHadoopFile`` in [[PairRDDFunctions]] // TODO: Maybe PairRDDFunctions should use Product2 instead of Tuple2? // .. then we could use the default one and could use [[MutablePair]] @@ -167,15 +194,21 @@ case class InsertIntoParquetTable( 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 wrappedConf = new SerializableWritable(job.getConfiguration) val formatter = new SimpleDateFormat("yyyyMMddHHmm") val jobtrackerID = formatter.format(new Date()) val stageId = sc.newRddId() + val taskIdOffset = + if (overwrite) 1 + else { + FileSystemHelper + .findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1 + } + 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. @@ -184,7 +217,7 @@ case class InsertIntoParquetTable( val attemptId = newTaskAttemptID(jobtrackerID, stageId, isMap = false, context.partitionId, attemptNumber) val hadoopContext = newTaskAttemptContext(wrappedConf.value, attemptId) - val format = outputFormatType.newInstance + val format = new AppendingParquetOutputFormat(taskIdOffset) val committer = format.getOutputCommitter(hadoopContext) committer.setupTask(hadoopContext) val writer = format.getRecordWriter(hadoopContext) @@ -196,7 +229,7 @@ case class InsertIntoParquetTable( committer.commitTask(hadoopContext) return 1 } - val jobFormat = outputFormatType.newInstance + val jobFormat = new AppendingParquetOutputFormat(taskIdOffset) /* 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. @@ -210,3 +243,55 @@ case class InsertIntoParquetTable( } } +// TODO: this will be able to append to directories it created itself, not necessarily +// to imported ones +private[parquet] class AppendingParquetOutputFormat(offset: Int) + extends parquet.hadoop.ParquetOutputFormat[Row] { + // override to accept existing directories as valid output directory + override def checkOutputSpecs(job: JobContext): Unit = {} + + // override to choose output filename so not overwrite existing ones + override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { + val taskId: TaskID = context.getTaskAttemptID.getTaskID + val partition: Int = taskId.getId + val filename = s"part-r-${partition + offset}.parquet" + val committer: FileOutputCommitter = + getOutputCommitter(context).asInstanceOf[FileOutputCommitter] + new Path(committer.getWorkPath, filename) + } +} + +private[parquet] object FileSystemHelper { + def listFiles(pathStr: String, conf: Configuration): Seq[Path] = { + val origPath = new Path(pathStr) + val fs = origPath.getFileSystem(conf) + if (fs == null) { + throw new IllegalArgumentException( + s"ParquetTableOperations: Path $origPath is incorrectly formatted") + } + val path = origPath.makeQualified(fs) + if (!fs.exists(path) || !fs.getFileStatus(path).isDir) { + throw new IllegalArgumentException( + s"ParquetTableOperations: path $path does not exist or is not a directory") + } + fs.listStatus(path).map(_.getPath) + } + + // finds the maximum taskid in the output file names at the given path + def findMaxTaskId(pathStr: String, conf: Configuration): Int = { + val files = FileSystemHelper.listFiles(pathStr, conf) + // filename pattern is part-r-.parquet + val nameP = new scala.util.matching.Regex("""part-r-(\d{1,}).parquet""", "taskid") + val hiddenFileP = new scala.util.matching.Regex("_.*") + files.map(_.getName).map { + case nameP(taskid) => taskid.toInt + case hiddenFileP() => 0 + case other: String => { + sys.error("ERROR: attempting to append to set of Parquet files and found file" + + s"that does not match name pattern: $other") + 0 + } + case _ => 0 + }.reduceLeft((a, b) => if (a < b) b else a) + } +} 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 c21e400282004..84b1b4609458b 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 @@ -35,7 +35,8 @@ import org.apache.spark.sql.catalyst.types._ * *@param root The root group converter for the record. */ -class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterializer[Row] { +private[parquet] class RowRecordMaterializer(root: CatalystGroupConverter) + extends RecordMaterializer[Row] { def this(parquetSchema: MessageType) = this(new CatalystGroupConverter(ParquetTypesConverter.convertToAttributes(parquetSchema))) @@ -48,14 +49,14 @@ class RowRecordMaterializer(root: CatalystGroupConverter) extends RecordMaterial /** * A `parquet.hadoop.api.ReadSupport` for Row objects. */ -class RowReadSupport extends ReadSupport[Row] with Logging { +private[parquet] class RowReadSupport extends ReadSupport[Row] with Logging { override def prepareForRead( conf: Configuration, stringMap: java.util.Map[String, String], fileSchema: MessageType, readContext: ReadContext): RecordMaterializer[Row] = { - log.debug(s"preparing for read with schema ${fileSchema.toString}") + log.debug(s"preparing for read with file schema $fileSchema") new RowRecordMaterializer(readContext.getRequestedSchema) } @@ -67,20 +68,20 @@ class RowReadSupport extends ReadSupport[Row] with Logging { 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}") + log.debug(s"read support initialized for requested schema $requested_schema") + ParquetRelation.enableLogForwarding() new ReadContext(requested_schema, keyValueMetaData) } } -object RowReadSupport { +private[parquet] 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 { +private[parquet] class RowWriteSupport extends WriteSupport[Row] with Logging { def setSchema(schema: MessageType, configuration: Configuration) { // for testing this.schema = schema @@ -104,6 +105,8 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { override def init(configuration: Configuration): WriteSupport.WriteContext = { schema = if (schema == null) getSchema(configuration) else schema attributes = ParquetTypesConverter.convertToAttributes(schema) + log.debug(s"write support initialized for requested schema $schema") + ParquetRelation.enableLogForwarding() new WriteSupport.WriteContext( schema, new java.util.HashMap[java.lang.String, java.lang.String]()) @@ -111,10 +114,16 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { override def prepareForWrite(recordConsumer: RecordConsumer): Unit = { writer = recordConsumer + log.debug(s"preparing for write with schema $schema") } // TODO: add groups (nested fields) override def write(record: Row): Unit = { + if (attributes.size > record.size) { + throw new IndexOutOfBoundsException( + s"Trying to write more fields than contained in row (${attributes.size}>${record.size})") + } + var index = 0 writer.startMessage() while(index < attributes.size) { @@ -130,7 +139,7 @@ class RowWriteSupport extends WriteSupport[Row] with Logging { } } -object RowWriteSupport { +private[parquet] object RowWriteSupport { val PARQUET_ROW_SCHEMA: String = "org.apache.spark.sql.parquet.row.schema" } @@ -139,7 +148,7 @@ object RowWriteSupport { * * @param schema The corresponding Catalyst schema in the form of a list of attributes. */ -class CatalystGroupConverter( +private[parquet] class CatalystGroupConverter( schema: Seq[Attribute], protected[parquet] val current: ParquetRelation.RowType) extends GroupConverter { @@ -177,13 +186,12 @@ class CatalystGroupConverter( * @param parent The parent group converter. * @param fieldIndex The index inside the record. */ -class CatalystPrimitiveConverter( +private[parquet] class CatalystPrimitiveConverter( parent: CatalystGroupConverter, fieldIndex: Int) extends PrimitiveConverter { // TODO: consider refactoring these together with ParquetTypesConverter override def addBinary(value: Binary): Unit = - // TODO: fix this once a setBinary will become available in MutableRow - parent.getCurrentRecord.setByte(fieldIndex, value.getBytes.apply(0)) + parent.getCurrentRecord.update(fieldIndex, value.getBytes) override def addBoolean(value: Boolean): Unit = parent.getCurrentRecord.setBoolean(fieldIndex, value) @@ -208,10 +216,9 @@ class CatalystPrimitiveConverter( * @param parent The parent group converter. * @param fieldIndex The index inside the record. */ -class CatalystPrimitiveStringConverter( +private[parquet] class CatalystPrimitiveStringConverter( parent: CatalystGroupConverter, fieldIndex: Int) extends CatalystPrimitiveConverter(parent, fieldIndex) { override def addBinary(value: Binary): Unit = 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 3340c3ff81f0a..728e3dd1dc02b 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,7 +26,7 @@ import parquet.hadoop.util.ContextUtil import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.sql.catalyst.expressions.GenericRow -import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.util.Utils object ParquetTestData { @@ -64,13 +64,13 @@ object ParquetTestData { "mylong:Long" ) - val testFile = getTempFilePath("testParquetFile").getCanonicalFile + val testDir = Utils.createTempDir() - lazy val testData = new ParquetRelation("testData", testFile.toURI.toString) + lazy val testData = new ParquetRelation(testDir.toURI.toString) def writeFile() = { - testFile.delete - val path: Path = new Path(testFile.toURI) + testDir.delete + val path: Path = new Path(new Path(testDir.toURI), new Path("part-r-0.parquet")) val job = new Job() val configuration: Configuration = ContextUtil.getConfiguration(job) val schema: MessageType = MessageTypeParser.parseMessageType(testSchema) diff --git a/sql/core/src/test/resources/log4j.properties b/sql/core/src/test/resources/log4j.properties index 7bb6789bd33a5..dffd15a61838b 100644 --- a/sql/core/src/test/resources/log4j.properties +++ b/sql/core/src/test/resources/log4j.properties @@ -45,8 +45,6 @@ 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=WARN -log4j.logger.parquet.hadoop.InternalParquetRecordReader=WARN -parquet.hadoop.ParquetInputFormat=WARN -log4j.logger.parquet.hadoop.ParquetInputFormat=WARN +# Parquet related logging +log4j.logger.parquet.hadoop=WARN +log4j.logger.org.apache.spark.sql.parquet=INFO diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index ea1733b3614e5..a62a3c4d02354 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -19,27 +19,40 @@ package org.apache.spark.sql.parquet import org.scalatest.{BeforeAndAfterAll, FunSuite} -import org.apache.hadoop.fs.{FileSystem, Path} +import org.apache.hadoop.fs.{Path, FileSystem} import org.apache.hadoop.mapreduce.Job + import parquet.hadoop.ParquetFileWriter -import parquet.hadoop.util.ContextUtil import parquet.schema.MessageTypeParser +import parquet.hadoop.util.ContextUtil -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.util.Utils +import org.apache.spark.sql.catalyst.types.{StringType, IntegerType, DataType} +import org.apache.spark.sql.{parquet, SchemaRDD} +import org.apache.spark.sql.catalyst.expressions.AttributeReference +import scala.Tuple2 // Implicits import org.apache.spark.sql.test.TestSQLContext._ +case class TestRDDEntry(key: Int, value: String) + class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { + + var testRDD: SchemaRDD = null + override def beforeAll() { ParquetTestData.writeFile() + testRDD = parquetFile(ParquetTestData.testDir.toString) + testRDD.registerAsTable("testsource") } override def afterAll() { - ParquetTestData.testFile.delete() + Utils.deleteRecursively(ParquetTestData.testDir) + // here we should also unregister the table?? } test("self-join parquet files") { @@ -55,11 +68,18 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { case Seq(_, _) => // All good } - // TODO: We can't run this query as it NPEs + val result = query.collect() + assert(result.size === 9, "self-join result has incorrect size") + assert(result(0).size === 12, "result row has incorrect size") + result.zipWithIndex.foreach { + case (row, index) => row.zipWithIndex.foreach { + case (field, column) => assert(field != null, s"self-join contains null value in row $index field $column") + } + } } test("Import of simple Parquet file") { - val result = getRDD(ParquetTestData.testData).collect() + val result = parquetFile(ParquetTestData.testDir.toString).collect() assert(result.size === 15) result.zipWithIndex.foreach { case (row, index) => { @@ -125,20 +145,82 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { fs.delete(path, true) } + test("Creating case class RDD table") { + TestSQLContext.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + .registerAsTable("tmp") + val rdd = sql("SELECT * FROM tmp").collect().sortBy(_.getInt(0)) + var counter = 1 + rdd.foreach { + // '===' does not like string comparison? + row: Row => { + assert(row.getString(1).equals(s"val_$counter"), s"row $counter value ${row.getString(1)} does not match val_$counter") + counter = counter + 1 + } + } + } + + test("Saving case class RDD table to file and reading it back in") { + val file = getTempFilePath("parquet") + val path = file.toString + val rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + rdd.saveAsParquetFile(path) + val readFile = parquetFile(path) + readFile.registerAsTable("tmpx") + val rdd_copy = sql("SELECT * FROM tmpx").collect() + val rdd_orig = rdd.collect() + for(i <- 0 to 99) { + assert(rdd_copy(i).apply(0) === rdd_orig(i).key, s"key error in line $i") + assert(rdd_copy(i).apply(1) === rdd_orig(i).value, s"value in line $i") + } + Utils.deleteRecursively(file) + assert(true) + } + + test("insert (overwrite) via Scala API (new SchemaRDD)") { + val dirname = Utils.createTempDir() + val source_rdd = TestSQLContext.sparkContext.parallelize((1 to 100)) + .map(i => TestRDDEntry(i, s"val_$i")) + source_rdd.registerAsTable("source") + val dest_rdd = createParquetFile(dirname.toString, ("key", IntegerType), ("value", StringType)) + dest_rdd.registerAsTable("dest") + sql("INSERT OVERWRITE INTO dest SELECT * FROM source").collect() + val rdd_copy1 = sql("SELECT * FROM dest").collect() + assert(rdd_copy1.size === 100) + assert(rdd_copy1(0).apply(0) === 1) + assert(rdd_copy1(0).apply(1) === "val_1") + sql("INSERT INTO dest SELECT * FROM source").collect() + val rdd_copy2 = sql("SELECT * FROM dest").collect() + assert(rdd_copy2.size === 200) + Utils.deleteRecursively(dirname) + } + + test("insert (appending) to same table via Scala API") { + sql("INSERT INTO testsource SELECT * FROM testsource").collect() + val double_rdd = sql("SELECT * FROM testsource").collect() + assert(double_rdd != null) + assert(double_rdd.size === 30) + for(i <- (0 to 14)) { + assert(double_rdd(i) === double_rdd(i+15), s"error: lines $i and ${i+15} to not match") + } + // let's restore the original test data + Utils.deleteRecursively(ParquetTestData.testDir) + ParquetTestData.writeFile() + } + /** - * Computes the given [[ParquetRelation]] and returns its RDD. + * Creates an empty SchemaRDD backed by a ParquetRelation. * - * @param parquetRelation The Parquet relation. - * @return An RDD of Rows. + * TODO: since this is so experimental it is better to have it here and not + * in SQLContext. Also note that when creating new AttributeReferences + * one needs to take care not to create duplicate Attribute ID's. */ - private def getRDD(parquetRelation: ParquetRelation): RDD[Row] = { - val scanner = new ParquetTableScan( - parquetRelation.output, - parquetRelation, - None)(TestSQLContext.sparkContext) - scanner - .execute - .map(_.copy()) + private def createParquetFile(path: String, schema: (Tuple2[String, DataType])*): SchemaRDD = { + val attributes = schema.map(t => new AttributeReference(t._1, t._2)()) + new SchemaRDD( + TestSQLContext, + parquet.ParquetRelation.createEmpty(path, attributes, sparkContext.hadoopConfiguration)) } } 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 29834a11f41dc..fc053c56c052d 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 @@ -148,6 +148,8 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { */ override def unregisterTable( databaseName: Option[String], tableName: String): Unit = ??? + + override def unregisterAllTables() = {} } object HiveMetastoreTypes extends RegexParsers { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index bc3447b9d802d..0a6bea0162430 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -313,6 +313,8 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { catalog.client.dropDatabase(db, true, false, true) } + catalog.unregisterAllTables() + FunctionRegistry.getFunctionNames.filterNot(originalUdfs.contains(_)).foreach { udfName => FunctionRegistry.unregisterTemporaryUDF(udfName) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 68d45e53cdf26..79ec1f1cde019 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -29,7 +29,7 @@ class CachedTableSuite extends HiveComparisonTest { } createQueryTest("read from cached table", - "SELECT * FROM src LIMIT 1") + "SELECT * FROM src LIMIT 1", reset = false) test("check that table is cached and uncache") { TestHive.table("src").queryExecution.analyzed match { @@ -40,7 +40,7 @@ class CachedTableSuite extends HiveComparisonTest { } createQueryTest("read from uncached table", - "SELECT * FROM src LIMIT 1") + "SELECT * FROM src LIMIT 1", reset = false) test("make sure table is uncached") { TestHive.table("src").queryExecution.analyzed match { 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 c7a350ef94edd..18654b308d234 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 @@ -170,7 +170,7 @@ abstract class HiveComparisonTest } val installHooksCommand = "(?i)SET.*hooks".r - def createQueryTest(testCaseName: String, sql: String) { + def createQueryTest(testCaseName: String, sql: String, reset: Boolean = true) { // 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 @@ -228,7 +228,7 @@ abstract class HiveComparisonTest try { // MINOR HACK: You must run a query before calling reset the first time. TestHive.sql("SHOW TABLES") - TestHive.reset() + if (reset) { TestHive.reset() } val hiveCacheFiles = queryList.zipWithIndex.map { case (queryString, i) => @@ -295,7 +295,7 @@ abstract class HiveComparisonTest fail(errorMessage) } }.toSeq - TestHive.reset() + if (reset) { TestHive.reset() } computedResults } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 05ad85b622ac8..314ca48ad8f6a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -17,147 +17,138 @@ package org.apache.spark.sql.parquet -import java.io.File - import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} -import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation -import org.apache.spark.sql.catalyst.expressions.Row -import org.apache.spark.sql.catalyst.plans.logical._ -import org.apache.spark.sql.catalyst.util.getTempFilePath +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} +import org.apache.spark.sql.catalyst.types.{DataType, StringType, IntegerType} +import org.apache.spark.sql.{parquet, SchemaRDD} import org.apache.spark.sql.hive.TestHive +import org.apache.spark.util.Utils + +// Implicits +import org.apache.spark.sql.hive.TestHive._ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { - 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] = { - - // call to resolve references in order to get CREATE TABLE AS to work - val query = TestHive - .parseSql(querystr) - val finalQuery = - if (tableName.nonEmpty && filename.nonEmpty) - resolveParquetTable(tableName.get, filename.get, query) - else - query - TestHive.executePlan(finalQuery) - .toRdd - .collect() - } - // stores a query output to a Parquet file - def storeQuery(querystr: String, filename: String): Unit = { - val query = WriteToFile( - filename, - TestHive.parseSql(querystr)) - TestHive - .executePlan(query) - .stringResult() - } + val dirname = Utils.createTempDir() - /** - * 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 = { - TestHive.loadTestTable("src") // may not be loaded now - 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 - val relation = ParquetRelation.create( - filename, - TestHive.analyzer(child), - TestHive.sparkContext.hadoopConfiguration, - Some(tableName)) - InsertIntoTable( - relation.asInstanceOf[BaseRelation], - Map.empty, - child, - overwrite = false) - } else - op - } - } + var testRDD: SchemaRDD = null override def beforeAll() { // write test data - ParquetTestData.writeFile() - // Override initial Parquet test table - TestHive.catalog.registerTable(Some[String]("parquet"), "testsource", ParquetTestData.testData) + ParquetTestData.writeFile + testRDD = parquetFile(ParquetTestData.testDir.toString) + testRDD.registerAsTable("testsource") } override def afterAll() { - ParquetTestData.testFile.delete() + Utils.deleteRecursively(ParquetTestData.testDir) + Utils.deleteRecursively(dirname) + reset() // drop all tables that were registered as part of the tests } + // in case tests are failing we delete before and after each test override def beforeEach() { - new File(filename).getAbsoluteFile.delete() + Utils.deleteRecursively(dirname) } override def afterEach() { - new File(filename).getAbsoluteFile.delete() + Utils.deleteRecursively(dirname) } test("SELECT on Parquet table") { - val rdd = runQuery("SELECT * FROM parquet.testsource") + val rdd = sql("SELECT * FROM testsource").collect() assert(rdd != null) assert(rdd.forall(_.size == 6)) } test("Simple column projection + filter on Parquet table") { - val rdd = runQuery("SELECT myboolean, mylong FROM parquet.testsource WHERE myboolean=true") + val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() 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)) + test("Converting Hive to Parquet Table via saveAsParquetFile") { + sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) + parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") + val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0)) + val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0)) compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String")) } 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") - compareRDDs(rddOrig, rddCopy, "parquet.testsource", ParquetTestData.testSchemaFieldNames) + sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) + parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") + // let's do three overwrites for good measure + sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + val rddCopy = sql("SELECT * FROM ptable").collect() + val rddOrig = sql("SELECT * FROM testsource").collect() + assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??") + compareRDDs(rddOrig, rddCopy, "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)) + test("CREATE TABLE of Parquet table") { + createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType)) + .registerAsTable("tmp") + val rddCopy = + sql("INSERT INTO TABLE tmp SELECT * FROM src") + .collect() .sortBy[Int](_.apply(0) match { case x: Int => x case _ => 0 }) - val rddOrig = runQuery("SELECT * FROM src").sortBy(_.getInt(0)) + val rddOrig = sql("SELECT * FROM src") + .collect() + .sortBy(_.getInt(0)) compareRDDs(rddOrig, rddCopy, "src (Hive)", Seq("key:Int", "value:String")) } + test("Appending to Parquet table") { + createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType)) + .registerAsTable("tmpnew") + sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() + sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() + sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() + val rddCopies = sql("SELECT * FROM tmpnew").collect() + val rddOrig = sql("SELECT * FROM src").collect() + assert(rddCopies.size === 3 * rddOrig.size, "number of copied rows via INSERT INTO did not match correct number") + } + + test("Appending to and then overwriting Parquet table") { + createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType)) + .registerAsTable("tmp") + sql("INSERT INTO TABLE tmp SELECT * FROM src").collect() + sql("INSERT INTO TABLE tmp SELECT * FROM src").collect() + sql("INSERT OVERWRITE TABLE tmp SELECT * FROM src").collect() + val rddCopies = sql("SELECT * FROM tmp").collect() + val rddOrig = sql("SELECT * FROM src").collect() + assert(rddCopies.size === rddOrig.size, "INSERT OVERWRITE did not actually overwrite") + } + 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: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") } counter = counter + 1 } } + + /** + * Creates an empty SchemaRDD backed by a ParquetRelation. + * + * TODO: since this is so experimental it is better to have it here and not + * in SQLContext. Also note that when creating new AttributeReferences + * one needs to take care not to create duplicate Attribute ID's. + */ + private def createParquetFile(path: String, schema: (Tuple2[String, DataType])*): SchemaRDD = { + val attributes = schema.map(t => new AttributeReference(t._1, t._2)()) + new SchemaRDD( + TestHive, + parquet.ParquetRelation.createEmpty(path, attributes, sparkContext.hadoopConfiguration)) + } } From 5d1feda217d25616d190f9bb369664e57417cd45 Mon Sep 17 00:00:00 2001 From: Cheng Hao Date: Thu, 3 Apr 2014 15:33:17 -0700 Subject: [PATCH 185/397] [SPARK-1360] Add Timestamp Support for SQL This PR includes: 1) Add new data type Timestamp 2) Add more data type casting base on Hive's Rule 3) Fix bug missing data type in both parsers (HiveQl & SQLParser). Author: Cheng Hao Closes #275 from chenghao-intel/timestamp and squashes the following commits: df709e5 [Cheng Hao] Move orc_ends_with_nulls to blacklist 24b04b0 [Cheng Hao] Put 3 cases into the black lists(describe_pretty,describe_syntax,lateral_view_outer) fc512c2 [Cheng Hao] remove the unnecessary data type equality check in data casting d0d1919 [Cheng Hao] Add more data type for scala reflection 3259808 [Cheng Hao] Add the new Golden files 3823b97 [Cheng Hao] Update the UnitTest cases & add timestamp type for HiveQL 54a0489 [Cheng Hao] fix bug mapping to 0 (which is supposed to be null) when NumberFormatException occurs 9cb505c [Cheng Hao] Fix issues according to PR comments e529168 [Cheng Hao] Fix bug of converting from String 6fc8100 [Cheng Hao] Update Unit Test & CodeStyle 8a1d4d6 [Cheng Hao] Add DataType for SqlParser ce4385e [Cheng Hao] Add TimestampType Support --- .../spark/sql/catalyst/ScalaReflection.scala | 7 +- .../spark/sql/catalyst/dsl/package.scala | 41 +++- .../spark/sql/catalyst/expressions/Cast.scala | 193 ++++++++++++++---- .../sql/catalyst/expressions/Expression.scala | 54 ++++- .../sql/catalyst/expressions/literals.scala | 5 + .../sql/catalyst/expressions/predicates.scala | 59 +----- .../spark/sql/catalyst/types/dataTypes.scala | 12 ++ .../ExpressionEvaluationSuite.scala | 53 +++++ .../sql/ScalaReflectionRelationSuite.scala | 5 +- .../org/apache/spark/sql/hive/HiveQl.scala | 13 +- ...insert1-0-7faa9807151781e4207103aa568e321c | 0 ...insert1-1-91d7b05c9024bff60b55f415cbeacc8b | 0 ...nsert1-10-64f83491a8fe675ef3a4a9a474ac0439 | 0 ...nsert1-11-6f2797b6f81943d3b53b8d247ae8512b | 0 ...nsert1-12-7a3c0a3f06484c912b9e951d8a2d8ac6 | 0 ...nsert1-13-42b03f938894fdafc7fff640711a9b2f | 0 ...nsert1-14-e021dfb28597811870c03b3242972927 | 0 ...nsert1-15-c7fca497a4580b54a0a13b3b72da5d7c | 2 + ...nsert1-16-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...nsert1-17-5528e36b3b0f5b14313898cc45f9c23a | 0 ...nsert1-18-16d78fba2d86277bc2f804037cc0a8b4 | 0 ...nsert1-19-62518ff6810db9cdd8926702192a206b | 0 ...insert1-2-3f1de4475930285c3fdbe3a5ccd4e868 | 0 ...nsert1-20-f4dc51ad64bb8662d066a8b9003da3d4 | 0 ...nsert1-21-bb7624250ab556f2d40bfb8d419be487 | 2 + ...insert1-3-89f8a028e32fae213b575b4df4e26e9c | 0 ...insert1-4-c7a68c0884785d0f5e62b287eb305d64 | 0 ...insert1-5-cb87ee12092fdf05daed82485c32a285 | 0 ...insert1-6-b97ba93a2c9ae671ecfc4fa95c024dda | 0 ...insert1-7-a2cd0615b9e79befd9c1842516150a61 | 0 ...insert1-8-5942e331621fe522fc297844046d2370 | 0 ...insert1-9-5c5132707d7a4fb6e6a3de1a6719721a | 0 ...titions-0-5528e36b3b0f5b14313898cc45f9c23a | 0 ...titions-1-deb504f4f70fd7db975950c3c47959ee | 0 ...itions-10-fda2e4be738186c0938f92d5072df55a | 0 ...itions-11-9fb177236623d1b62acff28507033436 | 5 + ...itions-12-99d5ad32bb81640cb284312841b60000 | 0 ...itions-13-9dda06e1aae1860bd19eee97703a8217 | 0 ...itions-14-19daabdd4c0d403c8781967248d09c53 | 0 ...itions-15-812006e1f11e005e5029866d1cf004f6 | 0 ...titions-2-bd042746328158822a25d711ffed18dd | 0 ...titions-3-b7aaedd7d624af4e48637ff1acabe485 | 0 ...titions-4-dece2650bf0615e566cd6c84181ce026 | 0 ...titions-5-1eb5c694e5a02aa292e24a0849350108 | 0 ...titions-6-ab49e0665a80a6b34dadc96f1d18ce26 | 0 ...titions-7-fda2e4be738186c0938f92d5072df55a | 0 ...titions-8-9fb177236623d1b62acff28507033436 | 5 + ...titions-9-ab49e0665a80a6b34dadc96f1d18ce26 | 0 ...ry_data-0-491edd0c42ceb79e799ba50555bc8c15 | 0 ...ry_data-1-5d72f8449b69df3c08e3f444f09428bc | 0 ...ary_data-2-242b1655c7e7325ee9f26552ea8fc25 | 0 ...ry_data-3-2a72df8d3e398d0963ef91162ce7d268 | 0 ...de_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...de_hook-1-2b1df88619e34f221d39598b5cd73283 | 1 + ...e_hook-10-60eadbb52f8857830a3034952c631ace | 0 ...e_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 | 0 ...e_hook-12-60018cae9a0476dc6a0ab4264310edb5 | 0 ...de_hook-2-7562d4fee13f3ba935a2e824f86a4224 | 1 + ...de_hook-3-bdb30a5d6887ee4fb089f8676313eafd | 1 + ...de_hook-4-10713b30ecb3c88acdd775bf9628c38c | 1 + ...de_hook-5-bab89dfffa77258e34a595e0e79986e3 | 1 + ...de_hook-6-6f53d5613262d393d82d159ec5dc16dc | 1 + ...de_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f | 0 ...de_hook-8-f9dd797f1c90e2108cfee585f443c132 | 0 ...de_hook-9-22fdd8380f2652de2492b34a425d46d7 | 0 ...adoop20-0-7a9e67189d3d4151f23b12c22bde06b5 | 0 ...adoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c | 1 + ...doop20-10-22fdd8380f2652de2492b34a425d46d7 | 0 ...doop20-11-60eadbb52f8857830a3034952c631ace | 0 ...doop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 | 0 ...doop20-13-60018cae9a0476dc6a0ab4264310edb5 | 0 ...adoop20-2-2b1df88619e34f221d39598b5cd73283 | 1 + ...adoop20-3-7562d4fee13f3ba935a2e824f86a4224 | 1 + ...adoop20-4-bdb30a5d6887ee4fb089f8676313eafd | 1 + ...adoop20-5-10713b30ecb3c88acdd775bf9628c38c | 1 + ...adoop20-6-bab89dfffa77258e34a595e0e79986e3 | 1 + ...adoop20-7-6f53d5613262d393d82d159ec5dc16dc | 1 + ...adoop20-8-7a45282169e5a15d70ae0afb9e67ec9a | 0 ...adoop20-9-f9dd797f1c90e2108cfee585f443c132 | 0 ...parison-0-48751533b44ea9e8ac3131767c2fed05 | 1 + ...parison-1-60557e7bd2822c89fa8b076a9d0520fc | 1 + ...parison-2-f96a9d88327951bd93f672dc2463ecd4 | 1 + ...parison-3-13e17ed811165196416f777cbc162592 | 1 + ...parison-4-4fa8a36edbefde4427c2ab2cf30e6399 | 1 + ...parison-5-7e4fb6e8ba01df422e4c67e06a0c8453 | 1 + ...parison-6-8c8e73673a950f6b3d960b08fcea076f | 1 + ...parison-7-510c0a2a57dc5df8588bd13c4152f8bc | 1 + ...parison-8-659d5b1ae8200f13f265270e52a3dd65 | 1 + ..._cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d | 0 ..._cast_1-1-53a667981ad567b2ab977f67d65c5825 | 1 + ..._printf-0-e86d559aeb84a4cc017a103182c22bfb | 0 ..._printf-1-19c61fce27310ab2590062d643f7b26e | 1 + ..._printf-2-25aa6950cae2bb781c336378f63ceaee | 4 + ..._printf-3-9c568a0473888396bd46507e8b330c36 | 0 ..._printf-4-91728e546b450bdcbb05ef30f13be475 | 1 + ..._printf-5-3141a0421605b091ee5a9e99d7d605fb | 1 + ..._printf-6-ec37b73012f3cbbbc0422744b0db8294 | 1 + ..._printf-7-5769f3a5b3300ca1d8b861229e976126 | 0 ...boolean-10-51822ac740629bebd81d2abda6e1144 | 1 + ...oolean-11-441306cae24618c49ec63445a31bf16b | 1 + ...oolean-12-bfcc534e73e320a1cfad9c584678d870 | 1 + ...oolean-13-a2bddaa5db1841bb4617239b9f17a06d | 1 + ...oolean-14-773801b833cf72d35016916b786275b5 | 1 + ...oolean-15-4071ed0ff57b53963d5ee662fa9db0b0 | 1 + ...oolean-16-6b441df08afdc0c6c4a82670997dabb5 | 1 + ...oolean-17-85342c694d7f35e7eedb24e850d0c7df | 1 + ...oolean-18-fcd7af0e71d3e2d934239ba606e3ed87 | 1 + ...oolean-19-dcdb12fe551aa68a56921822f5d1a343 | 1 + ...oolean-20-131900d39d9a20b431731a32fb9715f8 | 1 + ...oolean-21-a5e28f4eb819e5a5e292e279f2990a7a | 1 + ...oolean-22-93278c10d642fa242f303d89b3b1961d | 1 + ...oolean-23-828558020ce907ffa7e847762a5e2358 | 1 + ...oolean-24-e8ca597d87932af16c0cf29d662e92da | 1 + ...oolean-25-86245727f90de9ce65a12c97a03a5635 | 1 + ...oolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 | 1 + ...oolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 | 1 + ...boolean-8-37229f303635a030f6cab20e0381f51f | 1 + ...boolean-9-be623247e4dbf119b43458b72d1be017 | 1 + .../execution/HiveCompatibilitySuite.scala | 87 ++++++-- 119 files changed, 480 insertions(+), 115 deletions(-) create mode 100644 sql/hive/src/test/resources/golden/insert1-0-7faa9807151781e4207103aa568e321c create mode 100644 sql/hive/src/test/resources/golden/insert1-1-91d7b05c9024bff60b55f415cbeacc8b create mode 100644 sql/hive/src/test/resources/golden/insert1-10-64f83491a8fe675ef3a4a9a474ac0439 create mode 100644 sql/hive/src/test/resources/golden/insert1-11-6f2797b6f81943d3b53b8d247ae8512b create mode 100644 sql/hive/src/test/resources/golden/insert1-12-7a3c0a3f06484c912b9e951d8a2d8ac6 create mode 100644 sql/hive/src/test/resources/golden/insert1-13-42b03f938894fdafc7fff640711a9b2f create mode 100644 sql/hive/src/test/resources/golden/insert1-14-e021dfb28597811870c03b3242972927 create mode 100644 sql/hive/src/test/resources/golden/insert1-15-c7fca497a4580b54a0a13b3b72da5d7c create mode 100644 sql/hive/src/test/resources/golden/insert1-16-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 sql/hive/src/test/resources/golden/insert1-17-5528e36b3b0f5b14313898cc45f9c23a create mode 100644 sql/hive/src/test/resources/golden/insert1-18-16d78fba2d86277bc2f804037cc0a8b4 create mode 100644 sql/hive/src/test/resources/golden/insert1-19-62518ff6810db9cdd8926702192a206b create mode 100644 sql/hive/src/test/resources/golden/insert1-2-3f1de4475930285c3fdbe3a5ccd4e868 create mode 100644 sql/hive/src/test/resources/golden/insert1-20-f4dc51ad64bb8662d066a8b9003da3d4 create mode 100644 sql/hive/src/test/resources/golden/insert1-21-bb7624250ab556f2d40bfb8d419be487 create mode 100644 sql/hive/src/test/resources/golden/insert1-3-89f8a028e32fae213b575b4df4e26e9c create mode 100644 sql/hive/src/test/resources/golden/insert1-4-c7a68c0884785d0f5e62b287eb305d64 create mode 100644 sql/hive/src/test/resources/golden/insert1-5-cb87ee12092fdf05daed82485c32a285 create mode 100644 sql/hive/src/test/resources/golden/insert1-6-b97ba93a2c9ae671ecfc4fa95c024dda create mode 100644 sql/hive/src/test/resources/golden/insert1-7-a2cd0615b9e79befd9c1842516150a61 create mode 100644 sql/hive/src/test/resources/golden/insert1-8-5942e331621fe522fc297844046d2370 create mode 100644 sql/hive/src/test/resources/golden/insert1-9-5c5132707d7a4fb6e6a3de1a6719721a create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-0-5528e36b3b0f5b14313898cc45f9c23a create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-1-deb504f4f70fd7db975950c3c47959ee create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-10-fda2e4be738186c0938f92d5072df55a create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-11-9fb177236623d1b62acff28507033436 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-12-99d5ad32bb81640cb284312841b60000 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-13-9dda06e1aae1860bd19eee97703a8217 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-14-19daabdd4c0d403c8781967248d09c53 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-15-812006e1f11e005e5029866d1cf004f6 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-2-bd042746328158822a25d711ffed18dd create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-5-1eb5c694e5a02aa292e24a0849350108 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-6-ab49e0665a80a6b34dadc96f1d18ce26 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-7-fda2e4be738186c0938f92d5072df55a create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-8-9fb177236623d1b62acff28507033436 create mode 100644 sql/hive/src/test/resources/golden/insert2_overwrite_partitions-9-ab49e0665a80a6b34dadc96f1d18ce26 create mode 100644 sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 create mode 100644 sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc create mode 100644 sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 create mode 100644 sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a create mode 100644 sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 create mode 100644 sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 create mode 100644 sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc create mode 100644 sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 create mode 100644 sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 create mode 100644 sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 create mode 100644 sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 create mode 100644 sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f create mode 100644 sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc create mode 100644 sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 create mode 100644 sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d create mode 100644 sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 create mode 100644 sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb create mode 100644 sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e create mode 100644 sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee create mode 100644 sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 create mode 100644 sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 create mode 100644 sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb create mode 100644 sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 create mode 100644 sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f create mode 100644 sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala index 5aaa63bf3b4b4..446d0e0bd7f54 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst +import java.sql.Timestamp + import org.apache.spark.sql.catalyst.expressions.Attribute import org.apache.spark.sql.catalyst.expressions.AttributeReference import org.apache.spark.sql.catalyst.plans.logical.LocalRelation @@ -54,14 +56,15 @@ object ScalaReflection { val TypeRef(_, _, Seq(keyType, valueType)) = t MapType(schemaFor(keyType), schemaFor(valueType)) case t if t <:< typeOf[String] => StringType + case t if t <:< typeOf[Timestamp] => TimestampType + case t if t <:< typeOf[BigDecimal] => DecimalType case t if t <:< definitions.IntTpe => IntegerType case t if t <:< definitions.LongTpe => LongType - case t if t <:< definitions.FloatTpe => FloatType case t if t <:< definitions.DoubleTpe => DoubleType + case t if t <:< definitions.FloatTpe => FloatType case t if t <:< definitions.ShortTpe => ShortType case t if t <:< definitions.ByteTpe => ByteType case t if t <:< definitions.BooleanTpe => BooleanType - case t if t <:< typeOf[BigDecimal] => DecimalType } implicit class CaseClassRelation[A <: Product : TypeTag](data: Seq[A]) { 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 44abe671c07a4..2c4bf1715b646 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst +import java.sql.Timestamp + import scala.language.implicitConversions import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute @@ -72,6 +74,7 @@ package object dsl { def like(other: Expression) = Like(expr, other) def rlike(other: Expression) = RLike(expr, other) + def cast(to: DataType) = Cast(expr, to) def asc = SortOrder(expr, Ascending) def desc = SortOrder(expr, Descending) @@ -84,15 +87,22 @@ package object dsl { def expr = e } + implicit def booleanToLiteral(b: Boolean) = Literal(b) + implicit def byteToLiteral(b: Byte) = Literal(b) + implicit def shortToLiteral(s: Short) = 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 decimalToLiteral(d: BigDecimal) = Literal(d) + implicit def timestampToLiteral(t: Timestamp) = Literal(t) + implicit def binaryToLiteral(a: Array[Byte]) = Literal(a) implicit def symbolToUnresolvedAttribute(s: Symbol) = analysis.UnresolvedAttribute(s.name) implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } + // TODO more implicit class for literal? implicit class DslString(val s: String) extends ImplicitOperators { def expr: Expression = Literal(s) def attr = analysis.UnresolvedAttribute(s) @@ -103,11 +113,38 @@ package object dsl { def expr = attr def attr = analysis.UnresolvedAttribute(s) - /** Creates a new typed attributes of type int */ + /** Creates a new AttributeReference of type boolean */ + def boolean = AttributeReference(s, BooleanType, nullable = false)() + + /** Creates a new AttributeReference of type byte */ + def byte = AttributeReference(s, ByteType, nullable = false)() + + /** Creates a new AttributeReference of type short */ + def short = AttributeReference(s, ShortType, nullable = false)() + + /** Creates a new AttributeReference of type int */ def int = AttributeReference(s, IntegerType, nullable = false)() - /** Creates a new typed attributes of type string */ + /** Creates a new AttributeReference of type long */ + def long = AttributeReference(s, LongType, nullable = false)() + + /** Creates a new AttributeReference of type float */ + def float = AttributeReference(s, FloatType, nullable = false)() + + /** Creates a new AttributeReference of type double */ + def double = AttributeReference(s, DoubleType, nullable = false)() + + /** Creates a new AttributeReference of type string */ def string = AttributeReference(s, StringType, nullable = false)() + + /** Creates a new AttributeReference of type decimal */ + def decimal = AttributeReference(s, DecimalType, nullable = false)() + + /** Creates a new AttributeReference of type timestamp */ + def timestamp = AttributeReference(s, TimestampType, nullable = false)() + + /** Creates a new AttributeReference of type binary */ + def binary = AttributeReference(s, BinaryType, nullable = false)() } implicit class DslAttribute(a: AttributeReference) { 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 c26fc3d0f305f..941b53fe70d23 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.sql.Timestamp + import org.apache.spark.sql.catalyst.types._ /** Cast the child expression to the target data type. */ @@ -26,52 +28,169 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { override def toString = s"CAST($child, $dataType)" type EvaluatedType = Any + + def nullOrCast[T](a: Any, func: T => Any): Any = if(a == null) { + null + } else { + func(a.asInstanceOf[T]) + } - 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) => { - 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)) + // UDFToString + def castToString: Any => Any = child.dataType match { + case BinaryType => nullOrCast[Array[Byte]](_, new String(_, "UTF-8")) + case _ => nullOrCast[Any](_, _.toString) + } + + // BinaryConverter + def castToBinary: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, _.getBytes("UTF-8")) } - @inline - protected def castOrNull[A](a: Any, f: String => A) = - try f(a.asInstanceOf[String]) catch { - case _: java.lang.NumberFormatException => null - } + // UDFToBoolean + def castToBoolean: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, _.length() != 0) + case TimestampType => nullOrCast[Timestamp](_, b => {(b.getTime() != 0 || b.getNanos() != 0)}) + case LongType => nullOrCast[Long](_, _ != 0) + case IntegerType => nullOrCast[Int](_, _ != 0) + case ShortType => nullOrCast[Short](_, _ != 0) + case ByteType => nullOrCast[Byte](_, _ != 0) + case DecimalType => nullOrCast[BigDecimal](_, _ != 0) + case DoubleType => nullOrCast[Double](_, _ != 0) + case FloatType => nullOrCast[Float](_, _ != 0) + } + + // TimestampConverter + def castToTimestamp: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, s => { + // Throw away extra if more than 9 decimal places + val periodIdx = s.indexOf("."); + var n = s + if (periodIdx != -1) { + if (n.length() - periodIdx > 9) { + n = n.substring(0, periodIdx + 10) + } + } + try Timestamp.valueOf(n) catch { case _: java.lang.IllegalArgumentException => null} + }) + case BooleanType => nullOrCast[Boolean](_, b => new Timestamp((if(b) 1 else 0) * 1000)) + case LongType => nullOrCast[Long](_, l => new Timestamp(l * 1000)) + case IntegerType => nullOrCast[Int](_, i => new Timestamp(i * 1000)) + case ShortType => nullOrCast[Short](_, s => new Timestamp(s * 1000)) + case ByteType => nullOrCast[Byte](_, b => new Timestamp(b * 1000)) + // TimestampWritable.decimalToTimestamp + case DecimalType => nullOrCast[BigDecimal](_, d => decimalToTimestamp(d)) + // TimestampWritable.doubleToTimestamp + case DoubleType => nullOrCast[Double](_, d => decimalToTimestamp(d)) + // TimestampWritable.floatToTimestamp + case FloatType => nullOrCast[Float](_, f => decimalToTimestamp(f)) + } + + private def decimalToTimestamp(d: BigDecimal) = { + val seconds = d.longValue() + val bd = (d - seconds) * (1000000000) + val nanos = bd.intValue() + + // Convert to millis + val millis = seconds * 1000 + val t = new Timestamp(millis) + + // remaining fractional portion as nanos + t.setNanos(nanos) + + t + } + + private def timestampToDouble(t: Timestamp) = (t.getSeconds() + t.getNanos().toDouble / 1000) + + def castToLong: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, s => try s.toLong catch { + case _: NumberFormatException => null + }) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toLong) + case DecimalType => nullOrCast[BigDecimal](_, _.toLong) + case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) + } + + def castToInt: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, s => try s.toInt catch { + case _: NumberFormatException => null + }) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toInt) + case DecimalType => nullOrCast[BigDecimal](_, _.toInt) + case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b) + } + + def castToShort: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, s => try s.toShort catch { + case _: NumberFormatException => null + }) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toShort) + case DecimalType => nullOrCast[BigDecimal](_, _.toShort) + case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort + } + + def castToByte: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, s => try s.toByte catch { + case _: NumberFormatException => null + }) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toByte) + case DecimalType => nullOrCast[BigDecimal](_, _.toByte) + case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte + } + + def castToDecimal: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, s => try BigDecimal(s.toDouble) catch { + case _: NumberFormatException => null + }) + case BooleanType => nullOrCast[Boolean](_, b => if(b) BigDecimal(1) else BigDecimal(0)) + case TimestampType => nullOrCast[Timestamp](_, t => BigDecimal(timestampToDouble(t))) + case x: NumericType => b => BigDecimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)) + } + + def castToDouble: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, s => try s.toDouble catch { + case _: NumberFormatException => null + }) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t)) + case DecimalType => nullOrCast[BigDecimal](_, _.toDouble) + case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toDouble(b) + } + + def castToFloat: Any => Any = child.dataType match { + case StringType => nullOrCast[String](_, s => try s.toFloat catch { + case _: NumberFormatException => null + }) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toFloat) + case DecimalType => nullOrCast[BigDecimal](_, _.toFloat) + case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b) + } + + def cast: Any => Any = dataType match { + case StringType => castToString + case BinaryType => castToBinary + case DecimalType => castToDecimal + case TimestampType => castToTimestamp + case BooleanType => castToBoolean + case ByteType => castToByte + case ShortType => castToShort + case IntegerType => castToInt + case FloatType => castToFloat + case LongType => castToLong + case DoubleType => castToDouble + } override def apply(input: Row): Any = { val evaluated = child.apply(input) if (evaluated == null) { null } else { - castingFunction(evaluated) + cast(evaluated) } } } 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 81fd160e00ca1..a3d19525503ba 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 @@ -20,7 +20,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType} +import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType, NativeType} abstract class Expression extends TreeNode[Expression] { self: Product => @@ -86,6 +86,11 @@ abstract class Expression extends TreeNode[Expression] { } } + /** + * Evaluation helper function for 2 Numeric children expressions. Those expressions are supposed + * to be in the same data type, and also the return type. + * Either one of the expressions result is null, the evaluation result should be null. + */ @inline protected final def n2( i: Row, @@ -115,6 +120,11 @@ abstract class Expression extends TreeNode[Expression] { } } + /** + * Evaluation helper function for 2 Fractional children expressions. Those expressions are + * supposed to be in the same data type, and also the return type. + * Either one of the expressions result is null, the evaluation result should be null. + */ @inline protected final def f2( i: Row, @@ -143,6 +153,11 @@ abstract class Expression extends TreeNode[Expression] { } } + /** + * Evaluation helper function for 2 Integral children expressions. Those expressions are + * supposed to be in the same data type, and also the return type. + * Either one of the expressions result is null, the evaluation result should be null. + */ @inline protected final def i2( i: Row, @@ -170,6 +185,43 @@ abstract class Expression extends TreeNode[Expression] { } } } + + /** + * Evaluation helper function for 2 Comparable children expressions. Those expressions are + * supposed to be in the same data type, and the return type should be Integer: + * Negative value: 1st argument less than 2nd argument + * Zero: 1st argument equals 2nd argument + * Positive value: 1st argument greater than 2nd argument + * + * Either one of the expressions result is null, the evaluation result should be null. + */ + @inline + protected final def c2( + i: Row, + e1: Expression, + e2: Expression, + f: ((Ordering[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) + if(evalE1 == null) { + null + } else { + val evalE2 = e2.apply(i) + if (evalE2 == null) { + null + } else { + e1.dataType match { + case i: NativeType => + f.asInstanceOf[(Ordering[i.JvmType], i.JvmType, i.JvmType) => Boolean]( + i.ordering, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) + case other => sys.error(s"Type $other does not support ordered operations") + } + } + } + } } abstract class BinaryExpression extends Expression with trees.BinaryNode[Expression] { 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 b82a12e0f754e..d879b2b5e8ba1 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.sql.Timestamp + import org.apache.spark.sql.catalyst.types._ object Literal { @@ -29,6 +31,9 @@ object Literal { case s: Short => Literal(s, ShortType) case s: String => Literal(s, StringType) case b: Boolean => Literal(b, BooleanType) + case d: BigDecimal => Literal(d, DecimalType) + case t: Timestamp => Literal(t, TimestampType) + case a: Array[Byte] => Literal(a, BinaryType) case null => Literal(null, NullType) } } 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 02fedd16b8d4b..b74809e5ca67d 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 @@ -18,8 +18,9 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.types.{BooleanType, StringType} +import org.apache.spark.sql.catalyst.types.{BooleanType, StringType, TimestampType} object InterpretedPredicate { def apply(expression: Expression): (Row => Boolean) = { @@ -123,70 +124,22 @@ case class Equals(left: Expression, right: Expression) extends BinaryComparison 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(_, _)) - } - } + override def apply(input: Row): Any = c2(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(_, _)) - } - } + override def apply(input: Row): Any = c2(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(_, _)) - } - } + override def apply(input: Row): Any = c2(input, left, right, _.gt(_, _)) } case class GreaterThanOrEqual(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, _.gteq(_, _)) - } - } + override def apply(input: Row): Any = c2(input, left, right, _.gteq(_, _)) } case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) diff --git a/sql/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 index 7a45d1a1b8195..cdeb01a9656f4 100644 --- a/sql/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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.types +import java.sql.Timestamp + import scala.reflect.runtime.universe.{typeTag, TypeTag} import org.apache.spark.sql.catalyst.expressions.Expression @@ -51,6 +53,16 @@ case object BooleanType extends NativeType { val ordering = implicitly[Ordering[JvmType]] } +case object TimestampType extends NativeType { + type JvmType = Timestamp + + @transient lazy val tag = typeTag[JvmType] + + val ordering = new Ordering[JvmType] { + def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) + } +} + abstract class NumericType extends NativeType { // 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 diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 52a205be3e9f4..43876033d327b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import java.sql.Timestamp + import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types._ @@ -191,5 +193,56 @@ class ExpressionEvaluationSuite extends FunSuite { evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) } } + + test("data type casting") { + + val sts = "1970-01-01 00:00:01.0" + val ts = Timestamp.valueOf(sts) + + checkEvaluation("abdef" cast StringType, "abdef") + checkEvaluation("abdef" cast DecimalType, null) + checkEvaluation("abdef" cast TimestampType, null) + checkEvaluation("12.65" cast DecimalType, BigDecimal(12.65)) + + checkEvaluation(Literal(1) cast LongType, 1) + checkEvaluation(Cast(Literal(1) cast TimestampType, LongType), 1) + checkEvaluation(Cast(Literal(BigDecimal(1)) cast TimestampType, DecimalType), 1) + checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) + + checkEvaluation(Cast(Literal(sts) cast TimestampType, StringType), sts) + checkEvaluation(Cast(Literal(ts) cast StringType, TimestampType), ts) + + checkEvaluation(Cast("abdef" cast BinaryType, StringType), "abdef") + + checkEvaluation(Cast(Cast(Cast(Cast( + Cast("5" cast ByteType, ShortType), IntegerType), FloatType), DoubleType), LongType), 5) + checkEvaluation(Cast(Cast(Cast(Cast( + Cast("5" cast ByteType, TimestampType), DecimalType), LongType), StringType), ShortType), 5) + checkEvaluation(Cast(Cast(Cast(Cast( + Cast("5" cast TimestampType, ByteType), DecimalType), LongType), StringType), ShortType), null) + checkEvaluation(Cast(Cast(Cast(Cast( + Cast("5" cast DecimalType, ByteType), TimestampType), LongType), StringType), ShortType), 5) + checkEvaluation(Literal(true) cast IntegerType, 1) + checkEvaluation(Literal(false) cast IntegerType, 0) + checkEvaluation(Cast(Literal(1) cast BooleanType, IntegerType), 1) + checkEvaluation(Cast(Literal(0) cast BooleanType, IntegerType), 0) + checkEvaluation("23" cast DoubleType, 23) + checkEvaluation("23" cast IntegerType, 23) + checkEvaluation("23" cast FloatType, 23) + checkEvaluation("23" cast DecimalType, 23) + checkEvaluation("23" cast ByteType, 23) + checkEvaluation("23" cast ShortType, 23) + checkEvaluation("2012-12-11" cast DoubleType, null) + checkEvaluation(Literal(123) cast IntegerType, 123) + + intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} + } + + test("timestamp") { + val ts1 = new Timestamp(12) + val ts2 = new Timestamp(123) + checkEvaluation(Literal("ab") < Literal("abc"), true) + checkEvaluation(Literal(ts1) < Literal(ts2), true) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 70033a050c78c..65eae3357a21e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql +import java.sql.Timestamp + import org.scalatest.FunSuite import org.apache.spark.sql.test.TestSQLContext._ @@ -31,6 +33,7 @@ case class ReflectData( byteField: Byte, booleanField: Boolean, decimalField: BigDecimal, + timestampField: Timestamp, seqInt: Seq[Int]) case class ReflectBinary(data: Array[Byte]) @@ -38,7 +41,7 @@ case class ReflectBinary(data: Array[Byte]) class ScalaReflectionRelationSuite extends FunSuite { test("query case class RDD") { val data = ReflectData("a", 1, 1L, 1.toFloat, 1.toDouble, 1.toShort, 1.toByte, true, - BigDecimal(1), Seq(1,2,3)) + BigDecimal(1), new Timestamp(12345), Seq(1,2,3)) val rdd = sparkContext.parallelize(data :: Nil) rdd.registerAsTable("reflectData") 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 b2b03bc790fcc..4dac25b3f60e4 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 @@ -300,14 +300,17 @@ object HiveQl { } protected def nodeToDataType(node: Node): DataType = node match { - case Token("TOK_BIGINT", Nil) => IntegerType + case Token("TOK_DECIMAL", Nil) => DecimalType + case Token("TOK_BIGINT", Nil) => LongType case Token("TOK_INT", Nil) => IntegerType - case Token("TOK_TINYINT", Nil) => IntegerType - case Token("TOK_SMALLINT", Nil) => IntegerType + case Token("TOK_TINYINT", Nil) => ByteType + case Token("TOK_SMALLINT", Nil) => ShortType 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_DOUBLE", Nil) => DoubleType + case Token("TOK_TIMESTAMP", Nil) => TimestampType + case Token("TOK_BINARY", Nil) => BinaryType case Token("TOK_LIST", elementType :: Nil) => ArrayType(nodeToDataType(elementType)) case Token("TOK_STRUCT", Token("TOK_TABCOLLIST", fields) :: Nil) => @@ -829,6 +832,8 @@ object HiveQl { Cast(nodeToExpr(arg), BooleanType) case Token("TOK_FUNCTION", Token("TOK_DECIMAL", Nil) :: arg :: Nil) => Cast(nodeToExpr(arg), DecimalType) + case Token("TOK_FUNCTION", Token("TOK_TIMESTAMP", Nil) :: arg :: Nil) => + Cast(nodeToExpr(arg), TimestampType) /* Arithmetic */ case Token("-", child :: Nil) => UnaryMinus(nodeToExpr(child)) diff --git a/sql/hive/src/test/resources/golden/insert1-0-7faa9807151781e4207103aa568e321c b/sql/hive/src/test/resources/golden/insert1-0-7faa9807151781e4207103aa568e321c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-1-91d7b05c9024bff60b55f415cbeacc8b b/sql/hive/src/test/resources/golden/insert1-1-91d7b05c9024bff60b55f415cbeacc8b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-10-64f83491a8fe675ef3a4a9a474ac0439 b/sql/hive/src/test/resources/golden/insert1-10-64f83491a8fe675ef3a4a9a474ac0439 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-11-6f2797b6f81943d3b53b8d247ae8512b b/sql/hive/src/test/resources/golden/insert1-11-6f2797b6f81943d3b53b8d247ae8512b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-12-7a3c0a3f06484c912b9e951d8a2d8ac6 b/sql/hive/src/test/resources/golden/insert1-12-7a3c0a3f06484c912b9e951d8a2d8ac6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-13-42b03f938894fdafc7fff640711a9b2f b/sql/hive/src/test/resources/golden/insert1-13-42b03f938894fdafc7fff640711a9b2f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-14-e021dfb28597811870c03b3242972927 b/sql/hive/src/test/resources/golden/insert1-14-e021dfb28597811870c03b3242972927 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-15-c7fca497a4580b54a0a13b3b72da5d7c b/sql/hive/src/test/resources/golden/insert1-15-c7fca497a4580b54a0a13b3b72da5d7c new file mode 100644 index 0000000000000..5be49cad9a8ba --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1-15-c7fca497a4580b54a0a13b3b72da5d7c @@ -0,0 +1,2 @@ +db2_insert1 +db2_insert2 diff --git a/sql/hive/src/test/resources/golden/insert1-16-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/insert1-16-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-17-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/insert1-17-5528e36b3b0f5b14313898cc45f9c23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-18-16d78fba2d86277bc2f804037cc0a8b4 b/sql/hive/src/test/resources/golden/insert1-18-16d78fba2d86277bc2f804037cc0a8b4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-19-62518ff6810db9cdd8926702192a206b b/sql/hive/src/test/resources/golden/insert1-19-62518ff6810db9cdd8926702192a206b new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-2-3f1de4475930285c3fdbe3a5ccd4e868 b/sql/hive/src/test/resources/golden/insert1-2-3f1de4475930285c3fdbe3a5ccd4e868 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-20-f4dc51ad64bb8662d066a8b9003da3d4 b/sql/hive/src/test/resources/golden/insert1-20-f4dc51ad64bb8662d066a8b9003da3d4 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-21-bb7624250ab556f2d40bfb8d419be487 b/sql/hive/src/test/resources/golden/insert1-21-bb7624250ab556f2d40bfb8d419be487 new file mode 100644 index 0000000000000..1e3637ebc6af2 --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert1-21-bb7624250ab556f2d40bfb8d419be487 @@ -0,0 +1,2 @@ +db1_insert1 +db1_insert2 diff --git a/sql/hive/src/test/resources/golden/insert1-3-89f8a028e32fae213b575b4df4e26e9c b/sql/hive/src/test/resources/golden/insert1-3-89f8a028e32fae213b575b4df4e26e9c new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-4-c7a68c0884785d0f5e62b287eb305d64 b/sql/hive/src/test/resources/golden/insert1-4-c7a68c0884785d0f5e62b287eb305d64 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-5-cb87ee12092fdf05daed82485c32a285 b/sql/hive/src/test/resources/golden/insert1-5-cb87ee12092fdf05daed82485c32a285 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-6-b97ba93a2c9ae671ecfc4fa95c024dda b/sql/hive/src/test/resources/golden/insert1-6-b97ba93a2c9ae671ecfc4fa95c024dda new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-7-a2cd0615b9e79befd9c1842516150a61 b/sql/hive/src/test/resources/golden/insert1-7-a2cd0615b9e79befd9c1842516150a61 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-8-5942e331621fe522fc297844046d2370 b/sql/hive/src/test/resources/golden/insert1-8-5942e331621fe522fc297844046d2370 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert1-9-5c5132707d7a4fb6e6a3de1a6719721a b/sql/hive/src/test/resources/golden/insert1-9-5c5132707d7a4fb6e6a3de1a6719721a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-0-5528e36b3b0f5b14313898cc45f9c23a b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-0-5528e36b3b0f5b14313898cc45f9c23a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-1-deb504f4f70fd7db975950c3c47959ee b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-1-deb504f4f70fd7db975950c3c47959ee new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-10-fda2e4be738186c0938f92d5072df55a b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-10-fda2e4be738186c0938f92d5072df55a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-11-9fb177236623d1b62acff28507033436 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-11-9fb177236623d1b62acff28507033436 new file mode 100644 index 0000000000000..01f2b7063f91b --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-11-9fb177236623d1b62acff28507033436 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +98 val_98 +97 val_97 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-12-99d5ad32bb81640cb284312841b60000 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-12-99d5ad32bb81640cb284312841b60000 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-13-9dda06e1aae1860bd19eee97703a8217 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-13-9dda06e1aae1860bd19eee97703a8217 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-14-19daabdd4c0d403c8781967248d09c53 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-14-19daabdd4c0d403c8781967248d09c53 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-15-812006e1f11e005e5029866d1cf004f6 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-15-812006e1f11e005e5029866d1cf004f6 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-2-bd042746328158822a25d711ffed18dd b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-2-bd042746328158822a25d711ffed18dd new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-3-b7aaedd7d624af4e48637ff1acabe485 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-4-dece2650bf0615e566cd6c84181ce026 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-5-1eb5c694e5a02aa292e24a0849350108 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-5-1eb5c694e5a02aa292e24a0849350108 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-6-ab49e0665a80a6b34dadc96f1d18ce26 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-6-ab49e0665a80a6b34dadc96f1d18ce26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-7-fda2e4be738186c0938f92d5072df55a b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-7-fda2e4be738186c0938f92d5072df55a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-8-9fb177236623d1b62acff28507033436 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-8-9fb177236623d1b62acff28507033436 new file mode 100644 index 0000000000000..01f2b7063f91b --- /dev/null +++ b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-8-9fb177236623d1b62acff28507033436 @@ -0,0 +1,5 @@ +98 val_98 +98 val_98 +98 val_98 +97 val_97 +97 val_97 diff --git a/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-9-ab49e0665a80a6b34dadc96f1d18ce26 b/sql/hive/src/test/resources/golden/insert2_overwrite_partitions-9-ab49e0665a80a6b34dadc96f1d18ce26 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 b/sql/hive/src/test/resources/golden/load_binary_data-0-491edd0c42ceb79e799ba50555bc8c15 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc b/sql/hive/src/test/resources/golden/load_binary_data-1-5d72f8449b69df3c08e3f444f09428bc new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 b/sql/hive/src/test/resources/golden/load_binary_data-2-242b1655c7e7325ee9f26552ea8fc25 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 b/sql/hive/src/test/resources/golden/load_binary_data-3-2a72df8d3e398d0963ef91162ce7d268 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-0-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-1-2b1df88619e34f221d39598b5cd73283 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-10-60eadbb52f8857830a3034952c631ace new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-11-dbe79f90862dc5c6cc4a4fa4b4b6c655 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-12-60018cae9a0476dc6a0ab4264310edb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-2-7562d4fee13f3ba935a2e824f86a4224 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-3-bdb30a5d6887ee4fb089f8676313eafd @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-4-10713b30ecb3c88acdd775bf9628c38c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-5-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-6-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-7-ad4ddb5c5d6b994f4dba35f6162b6a9f new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-8-f9dd797f1c90e2108cfee585f443c132 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook-9-22fdd8380f2652de2492b34a425d46d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-0-7a9e67189d3d4151f23b12c22bde06b5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-1-86a409d8b868dc5f1a3bd1e04c2bc28c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-10-22fdd8380f2652de2492b34a425d46d7 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-11-60eadbb52f8857830a3034952c631ace new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-12-dbe79f90862dc5c6cc4a4fa4b4b6c655 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-13-60018cae9a0476dc6a0ab4264310edb5 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-2-2b1df88619e34f221d39598b5cd73283 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-3-7562d4fee13f3ba935a2e824f86a4224 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-4-bdb30a5d6887ee4fb089f8676313eafd @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-5-10713b30ecb3c88acdd775bf9628c38c @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-6-bab89dfffa77258e34a595e0e79986e3 @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc new file mode 100644 index 0000000000000..573541ac9702d --- /dev/null +++ b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-7-6f53d5613262d393d82d159ec5dc16dc @@ -0,0 +1 @@ +0 diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-8-7a45282169e5a15d70ae0afb9e67ec9a new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 b/sql/hive/src/test/resources/golden/sample_islocalmode_hook_hadoop20-9-f9dd797f1c90e2108cfee585f443c132 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 b/sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_comparison-0-48751533b44ea9e8ac3131767c2fed05 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc b/sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_comparison-1-60557e7bd2822c89fa8b076a9d0520fc @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 b/sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_comparison-2-f96a9d88327951bd93f672dc2463ecd4 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 b/sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_comparison-3-13e17ed811165196416f777cbc162592 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 b/sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_comparison-4-4fa8a36edbefde4427c2ab2cf30e6399 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 b/sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_comparison-5-7e4fb6e8ba01df422e4c67e06a0c8453 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f b/sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_comparison-6-8c8e73673a950f6b3d960b08fcea076f @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc b/sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_comparison-7-510c0a2a57dc5df8588bd13c4152f8bc @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 b/sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/timestamp_comparison-8-659d5b1ae8200f13f265270e52a3dd65 @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d b/sql/hive/src/test/resources/golden/type_cast_1-0-60ea21e6e7d054a65f959fc89acf1b3d new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 b/sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 new file mode 100644 index 0000000000000..7ed6ff82de6bc --- /dev/null +++ b/sql/hive/src/test/resources/golden/type_cast_1-1-53a667981ad567b2ab977f67d65c5825 @@ -0,0 +1 @@ +5 diff --git a/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb b/sql/hive/src/test/resources/golden/udf_printf-0-e86d559aeb84a4cc017a103182c22bfb new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e b/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e new file mode 100644 index 0000000000000..1635ff88dd768 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_printf-1-19c61fce27310ab2590062d643f7b26e @@ -0,0 +1 @@ +printf(String format, Obj... args) - function that can format strings according to printf-style format strings diff --git a/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee b/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee new file mode 100644 index 0000000000000..62440ee68e145 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_printf-2-25aa6950cae2bb781c336378f63ceaee @@ -0,0 +1,4 @@ +printf(String format, Obj... args) - function that can format strings according to printf-style format strings +Example: + > SELECT printf("Hello World %d %s", 100, "days")FROM src LIMIT 1; + "Hello World 100 days" diff --git a/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 b/sql/hive/src/test/resources/golden/udf_printf-3-9c568a0473888396bd46507e8b330c36 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 b/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 new file mode 100644 index 0000000000000..39cb945991403 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_printf-4-91728e546b450bdcbb05ef30f13be475 @@ -0,0 +1 @@ +Hello World 100 days diff --git a/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb b/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb new file mode 100644 index 0000000000000..04bf5e552a576 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_printf-5-3141a0421605b091ee5a9e99d7d605fb @@ -0,0 +1 @@ +All Type Test: false, A, 15000, 1.234000e+01, +27183.2401, 2300.41, 32, corret, 0x1.002p8 diff --git a/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 b/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 new file mode 100644 index 0000000000000..2e9f7509968a3 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_printf-6-ec37b73012f3cbbbc0422744b0db8294 @@ -0,0 +1 @@ +Color red, String Null: null, number1 123456, number2 00089, Integer Null: null, hex 0xff, float 3.14 Double Null: null diff --git a/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 b/sql/hive/src/test/resources/golden/udf_printf-7-5769f3a5b3300ca1d8b861229e976126 new file mode 100644 index 0000000000000..e69de29bb2d1d diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 b/sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-10-51822ac740629bebd81d2abda6e1144 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b b/sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-11-441306cae24618c49ec63445a31bf16b @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 b/sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-12-bfcc534e73e320a1cfad9c584678d870 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d b/sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-13-a2bddaa5db1841bb4617239b9f17a06d @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 b/sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-14-773801b833cf72d35016916b786275b5 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 b/sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-15-4071ed0ff57b53963d5ee662fa9db0b0 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 b/sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-16-6b441df08afdc0c6c4a82670997dabb5 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df b/sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-17-85342c694d7f35e7eedb24e850d0c7df @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 b/sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-18-fcd7af0e71d3e2d934239ba606e3ed87 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 b/sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-19-dcdb12fe551aa68a56921822f5d1a343 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 b/sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-20-131900d39d9a20b431731a32fb9715f8 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a b/sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-21-a5e28f4eb819e5a5e292e279f2990a7a @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d b/sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-22-93278c10d642fa242f303d89b3b1961d @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 b/sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-23-828558020ce907ffa7e847762a5e2358 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da b/sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-24-e8ca597d87932af16c0cf29d662e92da @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 b/sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-25-86245727f90de9ce65a12c97a03a5635 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 b/sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-26-552d7ec5a4e0c93dc59a61973e2d63a2 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 b/sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 new file mode 100644 index 0000000000000..7951defec192a --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-27-b61509b01b2fe3e7e4b72fedc74ff4f9 @@ -0,0 +1 @@ +NULL diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f b/sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f new file mode 100644 index 0000000000000..27ba77ddaf615 --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-8-37229f303635a030f6cab20e0381f51f @@ -0,0 +1 @@ +true diff --git a/sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 b/sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 new file mode 100644 index 0000000000000..c508d5366f70b --- /dev/null +++ b/sql/hive/src/test/resources/golden/udf_to_boolean-9-be623247e4dbf119b43458b72d1be017 @@ -0,0 +1 @@ +false diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index f74b0fbb97c83..f76e16bc1afc5 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -42,6 +42,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "bucket_num_reducers", "column_access_stats", "concatenate_inherit_table_location", + "describe_pretty", + "describe_syntax", + "orc_ends_with_nulls", // Setting a default property does not seem to get reset and thus changes the answer for many // subsequent tests. @@ -80,7 +83,6 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "index_auto_update", "index_auto_self_join", "index_stale.*", - "type_cast_1", "index_compression", "index_bitmap_compression", "index_auto_multiple", @@ -237,9 +239,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "compute_stats_binary", "compute_stats_boolean", "compute_stats_double", - "compute_stats_table", + "compute_stats_empty_table", "compute_stats_long", "compute_stats_string", + "compute_stats_table", "convert_enum_to_string", "correlationoptimizer11", "correlationoptimizer15", @@ -266,8 +269,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "desc_non_existent_tbl", "describe_comment_indent", "describe_database_json", - "describe_pretty", - "describe_syntax", + "describe_formatted_view_partitioned", + "describe_formatted_view_partitioned_json", "describe_table_json", "diff_part_input_formats", "disable_file_format_check", @@ -339,8 +342,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "input11_limit", "input12", "input12_hadoop20", + "input14", "input19", "input1_limit", + "input21", "input22", "input23", "input24", @@ -355,6 +360,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "input7", "input8", "input9", + "inputddl4", + "inputddl7", + "inputddl8", "input_limit", "input_part0", "input_part1", @@ -368,9 +376,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "input_part7", "input_part8", "input_part9", - "inputddl4", - "inputddl7", - "inputddl8", + "input_testsequencefile", + "insert1", + "insert2_overwrite_partitions", "insert_compressed", "join0", "join1", @@ -385,6 +393,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "join17", "join18", "join19", + "join_1to1", "join2", "join20", "join21", @@ -400,6 +409,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "join30", "join31", "join32", + "join32_lessSize", "join33", "join34", "join35", @@ -415,13 +425,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "join7", "join8", "join9", - "join_1to1", "join_array", "join_casesensitive", "join_empty", "join_filters", "join_hive_626", + "join_map_ppr", "join_nulls", + "join_rc", "join_reorder2", "join_reorder3", "join_reorder4", @@ -435,22 +446,32 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "literal_string", "load_dyn_part7", "load_file_with_space_in_the_name", + "loadpart1", "louter_join_ppr", "mapjoin_distinct", "mapjoin_mapjoin", "mapjoin_subquery", "mapjoin_subquery2", "mapjoin_test_outer", + "mapreduce1", + "mapreduce2", "mapreduce3", + "mapreduce4", + "mapreduce5", + "mapreduce6", "mapreduce7", + "mapreduce8", "merge1", "merge2", "mergejoins", "mergejoins_mixed", + "multigroupby_singlemr", + "multi_insert_gby", + "multi_insert_gby3", + "multi_insert_lateral_view", + "multi_join_union", "multiMapJoin1", "multiMapJoin2", - "multi_join_union", - "multigroupby_singlemr", "noalias_subq1", "nomore_ambiguous_table_col", "nonblock_op_deduplicate", @@ -466,16 +487,30 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "nullinput2", "nullscript", "optional_outer", + "orc_dictionary_threshold", + "orc_empty_files", "order", "order2", "outer_join_ppr", + "parallel", + "parenthesis_star_by", + "partcols1", "part_inherit_tbl_props", "part_inherit_tbl_props_empty", "part_inherit_tbl_props_with_star", "partition_schema1", + "partition_serde_format", "partition_varchar1", + "partition_wise_fileformat4", + "partition_wise_fileformat5", + "partition_wise_fileformat6", + "partition_wise_fileformat7", + "partition_wise_fileformat9", "plan_json", "ppd1", + "ppd2", + "ppd_clusterby", + "ppd_constant_expr", "ppd_constant_where", "ppd_gby", "ppd_gby2", @@ -491,6 +526,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "ppd_outer_join5", "ppd_random", "ppd_repeated_alias", + "ppd_transform", "ppd_udf_col", "ppd_union", "ppr_allchildsarenull", @@ -503,7 +539,15 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "query_with_semi", "quote1", "quote2", + "rcfile_columnar", + "rcfile_lazydecompress", + "rcfile_null_value", + "rcfile_toleratecorruptions", + "rcfile_union", + "reduce_deduplicate", + "reduce_deduplicate_exclude_gby", "reduce_deduplicate_exclude_join", + "reducesink_dedup", "rename_column", "router_join_ppr", "select_as_omitted", @@ -531,6 +575,8 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "smb_mapjoin_3", "smb_mapjoin_4", "smb_mapjoin_5", + "smb_mapjoin_6", + "smb_mapjoin_7", "smb_mapjoin_8", "sort", "sort_merge_join_desc_1", @@ -541,21 +587,27 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "sort_merge_join_desc_6", "sort_merge_join_desc_7", "stats0", + "stats_aggregator_error_1", "stats_empty_partition", + "stats_publisher_error_1", "subq2", "tablename_with_select", + "timestamp_comparison", "touch", + "transform_ppr1", + "transform_ppr2", + "type_cast_1", "type_widening", "udaf_collect_set", "udaf_corr", "udaf_covar_pop", "udaf_covar_samp", + "udaf_histogram_numeric", + "udf_10_trims", "udf2", "udf6", + "udf8", "udf9", - "udf_10_trims", - "udf_E", - "udf_PI", "udf_abs", "udf_acos", "udf_add", @@ -585,13 +637,14 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "udf_cos", "udf_count", "udf_date_add", - "udf_date_sub", "udf_datediff", + "udf_date_sub", "udf_day", "udf_dayofmonth", "udf_degrees", "udf_div", "udf_double", + "udf_E", "udf_exp", "udf_field", "udf_find_in_set", @@ -631,6 +684,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "udf_nvl", "udf_or", "udf_parse_url", + "udf_PI", "udf_positive", "udf_pow", "udf_power", @@ -671,9 +725,9 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "udf_trim", "udf_ucase", "udf_upper", + "udf_variance", "udf_var_pop", "udf_var_samp", - "udf_variance", "udf_weekofyear", "udf_when", "udf_xpath", @@ -703,8 +757,10 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "union27", "union28", "union29", + "union3", "union30", "union31", + "union33", "union34", "union4", "union5", @@ -714,6 +770,7 @@ class HiveCompatibilitySuite extends HiveQueryFileTest { "union9", "union_lateralview", "union_ppr", + "union_remove_11", "union_remove_3", "union_remove_6", "union_script", From c1ea3afb516c204925259f0928dfb17d0fa89621 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Thu, 3 Apr 2014 15:42:17 -0700 Subject: [PATCH 186/397] Spark 1162 Implemented takeOrdered in pyspark. Since python does not have a library for max heap and usual tricks like inverting values etc.. does not work for all cases. We have our own implementation of max heap. Author: Prashant Sharma Closes #97 from ScrapCodes/SPARK-1162/pyspark-top-takeOrdered2 and squashes the following commits: 35f86ba [Prashant Sharma] code review 2b1124d [Prashant Sharma] fixed tests e8a08e2 [Prashant Sharma] Code review comments. 49e6ba7 [Prashant Sharma] SPARK-1162 added takeOrdered to pyspark --- python/pyspark/rdd.py | 107 ++++++++++++++++++++++++++++++++++++++++-- 1 file changed, 102 insertions(+), 5 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 019c249699c2d..9943296b927dc 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -29,7 +29,7 @@ from tempfile import NamedTemporaryFile from threading import Thread import warnings -from heapq import heappush, heappop, heappushpop +import heapq from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long @@ -41,9 +41,9 @@ from py4j.java_collections import ListConverter, MapConverter - __all__ = ["RDD"] + def _extract_concise_traceback(): """ This function returns the traceback info for a callsite, returns a dict @@ -91,6 +91,73 @@ def __exit__(self, type, value, tb): if _spark_stack_depth == 0: self._context._jsc.setCallSite(None) +class MaxHeapQ(object): + """ + An implementation of MaxHeap. + >>> import pyspark.rdd + >>> heap = pyspark.rdd.MaxHeapQ(5) + >>> [heap.insert(i) for i in range(10)] + [None, None, None, None, None, None, None, None, None, None] + >>> sorted(heap.getElements()) + [0, 1, 2, 3, 4] + >>> heap = pyspark.rdd.MaxHeapQ(5) + >>> [heap.insert(i) for i in range(9, -1, -1)] + [None, None, None, None, None, None, None, None, None, None] + >>> sorted(heap.getElements()) + [0, 1, 2, 3, 4] + >>> heap = pyspark.rdd.MaxHeapQ(1) + >>> [heap.insert(i) for i in range(9, -1, -1)] + [None, None, None, None, None, None, None, None, None, None] + >>> heap.getElements() + [0] + """ + + def __init__(self, maxsize): + # we start from q[1], this makes calculating children as trivial as 2 * k + self.q = [0] + self.maxsize = maxsize + + def _swim(self, k): + while (k > 1) and (self.q[k/2] < self.q[k]): + self._swap(k, k/2) + k = k/2 + + def _swap(self, i, j): + t = self.q[i] + self.q[i] = self.q[j] + self.q[j] = t + + def _sink(self, k): + N = self.size() + while 2 * k <= N: + j = 2 * k + # Here we test if both children are greater than parent + # if not swap with larger one. + if j < N and self.q[j] < self.q[j + 1]: + j = j + 1 + if(self.q[k] > self.q[j]): + break + self._swap(k, j) + k = j + + def size(self): + return len(self.q) - 1 + + def insert(self, value): + if (self.size()) < self.maxsize: + self.q.append(value) + self._swim(self.size()) + else: + self._replaceRoot(value) + + def getElements(self): + return self.q[1:] + + def _replaceRoot(self, value): + if(self.q[1] > value): + self.q[1] = value + self._sink(1) + class RDD(object): """ A Resilient Distributed Dataset (RDD), the basic abstraction in Spark. @@ -696,16 +763,16 @@ def top(self, num): Note: It returns the list sorted in descending order. >>> sc.parallelize([10, 4, 2, 12, 3]).top(1) [12] - >>> sc.parallelize([2, 3, 4, 5, 6]).cache().top(2) + >>> sc.parallelize([2, 3, 4, 5, 6], 2).cache().top(2) [6, 5] """ def topIterator(iterator): q = [] for k in iterator: if len(q) < num: - heappush(q, k) + heapq.heappush(q, k) else: - heappushpop(q, k) + heapq.heappushpop(q, k) yield q def merge(a, b): @@ -713,6 +780,36 @@ def merge(a, b): return sorted(self.mapPartitions(topIterator).reduce(merge), reverse=True) + def takeOrdered(self, num, key=None): + """ + Get the N elements from a RDD ordered in ascending order or as specified + by the optional key function. + + >>> sc.parallelize([10, 1, 2, 9, 3, 4, 5, 6, 7]).takeOrdered(6) + [1, 2, 3, 4, 5, 6] + >>> sc.parallelize([10, 1, 2, 9, 3, 4, 5, 6, 7], 2).takeOrdered(6, key=lambda x: -x) + [10, 9, 7, 6, 5, 4] + """ + + def topNKeyedElems(iterator, key_=None): + q = MaxHeapQ(num) + for k in iterator: + if key_ != None: + k = (key_(k), k) + q.insert(k) + yield q.getElements() + + def unKey(x, key_=None): + if key_ != None: + x = [i[1] for i in x] + return x + + def merge(a, b): + return next(topNKeyedElems(a + b)) + result = self.mapPartitions(lambda i: topNKeyedElems(i, key)).reduce(merge) + return sorted(unKey(result, key), key=key) + + def take(self, num): """ Take the first num elements of the RDD. From b8f534196f9a8c99f75728a06e62282d139dee28 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 3 Apr 2014 15:45:34 -0700 Subject: [PATCH 187/397] [SQL] SPARK-1333 First draft of java API WIP: Some work remains... * [x] Hive support * [x] Tests * [x] Update docs Feedback welcome! Author: Michael Armbrust Closes #248 from marmbrus/javaSchemaRDD and squashes the following commits: b393913 [Michael Armbrust] @srowen 's java style suggestions. f531eb1 [Michael Armbrust] Address matei's comments. 33a1b1a [Michael Armbrust] Ignore JavaHiveSuite. 822f626 [Michael Armbrust] improve docs. ab91750 [Michael Armbrust] Improve Java SQL API: * Change JavaRow => Row * Add support for querying RDDs of JavaBeans * Docs * Tests * Hive support 0b859c8 [Michael Armbrust] First draft of java API. --- docs/sql-programming-guide.md | 204 ++++++++++++++++-- .../spark/examples/sql/JavaSparkSQL.java | 99 +++++++++ .../org/apache/spark/sql/SchemaRDD.scala | 42 +--- .../org/apache/spark/sql/SchemaRDDLike.scala | 66 ++++++ .../spark/sql/api/java/JavaSQLContext.scala | 100 +++++++++ .../spark/sql/api/java/JavaSchemaRDD.scala | 48 +++++ .../org/apache/spark/sql/api/java/Row.scala | 93 ++++++++ .../spark/sql/api/java/JavaSQLSuite.scala | 53 +++++ .../apache/spark/sql/hive/HiveContext.scala | 12 ++ .../sql/hive/api/java/JavaHiveContext.scala | 42 ++++ .../sql/hive/api/java/JavaHiveSuite.scala | 41 ++++ 11 files changed, 750 insertions(+), 50 deletions(-) create mode 100644 examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala create mode 100644 sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala create mode 100644 sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala create mode 100644 sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index b6f21a5dc62c3..f849716f7a48f 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -8,6 +8,10 @@ title: Spark SQL Programming Guide {:toc} # Overview + +
    +
    + Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, [SchemaRDD](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed @@ -18,11 +22,27 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac **All of the examples on this page use sample data included in the Spark distribution and can be run in the spark-shell.** +
    + +
    +Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using +Spark. At the core of this component is a new type of RDD, +[JavaSchemaRDD](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed +[Row](api/sql/catalyst/index.html#org.apache.spark.sql.api.java.Row) objects along with +a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table +in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, parquet +file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). +
    +
    + *************************************************************************************************** # Getting Started -The entry point into all relational functionallity in Spark is the +
    +
    + +The entry point into all relational functionality in Spark is the [SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its decendents. To create a basic SQLContext, all you need is a SparkContext. @@ -34,8 +54,30 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext._ {% endhighlight %} +
    + +
    + +The entry point into all relational functionality in Spark is the +[JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one +of its decendents. To create a basic JavaSQLContext, all you need is a JavaSparkContext. + +{% highlight java %} +JavaSparkContext ctx = ...; // An existing JavaSparkContext. +JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx); +{% endhighlight %} + +
    + +
    + ## Running SQL on RDDs -One type of table that is supported by Spark SQL is an RDD of Scala case classetees. The case class + +
    + +
    + +One type of table that is supported by Spark SQL is an RDD of Scala case classes. The case class defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be @@ -60,7 +102,83 @@ val teenagers = sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") teenagers.map(t => "Name: " + t(0)).collect().foreach(println) {% endhighlight %} -**Note that Spark SQL currently uses a very basic SQL parser, and the keywords are case sensitive.** +
    + +
    + +One type of table that is supported by Spark SQL is an RDD of [JavaBeans](http://stackoverflow.com/questions/3295496/what-is-a-javabean-exactly). The BeanInfo +defines the schema of the table. Currently, Spark SQL does not support JavaBeans that contain +nested or contain complex types such as Lists or Arrays. You can create a JavaBean by creating a +class that implements Serializable and has getters and setters for all of its fields. + +{% highlight java %} + +public static class Person implements Serializable { + private String name; + private int age; + + String getName() { + return name; + } + + void setName(String name) { + this.name = name; + } + + int getAge() { + return age; + } + + void setAge(int age) { + this.age = age; + } +} + +{% endhighlight %} + + +A schema can be applied to an existing RDD by calling `applySchema` and providing the Class object +for the JavaBean. + +{% highlight java %} +JavaSQLContext ctx = new org.apache.spark.sql.api.java.JavaSQLContext(sc) + +// Load a text file and convert each line to a JavaBean. +JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( + new Function() { + public Person call(String line) throws Exception { + String[] parts = line.split(","); + + Person person = new Person(); + person.setName(parts[0]); + person.setAge(Integer.parseInt(parts[1].trim())); + + return person; + } + }); + +// Apply a schema to an RDD of JavaBeans and register it as a table. +JavaSchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class); +schemaPeople.registerAsTable("people"); + +// SQL can be run over RDDs that have been registered as tables. +JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + +// The results of SQL queries are SchemaRDDs and support all the normal RDD operations. +// The columns of a row in the result can be accessed by ordinal. +List teenagerNames = teenagers.map(new Function() { + public String call(Row row) { + return "Name: " + row.getString(0); + } +}).collect(); + +{% endhighlight %} + +
    + +
    + +**Note that Spark SQL currently uses a very basic SQL parser.** Users that want a more complete dialect of SQL should look at the HiveQL support provided by `HiveContext`. @@ -70,17 +188,21 @@ Parquet is a columnar format that is supported by many other data processing sys provides support for both reading and writing parquet files that automatically preserves the schema of the original data. Using the data from the above example: +
    + +
    + {% highlight scala %} val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext._ -val people: RDD[Person] // An RDD of case class objects, from the previous example. +val people: RDD[Person] = ... // An RDD of case class objects, from the previous example. // The RDD is implicitly converted to a SchemaRDD, allowing it to be stored using parquet. people.saveAsParquetFile("people.parquet") // Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. -// The result of loading a parquet file is also a SchemaRDD. +// The result of loading a parquet file is also a JavaSchemaRDD. val parquetFile = sqlContext.parquetFile("people.parquet") //Parquet files can also be registered as tables and then used in SQL statements. @@ -89,15 +211,43 @@ val teenagers = sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19" teenagers.collect().foreach(println) {% endhighlight %} +
    + +
    + +{% highlight java %} + +JavaSchemaRDD schemaPeople = ... // The JavaSchemaRDD from the previous example. + +// JavaSchemaRDDs can be saved as parquet files, maintaining the schema information. +schemaPeople.saveAsParquetFile("people.parquet"); + +// Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. +// The result of loading a parquet file is also a JavaSchemaRDD. +JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet"); + +//Parquet files can also be registered as tables and then used in SQL statements. +parquetFile.registerAsTable("parquetFile"); +JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + + +{% endhighlight %} + +
    + +
    + ## Writing Language-Integrated Relational Queries +**Language-Integrated queries are currently only supported in Scala.** + Spark SQL also supports a domain specific language for writing queries. Once again, using the data from the above examples: {% highlight scala %} val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext._ -val people: RDD[Person] // An RDD of case class objects, from the first example. +val people: RDD[Person] = ... // An RDD of case class objects, from the first example. // The following is the same as 'SELECT name FROM people WHERE age >= 10 AND age <= 19' val teenagers = people.where('age >= 10).where('age <= 19).select('name) @@ -114,14 +264,17 @@ evaluated by the SQL execution engine. A full list of the functions supported c Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. -In order to use Hive you must first run '`sbt/sbt hive/assembly`'. This command builds a new assembly -jar that includes Hive. When this jar is present, Spark will use the Hive -assembly instead of the normal Spark assembly. Note that this Hive assembly jar must also be present +In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`'. This command builds a new assembly +jar that includes Hive. Note that this Hive assembly jar must also be present on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries (SerDes) in order to acccess data stored in Hive. Configuration of Hive is done by placing your `hive-site.xml` file in `conf/`. +
    + +
    + When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and adds support for finding tables in in the MetaStore and writing queries using HiveQL. Users who do not have an existing Hive deployment can also experiment with the `LocalHiveContext`, @@ -135,9 +288,34 @@ val hiveContext = new org.apache.spark.sql.hive.HiveContext(sc) // Importing the SQL context gives access to all the public SQL functions and implicit conversions. import hiveContext._ -sql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") -sql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") +hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") // Queries are expressed in HiveQL -sql("SELECT key, value FROM src").collect().foreach(println) -{% endhighlight %} \ No newline at end of file +hql("FROM src SELECT key, value").collect().foreach(println) +{% endhighlight %} + +
    + +
    + +When working with Hive one must construct a `JavaHiveContext`, which inherits from `JavaSQLContext`, and +adds support for finding tables in in the MetaStore and writing queries using HiveQL. In addition to +the `sql` method a `JavaHiveContext` also provides an `hql` methods, which allows queries to be +expressed in HiveQL. + +{% highlight java %} +JavaSparkContext ctx = ...; // An existing JavaSparkContext. +JavaHiveContext hiveCtx = new org.apache.spark.sql.hive.api.java.HiveContext(ctx); + +hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)"); +hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src"); + +// Queries are expressed in HiveQL. +Row[] results = hiveCtx.hql("FROM src SELECT key, value").collect(); + +{% endhighlight %} + +
    + +
    diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java new file mode 100644 index 0000000000000..e8e63d2745692 --- /dev/null +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -0,0 +1,99 @@ +/* + * 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.examples.sql; + +import java.io.Serializable; +import java.util.List; + +import org.apache.spark.api.java.JavaRDD; +import org.apache.spark.api.java.JavaSparkContext; +import org.apache.spark.api.java.function.Function; +import org.apache.spark.api.java.function.VoidFunction; + +import org.apache.spark.sql.api.java.JavaSQLContext; +import org.apache.spark.sql.api.java.JavaSchemaRDD; +import org.apache.spark.sql.api.java.Row; + +public class JavaSparkSQL { + public static class Person implements Serializable { + private String name; + private int age; + + String getName() { + return name; + } + + void setName(String name) { + this.name = name; + } + + int getAge() { + return age; + } + + void setAge(int age) { + this.age = age; + } + } + + public static void main(String[] args) throws Exception { + JavaSparkContext ctx = new JavaSparkContext("local", "JavaSparkSQL", + System.getenv("SPARK_HOME"), JavaSparkContext.jarOfClass(JavaSparkSQL.class)); + JavaSQLContext sqlCtx = new JavaSQLContext(ctx); + + // Load a text file and convert each line to a Java Bean. + JavaRDD people = ctx.textFile("examples/src/main/resources/people.txt").map( + new Function() { + public Person call(String line) throws Exception { + String[] parts = line.split(","); + + Person person = new Person(); + person.setName(parts[0]); + person.setAge(Integer.parseInt(parts[1].trim())); + + return person; + } + }); + + // Apply a schema to an RDD of Java Beans and register it as a table. + JavaSchemaRDD schemaPeople = sqlCtx.applySchema(people, Person.class); + schemaPeople.registerAsTable("people"); + + // SQL can be run over RDDs that have been registered as tables. + JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19"); + + // The results of SQL queries are SchemaRDDs and support all the normal RDD operations. + // The columns of a row in the result can be accessed by ordinal. + List teenagerNames = teenagers.map(new Function() { + public String call(Row row) { + return "Name: " + row.getString(0); + } + }).collect(); + + // JavaSchemaRDDs can be saved as parquet files, maintaining the schema information. + schemaPeople.saveAsParquetFile("people.parquet"); + + // Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. + // The result of loading a parquet file is also a JavaSchemaRDD. + JavaSchemaRDD parquetFile = sqlCtx.parquetFile("people.parquet"); + + //Parquet files can also be registered as tables and then used in SQL statements. + parquetFile.registerAsTable("parquetFile"); + JavaSchemaRDD teenagers2 = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 770cabcb31d13..a62cb8aa1321f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -17,13 +17,13 @@ package org.apache.spark.sql +import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType -import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} /** * ALPHA COMPONENT @@ -92,23 +92,10 @@ import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} */ class SchemaRDD( @transient val sqlContext: SQLContext, - @transient val logicalPlan: LogicalPlan) - extends RDD[Row](sqlContext.sparkContext, Nil) { + @transient protected[spark] val logicalPlan: LogicalPlan) + extends RDD[Row](sqlContext.sparkContext, Nil) with SchemaRDDLike { - /** - * A lazily computed query execution workflow. All other RDD operations are passed - * through to the RDD that is produced by this workflow. - * - * We want this to be lazy because invoking the whole query optimization pipeline can be - * expensive. - */ - @transient - protected[spark] lazy val queryExecution = sqlContext.executePlan(logicalPlan) - - override def toString = - s"""${super.toString} - |== Query Plan == - |${queryExecution.executedPlan}""".stripMargin.trim + def baseSchemaRDD = this // ========================================================================================= // RDD functions: Copy the interal row representation so we present immutable data to users. @@ -312,31 +299,12 @@ class SchemaRDD( sqlContext, InsertIntoTable(UnresolvedRelation(None, tableName), Map.empty, logicalPlan, overwrite)) - /** - * Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that - * are written out using this method can be read back in as a SchemaRDD using the ``function - * - * @group schema - */ - def saveAsParquetFile(path: String): Unit = { - sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd - } - - /** - * Registers this RDD as a temporary table using the given name. The lifetime of this temporary - * table is tied to the [[SQLContext]] that was used to create this SchemaRDD. - * - * @group schema - */ - def registerAsTable(tableName: String): Unit = { - sqlContext.registerRDDAsTable(this, tableName) - } - /** * Returns this RDD as a SchemaRDD. * @group schema */ def toSchemaRDD = this + /** FOR INTERNAL USE ONLY */ def analyze = sqlContext.analyzer(logicalPlan) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala new file mode 100644 index 0000000000000..840803a52c1cf --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -0,0 +1,66 @@ +/* +* 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.apache.spark.sql.catalyst.plans.logical._ + +/** + * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java) + */ +trait SchemaRDDLike { + @transient val sqlContext: SQLContext + @transient protected[spark] val logicalPlan: LogicalPlan + + private[sql] def baseSchemaRDD: SchemaRDD + + /** + * A lazily computed query execution workflow. All other RDD operations are passed + * through to the RDD that is produced by this workflow. + * + * We want this to be lazy because invoking the whole query optimization pipeline can be + * expensive. + */ + @transient + protected[spark] lazy val queryExecution = sqlContext.executePlan(logicalPlan) + + override def toString = + s"""${super.toString} + |== Query Plan == + |${queryExecution.executedPlan}""".stripMargin.trim + + + /** + * Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that + * are written out using this method can be read back in as a SchemaRDD using the ``function + * + * @group schema + */ + def saveAsParquetFile(path: String): Unit = { + sqlContext.executePlan(WriteToFile(path, logicalPlan)).toRdd + } + + /** + * Registers this RDD as a temporary table using the given name. The lifetime of this temporary + * table is tied to the [[SQLContext]] that was used to create this SchemaRDD. + * + * @group schema + */ + def registerAsTable(tableName: String): Unit = { + sqlContext.registerRDDAsTable(baseSchemaRDD, tableName) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala new file mode 100644 index 0000000000000..7b41aa1f1bbe6 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -0,0 +1,100 @@ +/* +* 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.api.java + +import java.beans.{Introspector, PropertyDescriptor} + +import org.apache.spark.api.java.{JavaRDD, JavaSparkContext} +import org.apache.spark.sql.SQLContext +import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow, Row => ScalaRow} +import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.parquet.ParquetRelation +import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} + +/** + * The entry point for executing Spark SQL queries from a Java program. + */ +class JavaSQLContext(sparkContext: JavaSparkContext) { + + val sqlContext = new SQLContext(sparkContext.sc) + + /** + * Executes a query expressed in SQL, returning the result as a JavaSchemaRDD + */ + def sql(sqlQuery: String): JavaSchemaRDD = { + val result = new JavaSchemaRDD(sqlContext, sqlContext.parseSql(sqlQuery)) + // We force query optimization to happen right away instead of letting it happen lazily like + // when using the query DSL. This is so DDL commands behave as expected. This is only + // generates the RDD lineage for DML queries, but do not perform any execution. + result.queryExecution.toRdd + result + } + + /** + * Applies a schema to an RDD of Java Beans. + */ + def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): JavaSchemaRDD = { + // TODO: All of this could probably be moved to Catalyst as it is mostly not Spark specific. + val beanInfo = Introspector.getBeanInfo(beanClass) + + val fields = beanInfo.getPropertyDescriptors.filterNot(_.getName == "class") + val schema = fields.map { property => + val dataType = property.getPropertyType match { + 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 => 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 + } + + AttributeReference(property.getName, dataType, true)() + } + + val className = beanClass.getCanonicalName + val rowRdd = rdd.rdd.mapPartitions { iter => + // BeanInfo is not serializable so we must rediscover it remotely for each partition. + val localBeanInfo = Introspector.getBeanInfo(Class.forName(className)) + val extractors = + localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod) + + iter.map { row => + new GenericRow(extractors.map(e => e.invoke(row)).toArray[Any]): ScalaRow + } + } + new JavaSchemaRDD(sqlContext, SparkLogicalPlan(ExistingRdd(schema, rowRdd))) + } + + + /** + * Loads a parquet file, returning the result as a [[JavaSchemaRDD]]. + */ + def parquetFile(path: String): JavaSchemaRDD = + new JavaSchemaRDD(sqlContext, ParquetRelation("ParquetFile", path)) + + + /** + * Registers the given RDD as a temporary table in the catalog. Temporary tables exist only + * during the lifetime of this instance of SQLContext. + */ + def registerRDDAsTable(rdd: JavaSchemaRDD, tableName: String): Unit = { + sqlContext.registerRDDAsTable(rdd.baseSchemaRDD, tableName) + } +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala new file mode 100644 index 0000000000000..d43d672938f51 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSchemaRDD.scala @@ -0,0 +1,48 @@ +/* + * 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.api.java + +import org.apache.spark.api.java.{JavaRDDLike, JavaRDD} +import org.apache.spark.sql.{SQLContext, SchemaRDD, SchemaRDDLike} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.rdd.RDD + +/** + * An RDD of [[Row]] objects that is returned as the result of a Spark SQL query. In addition to + * standard RDD operations, a JavaSchemaRDD can also be registered as a table in the JavaSQLContext + * that was used to create. Registering a JavaSchemaRDD allows its contents to be queried in + * future SQL statement. + * + * @groupname schema SchemaRDD Functions + * @groupprio schema -1 + * @groupname Ungrouped Base RDD Functions + */ +class JavaSchemaRDD( + @transient val sqlContext: SQLContext, + @transient protected[spark] val logicalPlan: LogicalPlan) + extends JavaRDDLike[Row, JavaRDD[Row]] + with SchemaRDDLike { + + private[sql] val baseSchemaRDD = new SchemaRDD(sqlContext, logicalPlan) + + override val classTag = scala.reflect.classTag[Row] + + override def wrapRDD(rdd: RDD[Row]): JavaRDD[Row] = JavaRDD.fromRDD(rdd) + + val rdd = baseSchemaRDD.map(new Row(_)) +} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala new file mode 100644 index 0000000000000..362fe769581d7 --- /dev/null +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/Row.scala @@ -0,0 +1,93 @@ +/* + * 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.api.java + +import org.apache.spark.sql.catalyst.expressions.{Row => ScalaRow} + +/** + * A result row from a SparkSQL query. + */ +class Row(row: ScalaRow) extends Serializable { + + /** Returns the number of columns present in this Row. */ + def length: Int = row.length + + /** Returns the value of column `i`. */ + def get(i: Int): Any = + row(i) + + /** Returns true if value at column `i` is NULL. */ + def isNullAt(i: Int) = get(i) == null + + /** + * Returns the value of column `i` as an int. This function will throw an exception if the value + * is at `i` is not an integer, or if it is null. + */ + def getInt(i: Int): Int = + row.getInt(i) + + /** + * Returns the value of column `i` as a long. This function will throw an exception if the value + * is at `i` is not a long, or if it is null. + */ + def getLong(i: Int): Long = + row.getLong(i) + + /** + * Returns the value of column `i` as a double. This function will throw an exception if the + * value is at `i` is not a double, or if it is null. + */ + def getDouble(i: Int): Double = + row.getDouble(i) + + /** + * Returns the value of column `i` as a bool. This function will throw an exception if the value + * is at `i` is not a boolean, or if it is null. + */ + def getBoolean(i: Int): Boolean = + row.getBoolean(i) + + /** + * Returns the value of column `i` as a short. This function will throw an exception if the value + * is at `i` is not a short, or if it is null. + */ + def getShort(i: Int): Short = + row.getShort(i) + + /** + * Returns the value of column `i` as a byte. This function will throw an exception if the value + * is at `i` is not a byte, or if it is null. + */ + def getByte(i: Int): Byte = + row.getByte(i) + + /** + * Returns the value of column `i` as a float. This function will throw an exception if the value + * is at `i` is not a float, or if it is null. + */ + def getFloat(i: Int): Float = + row.getFloat(i) + + /** + * Returns the value of column `i` as a String. This function will throw an exception if the + * value is at `i` is not a String. + */ + def getString(i: Int): String = + row.getString(i) +} + diff --git a/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala new file mode 100644 index 0000000000000..def0e046a3831 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/api/java/JavaSQLSuite.scala @@ -0,0 +1,53 @@ +/* + * 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.api.java + +import scala.beans.BeanProperty + +import org.scalatest.FunSuite + +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.test.TestSQLContext + +// Implicits +import scala.collection.JavaConversions._ + +class PersonBean extends Serializable { + @BeanProperty + var name: String = _ + + @BeanProperty + var age: Int = _ +} + +class JavaSQLSuite extends FunSuite { + val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext) + val javaSqlCtx = new JavaSQLContext(javaCtx) + + test("schema from JavaBeans") { + val person = new PersonBean + person.setName("Michael") + person.setAge(29) + + val rdd = javaCtx.parallelize(person :: Nil) + val schemaRDD = javaSqlCtx.applySchema(rdd, classOf[PersonBean]) + + schemaRDD.registerAsTable("people") + javaSqlCtx.sql("SELECT * FROM people").collect() + } +} 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 46febbfad037d..ff8eaacded4c8 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 @@ -71,6 +71,18 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } + /** + * Executes a query expressed in HiveQL using Spark, returning the result as a SchemaRDD. + */ + def hql(hqlQuery: String): SchemaRDD = { + val result = new SchemaRDD(this, HiveQl.parseSql(hqlQuery)) + // We force query optimization to happen right away instead of letting it happen lazily like + // when using the query DSL. This is so DDL commands behave as expected. This is only + // generates the RDD lineage for DML queries, but do not perform any execution. + result.queryExecution.toRdd + result + } + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. @transient protected val outputBuffer = new java.io.OutputStream { diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala new file mode 100644 index 0000000000000..6df76fa825101 --- /dev/null +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/api/java/JavaHiveContext.scala @@ -0,0 +1,42 @@ +/* + * 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.api.java + +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.api.java.{JavaSQLContext, JavaSchemaRDD} +import org.apache.spark.sql.hive.{HiveContext, HiveQl} + +/** + * The entry point for executing Spark SQL queries from a Java program. + */ +class JavaHiveContext(sparkContext: JavaSparkContext) extends JavaSQLContext(sparkContext) { + + override val sqlContext = new HiveContext(sparkContext) + + /** + * Executes a query expressed in HiveQL, returning the result as a JavaSchemaRDD. + */ + def hql(hqlQuery: String): JavaSchemaRDD = { + val result = new JavaSchemaRDD(sqlContext, HiveQl.parseSql(hqlQuery)) + // We force query optimization to happen right away instead of letting it happen lazily like + // when using the query DSL. This is so DDL commands behave as expected. This is only + // generates the RDD lineage for DML queries, but do not perform any execution. + result.queryExecution.toRdd + result + } +} diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala new file mode 100644 index 0000000000000..8137f99b227f4 --- /dev/null +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala @@ -0,0 +1,41 @@ +/* + * 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.api.java + +import org.scalatest.FunSuite + +import org.apache.spark.api.java.JavaSparkContext +import org.apache.spark.sql.test.TestSQLContext +import org.apache.spark.sql.hive.TestHive + +// Implicits +import scala.collection.JavaConversions._ + +class JavaHiveSQLSuite extends FunSuite { + ignore("SELECT * FROM src") { + val javaCtx = new JavaSparkContext(TestSQLContext.sparkContext) + // There is a little trickery here to avoid instantiating two HiveContexts in the same JVM + val javaSqlCtx = new JavaHiveContext(javaCtx) { + override val sqlContext = TestHive + } + + assert( + javaSqlCtx.hql("SELECT * FROM src").collect().map(_.getInt(0)) === + TestHive.sql("SELECT * FROM src").collect().map(_.getInt(0)).toSeq) + } +} From a599e43d6e0950f6b6b32150ce264a8c2711470c Mon Sep 17 00:00:00 2001 From: Diana Carroll Date: Thu, 3 Apr 2014 15:48:42 -0700 Subject: [PATCH 188/397] [SPARK-1134] Fix and document passing of arguments to IPython This is based on @dianacarroll's previous pull request https://github.com/apache/spark/pull/227, and @joshrosen's comments on https://github.com/apache/spark/pull/38. Since we do want to allow passing arguments to IPython, this does the following: * It documents that IPython can't be used with standalone jobs for now. (Later versions of IPython will deal with PYTHONSTARTUP properly and enable this, see https://github.com/ipython/ipython/pull/5226, but no released version has that fix.) * If you run `pyspark` with `IPYTHON=1`, it passes your command-line arguments to it. This way you can do stuff like `IPYTHON=1 bin/pyspark notebook`. * The old `IPYTHON_OPTS` remains, but I've removed it from the documentation. This is in case people read an old tutorial that uses it. This is not a perfect solution and I'd also be okay with keeping things as they are today (ignoring `$@` for IPython and using IPYTHON_OPTS), and only doing the doc change. With this change though, when IPython fixes https://github.com/ipython/ipython/pull/5226, people will immediately be able to do `IPYTHON=1 bin/pyspark myscript.py` to run a standalone script and get all the benefits of running scripts in IPython (presumably better debugging and such). Without it, there will be no way to run scripts in IPython. @joshrosen you should probably take the final call on this. Author: Diana Carroll Closes #294 from mateiz/spark-1134 and squashes the following commits: 747bb13 [Diana Carroll] SPARK-1134 bug with ipython prevents non-interactive use with spark; only call ipython if no command line arguments were supplied --- bin/pyspark | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/bin/pyspark b/bin/pyspark index 67e1f61eeb1e5..cad982bc33477 100755 --- a/bin/pyspark +++ b/bin/pyspark @@ -55,7 +55,8 @@ if [ -n "$IPYTHON_OPTS" ]; then IPYTHON=1 fi -if [[ "$IPYTHON" = "1" ]] ; then +# Only use ipython if no command line arguments were provided [SPARK-1134] +if [[ "$IPYTHON" = "1" && $# = 0 ]] ; then exec ipython $IPYTHON_OPTS else exec "$PYSPARK_PYTHON" "$@" From d94826be6d46edf3bc6377d33787df23a6030a6c Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 3 Apr 2014 16:12:08 -0700 Subject: [PATCH 189/397] [BUILD FIX] Fix compilation of Spark SQL Java API. The JavaAPI and the Parquet improvements PRs didn't conflict, but broke the build. Author: Michael Armbrust Closes #316 from marmbrus/hotFixJavaApi and squashes the following commits: 0b84c2d [Michael Armbrust] Fix compilation of Spark SQL Java API. --- .../scala/org/apache/spark/sql/api/java/JavaSQLContext.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 7b41aa1f1bbe6..573345e42c43c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -87,7 +87,7 @@ class JavaSQLContext(sparkContext: JavaSparkContext) { * Loads a parquet file, returning the result as a [[JavaSchemaRDD]]. */ def parquetFile(path: String): JavaSchemaRDD = - new JavaSchemaRDD(sqlContext, ParquetRelation("ParquetFile", path)) + new JavaSchemaRDD(sqlContext, ParquetRelation(path)) /** From 9231b011a9ba5a2b25bd3d1a68be7d1a7cb735da Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 3 Apr 2014 16:53:35 -0700 Subject: [PATCH 190/397] Fix jenkins from giving the green light to builds that don't compile. Adding `| grep` swallows the non-zero return code from sbt failures. See [here](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/13735/consoleFull) for a Jenkins run that fails to compile, but still gets a green light. Note the [BUILD FIX] commit isn't actually part of this PR, but github is out of date. Author: Michael Armbrust Closes #317 from marmbrus/fixJenkins and squashes the following commits: 7c77ff9 [Michael Armbrust] Remove output filter that was swallowing non-zero exit codes for test failures. --- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/run-tests b/dev/run-tests index a6fcc40a5ba6e..fff949e04fcd7 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -49,7 +49,7 @@ dev/scalastyle echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -sbt/sbt assembly test | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" +sbt/sbt assembly test echo "=========================================================================" echo "Running PySpark tests" From 33e63618d061eeaae257a7350ea3287a702fc123 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 3 Apr 2014 17:00:06 -0700 Subject: [PATCH 191/397] Revert "[SPARK-1398] Removed findbugs jsr305 dependency" This reverts commit 92a86b285f8a4af1bdf577dd4c4ea0fd5ca8d682. --- core/pom.xml | 4 ++++ pom.xml | 5 +++++ project/SparkBuild.scala | 1 + 3 files changed, 10 insertions(+) diff --git a/core/pom.xml b/core/pom.xml index 273aa69659336..e4c32eff0cd77 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -82,6 +82,10 @@ com.google.guava guava
    + + com.google.code.findbugs + jsr305 + org.slf4j slf4j-api diff --git a/pom.xml b/pom.xml index b91b14d2f84d0..7d58060cba606 100644 --- a/pom.xml +++ b/pom.xml @@ -214,6 +214,11 @@ guava 14.0.1 + + com.google.code.findbugs + jsr305 + 1.3.9 + org.slf4j slf4j-api diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a2a21d9763548..c5c697e8e2427 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -296,6 +296,7 @@ object SparkBuild extends Build { name := "spark-core", libraryDependencies ++= Seq( "com.google.guava" % "guava" % "14.0.1", + "com.google.code.findbugs" % "jsr305" % "1.3.9", "log4j" % "log4j" % "1.2.17", "org.slf4j" % "slf4j-api" % slf4jVersion, "org.slf4j" % "slf4j-log4j12" % slf4jVersion, From ee6e9e7d863022304ac9ced405b353b63accb6ab Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 3 Apr 2014 22:13:56 -0700 Subject: [PATCH 192/397] SPARK-1337: Application web UI garbage collects newest stages Simple fix... Author: Patrick Wendell Closes #320 from pwendell/stage-clean-up and squashes the following commits: 29be62e [Patrick Wendell] SPARK-1337: Application web UI garbage collects newest stages instead old ones --- .../spark/ui/jobs/JobProgressListener.scala | 8 ++--- .../ui/jobs/JobProgressListenerSuite.scala | 33 +++++++++++++++++-- 2 files changed, 35 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index d10aa12b9ebca..cd4be57227a16 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -81,8 +81,8 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { /** If stages is too large, remove and garbage collect old stages */ private def trimIfNecessary(stages: ListBuffer[StageInfo]) = synchronized { if (stages.size > retainedStages) { - val toRemove = retainedStages / 10 - stages.takeRight(toRemove).foreach( s => { + val toRemove = math.max(retainedStages / 10, 1) + stages.take(toRemove).foreach { s => stageIdToTaskData.remove(s.stageId) stageIdToTime.remove(s.stageId) stageIdToShuffleRead.remove(s.stageId) @@ -94,8 +94,8 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { stageIdToTasksFailed.remove(s.stageId) stageIdToPool.remove(s.stageId) if (stageIdToDescription.contains(s.stageId)) {stageIdToDescription.remove(s.stageId)} - }) - stages.trimEnd(toRemove) + } + stages.trimStart(toRemove) } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index d8a3e859f85cd..67ceee505db3c 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -18,13 +18,42 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite +import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.{LocalSparkContext, SparkContext, Success} +import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils -class JobProgressListenerSuite extends FunSuite with LocalSparkContext { +class JobProgressListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers { + test("test LRU eviction of stages") { + val conf = new SparkConf() + conf.set("spark.ui.retainedStages", 5.toString) + val listener = new JobProgressListener(conf) + + def createStageStartEvent(stageId: Int) = { + val stageInfo = new StageInfo(stageId, stageId.toString, 0, null) + SparkListenerStageSubmitted(stageInfo) + } + + def createStageEndEvent(stageId: Int) = { + val stageInfo = new StageInfo(stageId, stageId.toString, 0, null) + SparkListenerStageCompleted(stageInfo) + } + + for (i <- 1 to 50) { + listener.onStageSubmitted(createStageStartEvent(i)) + listener.onStageCompleted(createStageEndEvent(i)) + } + + listener.completedStages.size should be (5) + listener.completedStages.filter(_.stageId == 50).size should be (1) + listener.completedStages.filter(_.stageId == 49).size should be (1) + listener.completedStages.filter(_.stageId == 48).size should be (1) + listener.completedStages.filter(_.stageId == 47).size should be (1) + listener.completedStages.filter(_.stageId == 46).size should be (1) + } + test("test executor id to summary") { val sc = new SparkContext("local", "test") val listener = new JobProgressListener(sc.conf) From 7f32fd42aaadcf6626b4d8bcf6295203b43b2037 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 4 Apr 2014 08:54:04 -0500 Subject: [PATCH 193/397] SPARK-1350. Always use JAVA_HOME to run executor container JVMs. Author: Sandy Ryza Closes #313 from sryza/sandy-spark-1350 and squashes the following commits: bb6d187 [Sandy Ryza] SPARK-1350. Always use JAVA_HOME to run executor container JVMs. --- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 8 +------- .../apache/spark/deploy/yarn/ExecutorRunnableUtil.scala | 9 ++------- 2 files changed, 3 insertions(+), 14 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 6568003bf1008..bc267900fcf1d 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -345,14 +345,8 @@ trait ClientBase extends Logging { } // Command for the ApplicationMaster - var javaCommand = "java" - val javaHome = System.getenv("JAVA_HOME") - if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { - javaCommand = Environment.JAVA_HOME.$() + "/bin/java" - } - val commands = List[String]( - javaCommand + + Environment.JAVA_HOME.$() + "/bin/java" + " -server " + JAVA_OPTS + " " + args.amClass + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index da0a6f74efcd5..2079697d8160e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -88,13 +88,8 @@ trait ExecutorRunnableUtil extends Logging { } */ - var javaCommand = "java" - val javaHome = System.getenv("JAVA_HOME") - if ((javaHome != null && !javaHome.isEmpty()) || env.isDefinedAt("JAVA_HOME")) { - javaCommand = Environment.JAVA_HOME.$() + "/bin/java" - } - - val commands = List[String](javaCommand + + val commands = List[String]( + Environment.JAVA_HOME.$() + "/bin/java" + " -server " + // Kill if OOM is raised - leverage yarn's failure handling to cause rescheduling. // Not killing the task leaves various aspects of the executor and (to some extent) the jvm in From 01cf4c402b9fda59680e56112bfaa2b748416d0e Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Fri, 4 Apr 2014 09:50:24 -0700 Subject: [PATCH 194/397] SPARK-1404: Always upgrade spark-env.sh vars to environment vars This was broken when spark-env.sh was made idempotent, as the idempotence check is an environment variable, but the spark-env.sh variables may not have been. Tested in zsh, bash, and sh. Author: Aaron Davidson Closes #310 from aarondav/SPARK-1404 and squashes the following commits: c3406a5 [Aaron Davidson] Add extra export in spark-shell 6a0e340 [Aaron Davidson] SPARK-1404: Always upgrade spark-env.sh vars to environment vars --- bin/load-spark-env.sh | 3 +++ bin/spark-shell | 4 ++-- 2 files changed, 5 insertions(+), 2 deletions(-) diff --git a/bin/load-spark-env.sh b/bin/load-spark-env.sh index 476dd826551fd..d425f9feaac54 100644 --- a/bin/load-spark-env.sh +++ b/bin/load-spark-env.sh @@ -30,6 +30,9 @@ if [ -z "$SPARK_ENV_LOADED" ]; then use_conf_dir=${SPARK_CONF_DIR:-"$parent_dir/conf"} if [ -f "${use_conf_dir}/spark-env.sh" ]; then + # Promote all variable declarations to environment (exported) variables + set -a . "${use_conf_dir}/spark-env.sh" + set +a fi fi diff --git a/bin/spark-shell b/bin/spark-shell index fac006cf492ed..535ee3ccd8269 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -127,7 +127,7 @@ function set_spark_log_conf(){ function set_spark_master(){ if ! [[ "$1" =~ $ARG_FLAG_PATTERN ]]; then - MASTER="$1" + export MASTER="$1" else out_error "wrong format for $2" fi @@ -145,7 +145,7 @@ function resolve_spark_master(){ fi if [ -z "$MASTER" ]; then - MASTER="$DEFAULT_MASTER" + export MASTER="$DEFAULT_MASTER" fi } From f1fa617023d30d8cdc5acef0274bad8cc3e89cea Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Fri, 4 Apr 2014 11:12:47 -0700 Subject: [PATCH 195/397] [SPARK-1133] Add whole text files reader in MLlib Here is a pointer to the former [PR164](https://github.com/apache/spark/pull/164). I add the pull request for the JIRA issue [SPARK-1133](https://spark-project.atlassian.net/browse/SPARK-1133), which brings a new files reader API in MLlib. Author: Xusen Yin Closes #252 from yinxusen/whole-files-input and squashes the following commits: 7191be6 [Xusen Yin] refine comments 0af3faf [Xusen Yin] add JavaAPI test 01745ee [Xusen Yin] fix deletion error cc97dca [Xusen Yin] move whole text file API to Spark core d792cee [Xusen Yin] remove the typo character "+" 6bdf2c2 [Xusen Yin] test for small local file system block size a1f1e7e [Xusen Yin] add two extra spaces 28cb0fe [Xusen Yin] add whole text files reader --- .../scala/org/apache/spark/SparkContext.scala | 34 ++++++ .../spark/api/java/JavaSparkContext.scala | 28 +++++ .../input/WholeTextFileInputFormat.scala | 47 ++++++++ .../input/WholeTextFileRecordReader.scala | 72 ++++++++++++ .../java/org/apache/spark/JavaAPISuite.java | 30 ++++- .../WholeTextFileRecordReaderSuite.scala | 105 ++++++++++++++++++ 6 files changed, 313 insertions(+), 3 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala create mode 100644 core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala create mode 100644 core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index b23accbbb9410..28a865c0ad3b5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -37,6 +37,7 @@ import org.apache.mesos.MesosNativeLibrary import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} +import org.apache.spark.input.WholeTextFileInputFormat import org.apache.spark.partial.{ApproximateEvaluator, PartialResult} import org.apache.spark.rdd._ import org.apache.spark.scheduler._ @@ -371,6 +372,39 @@ class SparkContext( minSplits).map(pair => pair._2.toString) } + /** + * Read a directory of text files from HDFS, a local file system (available on all nodes), or any + * Hadoop-supported file system URI. Each file is read as a single record and returned in a + * key-value pair, where the key is the path of each file, the value is the content of each file. + * + *

    For example, if you have the following files: + * {{{ + * hdfs://a-hdfs-path/part-00000 + * hdfs://a-hdfs-path/part-00001 + * ... + * hdfs://a-hdfs-path/part-nnnnn + * }}} + * + * Do `val rdd = sparkContext.wholeTextFile("hdfs://a-hdfs-path")`, + * + *

    then `rdd` contains + * {{{ + * (a-hdfs-path/part-00000, its content) + * (a-hdfs-path/part-00001, its content) + * ... + * (a-hdfs-path/part-nnnnn, its content) + * }}} + * + * @note Small files are perferred, large file is also allowable, but may cause bad performance. + */ + def wholeTextFiles(path: String): RDD[(String, String)] = { + newAPIHadoopFile( + path, + classOf[WholeTextFileInputFormat], + classOf[String], + classOf[String]) + } + /** * Get an RDD for a Hadoop-readable dataset from a Hadoop JobConf given its InputFormat and other * necessary info (e.g. file name for a filesystem-based dataset, table name for HyperTable), diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index e531a57aced31..6cbdeac58d5e2 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -154,6 +154,34 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def textFile(path: String, minSplits: Int): JavaRDD[String] = sc.textFile(path, minSplits) + /** + * Read a directory of text files from HDFS, a local file system (available on all nodes), or any + * Hadoop-supported file system URI. Each file is read as a single record and returned in a + * key-value pair, where the key is the path of each file, the value is the content of each file. + * + *

    For example, if you have the following files: + * {{{ + * hdfs://a-hdfs-path/part-00000 + * hdfs://a-hdfs-path/part-00001 + * ... + * hdfs://a-hdfs-path/part-nnnnn + * }}} + * + * Do `JavaPairRDD rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path")`, + * + *

    then `rdd` contains + * {{{ + * (a-hdfs-path/part-00000, its content) + * (a-hdfs-path/part-00001, its content) + * ... + * (a-hdfs-path/part-nnnnn, its content) + * }}} + * + * @note Small files are perferred, large file is also allowable, but may cause bad performance. + */ + def wholeTextFiles(path: String): JavaPairRDD[String, String] = + new JavaPairRDD(sc.wholeTextFiles(path)) + /** Get an RDD for a Hadoop SequenceFile with given key and value types. * * '''Note:''' Because Hadoop's RecordReader class re-uses the same Writable object for each diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala new file mode 100644 index 0000000000000..4887fb6b84eb2 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -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. + */ + +package org.apache.spark.input + +import org.apache.hadoop.fs.Path +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.JobContext +import org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext +import org.apache.hadoop.mapreduce.lib.input.CombineFileRecordReader +import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit + +/** + * A [[org.apache.hadoop.mapreduce.lib.input.CombineFileInputFormat CombineFileInputFormat]] for + * reading whole text files. Each file is read as key-value pair, where the key is the file path and + * the value is the entire content of file. + */ + +private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[String, String] { + override protected def isSplitable(context: JobContext, file: Path): Boolean = false + + override def createRecordReader( + split: InputSplit, + context: TaskAttemptContext): RecordReader[String, String] = { + + new CombineFileRecordReader[String, String]( + split.asInstanceOf[CombineFileSplit], + context, + classOf[WholeTextFileRecordReader]) + } +} diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala new file mode 100644 index 0000000000000..c3dabd2e79995 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileRecordReader.scala @@ -0,0 +1,72 @@ +/* + * 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.input + +import com.google.common.io.{ByteStreams, Closeables} + +import org.apache.hadoop.io.Text +import org.apache.hadoop.mapreduce.InputSplit +import org.apache.hadoop.mapreduce.lib.input.CombineFileSplit +import org.apache.hadoop.mapreduce.RecordReader +import org.apache.hadoop.mapreduce.TaskAttemptContext + +/** + * A [[org.apache.hadoop.mapreduce.RecordReader RecordReader]] for reading a single whole text file + * out in a key-value pair, where the key is the file path and the value is the entire content of + * the file. + */ +private[spark] class WholeTextFileRecordReader( + split: CombineFileSplit, + context: TaskAttemptContext, + index: Integer) + extends RecordReader[String, String] { + + private val path = split.getPath(index) + private val fs = path.getFileSystem(context.getConfiguration) + + // True means the current file has been processed, then skip it. + private var processed = false + + private val key = path.toString + private var value: String = null + + override def initialize(split: InputSplit, context: TaskAttemptContext) = {} + + override def close() = {} + + override def getProgress = if (processed) 1.0f else 0.0f + + override def getCurrentKey = key + + override def getCurrentValue = value + + override def nextKeyValue = { + if (!processed) { + val fileIn = fs.open(path) + val innerBuffer = ByteStreams.toByteArray(fileIn) + + value = new Text(innerBuffer).toString + Closeables.close(fileIn, false) + + processed = true + true + } else { + false + } + } +} diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index c6b65c7348ae0..2372f2d9924a1 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -17,9 +17,7 @@ package org.apache.spark; -import java.io.File; -import java.io.IOException; -import java.io.Serializable; +import java.io.*; import java.util.*; import scala.Tuple2; @@ -599,6 +597,32 @@ public void textFiles() throws IOException { Assert.assertEquals(expected, readRDD.collect()); } + @Test + public void wholeTextFiles() throws IOException { + byte[] content1 = "spark is easy to use.\n".getBytes(); + byte[] content2 = "spark is also easy to use.\n".getBytes(); + + File tempDir = Files.createTempDir(); + String tempDirName = tempDir.getAbsolutePath(); + DataOutputStream ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00000")); + ds.write(content1); + ds.close(); + ds = new DataOutputStream(new FileOutputStream(tempDirName + "/part-00001")); + ds.write(content2); + ds.close(); + + HashMap container = new HashMap(); + container.put(tempDirName+"/part-00000", new Text(content1).toString()); + container.put(tempDirName+"/part-00001", new Text(content2).toString()); + + JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName); + List> result = readRDD.collect(); + + for (Tuple2 res : result) { + Assert.assertEquals(res._2(), container.get(res._1())); + } + } + @Test public void textFilesCompressed() throws IOException { File tempDir = Files.createTempDir(); diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala new file mode 100644 index 0000000000000..09e35bfc8f85f --- /dev/null +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -0,0 +1,105 @@ +/* + * 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.input + +import java.io.DataOutputStream +import java.io.File +import java.io.FileOutputStream + +import scala.collection.immutable.IndexedSeq + +import com.google.common.io.Files + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import org.apache.hadoop.io.Text + +import org.apache.spark.SparkContext + +/** + * Tests the correctness of + * [[org.apache.spark.input.WholeTextFileRecordReader WholeTextFileRecordReader]]. A temporary + * directory is created as fake input. Temporal storage would be deleted in the end. + */ +class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { + private var sc: SparkContext = _ + + override def beforeAll() { + sc = new SparkContext("local", "test") + + // Set the block size of local file system to test whether files are split right or not. + sc.hadoopConfiguration.setLong("fs.local.block.size", 32) + } + + override def afterAll() { + sc.stop() + } + + private def createNativeFile(inputDir: File, fileName: String, contents: Array[Byte]) = { + val out = new DataOutputStream(new FileOutputStream(s"${inputDir.toString}/$fileName")) + out.write(contents, 0, contents.length) + out.close() + } + + /** + * This code will test the behaviors of WholeTextFileRecordReader based on local disk. There are + * three aspects to check: + * 1) Whether all files are read; + * 2) Whether paths are read correctly; + * 3) Does the contents be the same. + */ + test("Correctness of WholeTextFileRecordReader.") { + + val dir = Files.createTempDir() + println(s"Local disk address is ${dir.toString}.") + + WholeTextFileRecordReaderSuite.files.foreach { case (filename, contents) => + createNativeFile(dir, filename, contents) + } + + val res = sc.wholeTextFiles(dir.toString).collect() + + assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, + "Number of files read out does not fit with the actual value.") + + for ((filename, contents) <- res) { + val shortName = filename.split('/').last + assert(WholeTextFileRecordReaderSuite.fileNames.contains(shortName), + s"Missing file name $filename.") + assert(contents === new Text(WholeTextFileRecordReaderSuite.files(shortName)).toString, + s"file $filename contents can not match.") + } + + dir.delete() + } +} + +/** + * Files to be tested are defined here. + */ +object WholeTextFileRecordReaderSuite { + private val testWords: IndexedSeq[Byte] = "Spark is easy to use.\n".map(_.toByte) + + private val fileNames = Array("part-00000", "part-00001", "part-00002") + private val fileLengths = Array(10, 100, 1000) + + private val files = fileLengths.zip(fileNames).map { case (upperBound, filename) => + filename -> Stream.continually(testWords.toList.toStream).flatten.take(upperBound).toArray + }.toMap +} From 16b830888734de260f460506b766edab79d30ecd Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 4 Apr 2014 13:28:42 -0700 Subject: [PATCH 196/397] SPARK-1375. Additional spark-submit cleanup Author: Sandy Ryza Closes #278 from sryza/sandy-spark-1375 and squashes the following commits: 5fbf1e9 [Sandy Ryza] SPARK-1375. Additional spark-submit cleanup --- .../scala/org/apache/spark/deploy/SparkSubmit.scala | 13 ++++++++----- .../apache/spark/deploy/SparkSubmitArguments.scala | 2 +- docs/cluster-overview.md | 2 +- 3 files changed, 10 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 1fa799190409f..e05fbfe321495 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -79,20 +79,23 @@ object SparkSubmit { printErrorAndExit("master must start with yarn, mesos, spark, or local") } - // Because "yarn-standalone" and "yarn-client" encapsulate both the master + // Because "yarn-cluster" and "yarn-client" encapsulate both the master // and deploy mode, we have some logic to infer the master and deploy mode // from each other if only one is specified, or exit early if they are at odds. - if (appArgs.deployMode == null && appArgs.master == "yarn-standalone") { + if (appArgs.deployMode == null && + (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { appArgs.deployMode = "cluster" } if (appArgs.deployMode == "cluster" && appArgs.master == "yarn-client") { printErrorAndExit("Deploy mode \"cluster\" and master \"yarn-client\" are not compatible") } - if (appArgs.deployMode == "client" && appArgs.master == "yarn-standalone") { - printErrorAndExit("Deploy mode \"client\" and master \"yarn-standalone\" are not compatible") + if (appArgs.deployMode == "client" && + (appArgs.master == "yarn-standalone" || appArgs.master == "yarn-cluster")) { + printErrorAndExit("Deploy mode \"client\" and master \"" + appArgs.master + + "\" are not compatible") } if (appArgs.deployMode == "cluster" && appArgs.master.startsWith("yarn")) { - appArgs.master = "yarn-standalone" + appArgs.master = "yarn-cluster" } if (appArgs.deployMode != "cluster" && appArgs.master.startsWith("yarn")) { appArgs.master = "yarn-client" diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 9c8f54ea6f77a..834b3df2f164b 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -171,7 +171,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { outStream.println("Unknown/unsupported param " + unknownParam) } outStream.println( - """Usage: spark-submit [options] + """Usage: spark-submit [options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index b69e3416fb322..7f75ea44e4cea 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -56,7 +56,7 @@ The recommended way to launch a compiled Spark application is through the spark- bin directory), which takes care of setting up the classpath with Spark and its dependencies, as well as provides a layer over the different cluster managers and deploy modes that Spark supports. It's usage is - spark-submit `` `` + spark-submit `` `` Where options are any of: From a02b535d5e18e987a4b9c4c352838d294f9e853b Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 4 Apr 2014 14:46:32 -0700 Subject: [PATCH 197/397] Don't create SparkContext in JobProgressListenerSuite. This reduces the time of the test from 11 seconds to 20 milliseconds. Author: Patrick Wendell Closes #324 from pwendell/job-test and squashes the following commits: 868d9eb [Patrick Wendell] Don't create SparkContext in JobProgressListenerSuite. --- .../org/apache/spark/ui/jobs/JobProgressListenerSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index 67ceee505db3c..beac656f573b4 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -55,8 +55,8 @@ class JobProgressListenerSuite extends FunSuite with LocalSparkContext with Shou } test("test executor id to summary") { - val sc = new SparkContext("local", "test") - val listener = new JobProgressListener(sc.conf) + val conf = new SparkConf() + val listener = new JobProgressListener(conf) val taskMetrics = new TaskMetrics() val shuffleReadMetrics = new ShuffleReadMetrics() From 198892fe8d39a2fad585fa2a7579d8b478456c33 Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 4 Apr 2014 17:16:31 -0700 Subject: [PATCH 198/397] [SPARK-1198] Allow pipes tasks to run in different sub-directories This works as is on Linux/Mac/etc but doesn't cover working on Windows. In here I use ln -sf for symlinks. Putting this up for comments on that. Do we want to create perhaps some classes for doing shell commands - Linux vs Windows. Is there some other way we want to do this? I assume we are still supporting jdk1.6? Also should I update the Java API for pipes to allow this parameter? Author: Thomas Graves Closes #128 from tgravescs/SPARK1198 and squashes the following commits: abc1289 [Thomas Graves] remove extra tag in pom file ba23fc0 [Thomas Graves] Add support for symlink on windows, remove commons-io usage da4b221 [Thomas Graves] Merge branch 'master' of https://github.com/tgravescs/spark into SPARK1198 61be271 [Thomas Graves] Fix file name filter 6b783bd [Thomas Graves] style fixes 1ab49ca [Thomas Graves] Add support for running pipe tasks is separate directories --- .../scala/org/apache/spark/rdd/PipedRDD.scala | 64 ++++++++++++++++++- .../main/scala/org/apache/spark/rdd/RDD.scala | 7 +- .../scala/org/apache/spark/util/Utils.scala | 45 ++++++++++++- .../org/apache/spark/PipedRDDSuite.scala | 28 +++++++- 4 files changed, 137 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 4250a9d02f764..41ae0fec823e7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -17,6 +17,9 @@ package org.apache.spark.rdd +import java.io.File +import java.io.FilenameFilter +import java.io.IOException import java.io.PrintWriter import java.util.StringTokenizer @@ -27,6 +30,7 @@ import scala.io.Source import scala.reflect.ClassTag import org.apache.spark.{Partition, SparkEnv, TaskContext} +import org.apache.spark.util.Utils /** @@ -38,7 +42,8 @@ class PipedRDD[T: ClassTag]( command: Seq[String], envVars: Map[String, String], printPipeContext: (String => Unit) => Unit, - printRDDElement: (T, String => Unit) => Unit) + printRDDElement: (T, String => Unit) => Unit, + separateWorkingDir: Boolean) extends RDD[String](prev) { // Similar to Runtime.exec(), if we are given a single string, split it into words @@ -48,12 +53,24 @@ class PipedRDD[T: ClassTag]( command: String, envVars: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, - printRDDElement: (T, String => Unit) => Unit = null) = - this(prev, PipedRDD.tokenize(command), envVars, printPipeContext, printRDDElement) + printRDDElement: (T, String => Unit) => Unit = null, + separateWorkingDir: Boolean = false) = + this(prev, PipedRDD.tokenize(command), envVars, printPipeContext, printRDDElement, + separateWorkingDir) override def getPartitions: Array[Partition] = firstParent[T].partitions + /** + * A FilenameFilter that accepts anything that isn't equal to the name passed in. + * @param name of file or directory to leave out + */ + class NotEqualsFileNameFilter(filterName: String) extends FilenameFilter { + def accept(dir: File, name: String): Boolean = { + !name.equals(filterName) + } + } + override def compute(split: Partition, context: TaskContext): Iterator[String] = { val pb = new ProcessBuilder(command) // Add the environmental variables to the process. @@ -67,6 +84,38 @@ class PipedRDD[T: ClassTag]( currentEnvVars.putAll(hadoopSplit.getPipeEnvVars()) } + // When spark.worker.separated.working.directory option is turned on, each + // task will be run in separate directory. This should be resolve file + // access conflict issue + val taskDirectory = "./tasks/" + java.util.UUID.randomUUID.toString + var workInTaskDirectory = false + logDebug("taskDirectory = " + taskDirectory) + if (separateWorkingDir == true) { + val currentDir = new File(".") + logDebug("currentDir = " + currentDir.getAbsolutePath()) + val taskDirFile = new File(taskDirectory) + taskDirFile.mkdirs() + + try { + val tasksDirFilter = new NotEqualsFileNameFilter("tasks") + + // Need to add symlinks to jars, files, and directories. On Yarn we could have + // directories and other files not known to the SparkContext that were added via the + // Hadoop distributed cache. We also don't want to symlink to the /tasks directories we + // are creating here. + for (file <- currentDir.list(tasksDirFilter)) { + val fileWithDir = new File(currentDir, file) + Utils.symlink(new File(fileWithDir.getAbsolutePath()), + new File(taskDirectory + "/" + fileWithDir.getName())) + } + pb.directory(taskDirFile) + workInTaskDirectory = true + } catch { + case e: Exception => logError("Unable to setup task working directory: " + e.getMessage + + " (" + taskDirectory + ")") + } + } + val proc = pb.start() val env = SparkEnv.get @@ -112,6 +161,15 @@ class PipedRDD[T: ClassTag]( if (exitStatus != 0) { throw new Exception("Subprocess exited with status " + exitStatus) } + + // cleanup task working directory if used + if (workInTaskDirectory == true) { + scala.util.control.Exception.ignoring(classOf[IOException]) { + Utils.deleteRecursively(new File(taskDirectory)) + } + logDebug("Removed task working directory " + taskDirectory) + } + false } } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index ce2b8ac27206b..08c42c5ee87b6 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -481,16 +481,19 @@ abstract class RDD[T: ClassTag]( * instead of constructing a huge String to concat all the elements: * def printRDDElement(record:(String, Seq[String]), f:String=>Unit) = * for (e <- record._2){f(e)} + * @param separateWorkingDir Use separate working directories for each task. * @return the result RDD */ def pipe( command: Seq[String], env: Map[String, String] = Map(), printPipeContext: (String => Unit) => Unit = null, - printRDDElement: (T, String => Unit) => Unit = null): RDD[String] = { + printRDDElement: (T, String => Unit) => Unit = null, + separateWorkingDir: Boolean = false): RDD[String] = { new PipedRDD(this, command, env, if (printPipeContext ne null) sc.clean(printPipeContext) else null, - if (printRDDElement ne null) sc.clean(printRDDElement) else null) + if (printRDDElement ne null) sc.clean(printRDDElement) else null, + separateWorkingDir) } /** diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 62ee704d580c2..737b765e2aed6 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,6 +26,7 @@ import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.SortedSet import scala.io.Source import scala.reflect.ClassTag @@ -43,6 +44,8 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, */ private[spark] object Utils extends Logging { + val osName = System.getProperty("os.name") + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -521,9 +524,10 @@ private[spark] object Utils extends Logging { /** * Delete a file or directory and its contents recursively. + * Don't follow directories if they are symlinks. */ def deleteRecursively(file: File) { - if (file.isDirectory) { + if ((file.isDirectory) && !isSymlink(file)) { for (child <- listFilesSafely(file)) { deleteRecursively(child) } @@ -536,6 +540,25 @@ private[spark] object Utils extends Logging { } } + /** + * Check to see if file is a symbolic link. + */ + def isSymlink(file: File): Boolean = { + if (file == null) throw new NullPointerException("File must not be null") + if (osName.startsWith("Windows")) return false + val fileInCanonicalDir = if (file.getParent() == null) { + file + } else { + new File(file.getParentFile().getCanonicalFile(), file.getName()) + } + + if (fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile())) { + return false; + } else { + return true; + } + } + /** * Convert a Java memory parameter passed to -Xmx (such as 300m or 1g) to a number of megabytes. */ @@ -898,6 +921,26 @@ private[spark] object Utils extends Logging { count } + /** + * Creates a symlink. Note jdk1.7 has Files.createSymbolicLink but not used here + * for jdk1.6 support. Supports windows by doing copy, everything else uses "ln -sf". + * @param src absolute path to the source + * @param dst relative path for the destination + */ + def symlink(src: File, dst: File) { + if (!src.isAbsolute()) { + throw new IOException("Source must be absolute") + } + if (dst.isAbsolute()) { + throw new IOException("Destination must be relative") + } + val linkCmd = if (osName.startsWith("Windows")) "copy" else "ln -sf" + import scala.sys.process._ + (linkCmd + " " + src.getAbsolutePath() + " " + dst.getPath()) lines_! ProcessLogger(line => + (logInfo(line))) + } + + /** Return the class name of the given object, removing all dollar signs */ def getFormattedClassName(obj: AnyRef) = { obj.getClass.getSimpleName.replace("$", "") diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 6e7fd55fa4bb1..627e9b5cd9060 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -17,8 +17,11 @@ package org.apache.spark -import org.scalatest.FunSuite +import java.io.File + +import com.google.common.io.Files +import org.scalatest.FunSuite import org.apache.spark.rdd.{HadoopRDD, PipedRDD, HadoopPartition} import org.apache.hadoop.mapred.{JobConf, TextInputFormat, FileSplit} @@ -126,6 +129,29 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { } } + test("basic pipe with separate working directory") { + if (testCommandAvailable("cat")) { + val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) + val piped = nums.pipe(Seq("cat"), separateWorkingDir = true) + val c = piped.collect() + assert(c.size === 4) + assert(c(0) === "1") + assert(c(1) === "2") + assert(c(2) === "3") + assert(c(3) === "4") + val pipedPwd = nums.pipe(Seq("pwd"), separateWorkingDir = true) + val collectPwd = pipedPwd.collect() + assert(collectPwd(0).contains("tasks/")) + val pipedLs = nums.pipe(Seq("ls"), separateWorkingDir = true).collect() + // make sure symlinks were created + assert(pipedLs.length > 0) + // clean up top level tasks directory + new File("tasks").delete() + } else { + assert(true) + } + } + test("test pipe exports map_input_file") { testExportInputFile("map_input_file") } From d956cc251676d67d87bd6dbfa82be864933d8136 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 4 Apr 2014 17:23:17 -0700 Subject: [PATCH 199/397] [SQL] Minor fixes. Author: Michael Armbrust Closes #315 from marmbrus/minorFixes and squashes the following commits: b23a15d [Michael Armbrust] fix scaladoc 11062ac [Michael Armbrust] Fix registering "SELECT *" queries as tables and caching them. As some tests for this and self-joins. 3997dc9 [Michael Armbrust] Move Row extractor to catalyst. 208bf5e [Michael Armbrust] More idiomatic naming of DSL functions. * subquery => as * for join condition => on, i.e., `r.join(s, condition = 'a == 'b)` =>`r.join(s, on = 'a == 'b)` 87211ce [Michael Armbrust] Correctly handle self joins of in-memory cached tables. 69e195e [Michael Armbrust] Change != to !== in the DSL since != will always translate to != on Any. 01f2dd5 [Michael Armbrust] Correctly assign aliases to tables in SqlParser. --- .../apache/spark/sql/catalyst/SqlParser.scala | 2 +- .../apache/spark/sql/catalyst/dsl/package.scala | 2 +- .../spark/sql/catalyst/expressions/Row.scala | 15 +++++++++++++++ .../catalyst/plans/logical/basicOperators.scala | 1 + .../scala/org/apache/spark/sql/package.scala | 15 +-------------- .../scala/org/apache/spark/sql/SchemaRDD.scala | 16 ++++++++-------- .../apache/spark/sql/execution/SparkPlan.scala | 3 +++ .../org/apache/spark/sql/CachedTableSuite.scala | 13 +++++++++++++ .../org/apache/spark/sql/DslQuerySuite.scala | 16 ++++++++-------- .../spark/sql/parquet/ParquetQuerySuite.scala | 4 ++-- 10 files changed, 53 insertions(+), 34 deletions(-) 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 index 4ea80fee23e1e..5b6aea81cb7d1 100644 --- 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 @@ -219,7 +219,7 @@ class SqlParser extends StandardTokenParsers { protected lazy val relationFactor: Parser[LogicalPlan] = ident ~ (opt(AS) ~> opt(ident)) ^^ { - case ident ~ alias => UnresolvedRelation(alias, ident) + case tableName ~ alias => UnresolvedRelation(None, tableName, alias) } | "(" ~> query ~ ")" ~ opt(AS) ~ ident ^^ { case s ~ _ ~ _ ~ a => Subquery(a, s) } 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 2c4bf1715b646..2d62e4cbbce01 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 @@ -70,7 +70,7 @@ package object dsl { def > (other: Expression) = GreaterThan(expr, other) def >= (other: Expression) = GreaterThanOrEqual(expr, other) def === (other: Expression) = Equals(expr, other) - def != (other: Expression) = Not(Equals(expr, other)) + def !== (other: Expression) = Not(Equals(expr, other)) def like(other: Expression) = Like(expr, other) def rlike(other: Expression) = RLike(expr, other) 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 6f939e6c41f6b..9f4d84466e6ac 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 @@ -19,6 +19,21 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types.NativeType +object Row { + /** + * This method can be used to extract fields from a [[Row]] object in a pattern 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) +} + /** * 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. 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 b39c2b32cc42c..cfc0b0c3a8d98 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 @@ -162,6 +162,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { a.nullable)( a.exprId, a.qualifiers) + case other => other } def references = Set.empty 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 9ec31689b5098..4589129cd1c90 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 @@ -32,18 +32,5 @@ package object sql { type Row = catalyst.expressions.Row - object Row { - /** - * This method can be used to extract fields from a [[Row]] object in a pattern 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) - } + val Row = catalyst.expressions.Row } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index a62cb8aa1321f..fc95781448569 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -148,17 +148,17 @@ class SchemaRDD( * * @param otherPlan the [[SchemaRDD]] that should be joined with this one. * @param joinType One of `Inner`, `LeftOuter`, `RightOuter`, or `FullOuter`. Defaults to `Inner.` - * @param condition An optional condition for the join operation. This is equivilent to the `ON` - * clause in standard SQL. In the case of `Inner` joins, specifying a - * `condition` is equivilent to adding `where` clauses after the `join`. + * @param on An optional condition for the join operation. This is equivilent to the `ON` + * clause in standard SQL. In the case of `Inner` joins, specifying a + * `condition` is equivilent to adding `where` clauses after the `join`. * * @group Query */ def join( otherPlan: SchemaRDD, joinType: JoinType = Inner, - condition: Option[Expression] = None): SchemaRDD = - new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, condition)) + on: Option[Expression] = None): SchemaRDD = + new SchemaRDD(sqlContext, Join(logicalPlan, otherPlan.logicalPlan, joinType, on)) /** * Sorts the results by the given expressions. @@ -195,14 +195,14 @@ class SchemaRDD( * with the same name, for example, when peforming self-joins. * * {{{ - * val x = schemaRDD.where('a === 1).subquery('x) - * val y = schemaRDD.where('a === 2).subquery('y) + * val x = schemaRDD.where('a === 1).as('x) + * val y = schemaRDD.where('a === 2).as('y) * x.join(y).where("x.a".attr === "y.a".attr), * }}} * * @group Query */ - def subquery(alias: Symbol) = + def as(alias: Symbol) = new SchemaRDD(sqlContext, Subquery(alias.name, logicalPlan)) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index acb1ee83a72f6..daa423cb8ea1a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -24,6 +24,7 @@ import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} import org.apache.spark.sql.catalyst.plans.physical._ +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { self: Product => @@ -69,6 +70,8 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) + case InMemoryColumnarTableScan(output, child) => + InMemoryColumnarTableScan(output.map(_.newInstance), child) case _ => sys.error("Multiple instance of the same relation detected.") }).asInstanceOf[this.type] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index e5902c3cae381..7c6a642278226 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -58,4 +58,17 @@ class CachedTableSuite extends QueryTest { TestSQLContext.uncacheTable("testData") } } + + test("SELECT Star Cached Table") { + TestSQLContext.sql("SELECT * FROM testData").registerAsTable("selectStar") + TestSQLContext.cacheTable("selectStar") + TestSQLContext.sql("SELECT * FROM selectStar") + TestSQLContext.uncacheTable("selectStar") + } + + test("Self-join cached") { + TestSQLContext.cacheTable("testData") + TestSQLContext.sql("SELECT * FROM testData a JOIN testData b ON a.key = b.key") + TestSQLContext.uncacheTable("testData") + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala index 2524a37cbac13..be0f4a4c73b36 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DslQuerySuite.scala @@ -119,8 +119,8 @@ class DslQuerySuite extends QueryTest { } test("inner join, where, multiple matches") { - val x = testData2.where('a === 1).subquery('x) - val y = testData2.where('a === 1).subquery('y) + val x = testData2.where('a === 1).as('x) + val y = testData2.where('a === 1).as('y) checkAnswer( x.join(y).where("x.a".attr === "y.a".attr), (1,1,1,1) :: @@ -131,8 +131,8 @@ class DslQuerySuite extends QueryTest { } test("inner join, no matches") { - val x = testData2.where('a === 1).subquery('x) - val y = testData2.where('a === 2).subquery('y) + val x = testData2.where('a === 1).as('x) + val y = testData2.where('a === 2).as('y) checkAnswer( x.join(y).where("x.a".attr === "y.a".attr), Nil) @@ -140,8 +140,8 @@ class DslQuerySuite extends QueryTest { test("big inner join, 4 matches per row") { val bigData = testData.unionAll(testData).unionAll(testData).unionAll(testData) - val bigDataX = bigData.subquery('x) - val bigDataY = bigData.subquery('y) + val bigDataX = bigData.as('x) + val bigDataY = bigData.as('y) checkAnswer( bigDataX.join(bigDataY).where("x.key".attr === "y.key".attr), @@ -181,8 +181,8 @@ class DslQuerySuite extends QueryTest { } test("full outer join") { - val left = upperCaseData.where('N <= 4).subquery('left) - val right = upperCaseData.where('N >= 3).subquery('right) + val left = upperCaseData.where('N <= 4).as('left) + val right = upperCaseData.where('N >= 3).as('right) checkAnswer( left.join(right, FullOuter, Some("left.N".attr === "right.N".attr)), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala index a62a3c4d02354..fc68d6c5620d3 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/parquet/ParquetQuerySuite.scala @@ -56,8 +56,8 @@ class ParquetQuerySuite extends FunSuite with BeforeAndAfterAll { } test("self-join parquet files") { - val x = ParquetTestData.testData.subquery('x) - val y = ParquetTestData.testData.subquery('y) + val x = ParquetTestData.testData.as('x) + val y = ParquetTestData.testData.as('y) val query = x.join(y).where("x.myint".attr === "y.myint".attr) // Check to make sure that the attributes from either side of the join have unique expression From 60e18ce7dd1016647b63586520b713efc45494a8 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 4 Apr 2014 17:29:29 -0700 Subject: [PATCH 200/397] SPARK-1414. Python API for SparkContext.wholeTextFiles Also clarified comment on each file having to fit in memory Author: Matei Zaharia Closes #327 from mateiz/py-whole-files and squashes the following commits: 9ad64a5 [Matei Zaharia] SPARK-1414. Python API for SparkContext.wholeTextFiles --- .../scala/org/apache/spark/SparkContext.scala | 2 +- .../spark/api/java/JavaSparkContext.scala | 2 +- .../apache/spark/api/python/PythonRDD.scala | 6 ++- python/pyspark/context.py | 44 ++++++++++++++++++- python/pyspark/serializers.py | 2 +- 5 files changed, 49 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 28a865c0ad3b5..835cffe37a938 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -395,7 +395,7 @@ class SparkContext( * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are perferred, large file is also allowable, but may cause bad performance. + * @note Small files are preferred, as each file will be loaded fully in memory. */ def wholeTextFiles(path: String): RDD[(String, String)] = { newAPIHadoopFile( diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 6cbdeac58d5e2..a2855d4db1d2e 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -177,7 +177,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are perferred, large file is also allowable, but may cause bad performance. + * @note Small files are preferred, as each file will be loaded fully in memory. */ def wholeTextFiles(path: String): JavaPairRDD[String, String] = new JavaPairRDD(sc.wholeTextFiles(path)) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index b67286a4e3b75..32f1100406d74 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -19,6 +19,7 @@ package org.apache.spark.api.python import java.io._ import java.net._ +import java.nio.charset.Charset import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collections} import scala.collection.JavaConversions._ @@ -206,6 +207,7 @@ private object SpecialLengths { } private[spark] object PythonRDD { + val UTF8 = Charset.forName("UTF-8") def readRDDFromFile(sc: JavaSparkContext, filename: String, parallelism: Int): JavaRDD[Array[Byte]] = { @@ -266,7 +268,7 @@ private[spark] object PythonRDD { } def writeUTF(str: String, dataOut: DataOutputStream) { - val bytes = str.getBytes("UTF-8") + val bytes = str.getBytes(UTF8) dataOut.writeInt(bytes.length) dataOut.write(bytes) } @@ -286,7 +288,7 @@ private[spark] object PythonRDD { private class BytesToString extends org.apache.spark.api.java.function.Function[Array[Byte], String] { - override def call(arr: Array[Byte]) : String = new String(arr, "UTF-8") + override def call(arr: Array[Byte]) : String = new String(arr, PythonRDD.UTF8) } /** diff --git a/python/pyspark/context.py b/python/pyspark/context.py index bf2454fd7e38e..ff1023bbfa539 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -28,7 +28,8 @@ from pyspark.conf import SparkConf from pyspark.files import SparkFiles from pyspark.java_gateway import launch_gateway -from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer +from pyspark.serializers import PickleSerializer, BatchedSerializer, UTF8Deserializer, \ + PairDeserializer from pyspark.storagelevel import StorageLevel from pyspark import rdd from pyspark.rdd import RDD @@ -257,6 +258,45 @@ def textFile(self, name, minSplits=None): return RDD(self._jsc.textFile(name, minSplits), self, UTF8Deserializer()) + def wholeTextFiles(self, path): + """ + Read a directory of text files from HDFS, a local file system + (available on all nodes), or any Hadoop-supported file system + URI. Each file is read as a single record and returned in a + key-value pair, where the key is the path of each file, the + value is the content of each file. + + For example, if you have the following files:: + + hdfs://a-hdfs-path/part-00000 + hdfs://a-hdfs-path/part-00001 + ... + hdfs://a-hdfs-path/part-nnnnn + + Do C{rdd = sparkContext.wholeTextFiles("hdfs://a-hdfs-path")}, + then C{rdd} contains:: + + (a-hdfs-path/part-00000, its content) + (a-hdfs-path/part-00001, its content) + ... + (a-hdfs-path/part-nnnnn, its content) + + NOTE: Small files are preferred, as each file will be loaded + fully in memory. + + >>> dirPath = os.path.join(tempdir, "files") + >>> os.mkdir(dirPath) + >>> with open(os.path.join(dirPath, "1.txt"), "w") as file1: + ... file1.write("1") + >>> with open(os.path.join(dirPath, "2.txt"), "w") as file2: + ... file2.write("2") + >>> textFiles = sc.wholeTextFiles(dirPath) + >>> sorted(textFiles.collect()) + [(u'.../1.txt', u'1'), (u'.../2.txt', u'2')] + """ + return RDD(self._jsc.wholeTextFiles(path), self, + PairDeserializer(UTF8Deserializer(), UTF8Deserializer())) + def _checkpointFile(self, name, input_deserializer): jrdd = self._jsc.checkpointFile(name) return RDD(jrdd, self, input_deserializer) @@ -425,7 +465,7 @@ def _test(): globs['sc'] = SparkContext('local[4]', 'PythonTest', batchSize=2) globs['tempdir'] = tempfile.mkdtemp() atexit.register(lambda: shutil.rmtree(globs['tempdir'])) - (failure_count, test_count) = doctest.testmod(globs=globs) + (failure_count, test_count) = doctest.testmod(globs=globs, optionflags=doctest.ELLIPSIS) globs['sc'].stop() if failure_count: exit(-1) diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 12c63f186a2b7..4d802924df4a1 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -290,7 +290,7 @@ class MarshalSerializer(FramedSerializer): class UTF8Deserializer(Serializer): """ - Deserializes streams written by getBytes. + Deserializes streams written by String.getBytes. """ def loads(self, stream): From 5f3c1bb5136b3389bea3af4fb39a083d979efa4c Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 4 Apr 2014 19:15:15 -0700 Subject: [PATCH 201/397] Add test utility for generating Jar files with compiled classes. This was requested by a few different people and may be generally useful, so I'd like to contribute this and not block on a different PR for it to get in. Author: Patrick Wendell Closes #326 from pwendell/class-loader-test-utils and squashes the following commits: ff3e88e [Patrick Wendell] Add test utility for generating Jar files with compiled classes. --- .../scala/org/apache/spark/TestUtils.scala | 98 +++++++++++++++++++ 1 file changed, 98 insertions(+) create mode 100644 core/src/test/scala/org/apache/spark/TestUtils.scala diff --git a/core/src/test/scala/org/apache/spark/TestUtils.scala b/core/src/test/scala/org/apache/spark/TestUtils.scala new file mode 100644 index 0000000000000..1611d09652d40 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/TestUtils.scala @@ -0,0 +1,98 @@ +/* + * 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 + +import java.io.{File, FileInputStream, FileOutputStream} +import java.net.{URI, URL} +import java.util.jar.{JarEntry, JarOutputStream} + +import scala.collection.JavaConversions._ + +import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} +import com.google.common.io.Files + +object TestUtils { + + /** + * Create a jar that defines classes with the given names. + * + * Note: if this is used during class loader tests, class names should be unique + * in order to avoid interference between tests. + */ + def createJarWithClasses(classNames: Seq[String]): URL = { + val tempDir = Files.createTempDir() + val files = for (name <- classNames) yield createCompiledClass(name, tempDir) + val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) + createJar(files, jarFile) + } + + /** + * Create a jar file that contains this set of files. All files will be located at the root + * of the jar. + */ + def createJar(files: Seq[File], jarFile: File): URL = { + val jarFileStream = new FileOutputStream(jarFile) + val jarStream = new JarOutputStream(jarFileStream, new java.util.jar.Manifest()) + + for (file <- files) { + val jarEntry = new JarEntry(file.getName) + jarStream.putNextEntry(jarEntry) + + val in = new FileInputStream(file) + val buffer = new Array[Byte](10240) + var nRead = 0 + while (nRead <= 0) { + nRead = in.read(buffer, 0, buffer.length) + jarStream.write(buffer, 0, nRead) + } + in.close() + } + jarStream.close() + jarFileStream.close() + + jarFile.toURI.toURL + } + + // Adapted from the JavaCompiler.java doc examples + private val SOURCE = JavaFileObject.Kind.SOURCE + private def createURI(name: String) = { + URI.create(s"string:///${name.replace(".", "/")}${SOURCE.extension}") + } + + private class JavaSourceFromString(val name: String, val code: String) + extends SimpleJavaFileObject(createURI(name), SOURCE) { + override def getCharContent(ignoreEncodingErrors: Boolean) = code + } + + /** Creates a compiled class with the given name. Class file will be placed in destDir. */ + def createCompiledClass(className: String, destDir: File): File = { + val compiler = ToolProvider.getSystemJavaCompiler + val sourceFile = new JavaSourceFromString(className, s"public class $className {}") + + // Calling this outputs a class file in pwd. It's easier to just rename the file than + // build a custom FileManager that controls the output location. + compiler.getTask(null, null, null, null, null, Seq(sourceFile)).call() + + val fileName = className + ".class" + val result = new File(fileName) + if (!result.exists()) throw new Exception("Compiled file not found: " + fileName) + val out = new File(destDir, fileName) + result.renameTo(out) + out + } +} From 1347ebd4b52ffb9197fc4137a55dff6badb149ba Mon Sep 17 00:00:00 2001 From: Mark Hamstra Date: Fri, 4 Apr 2014 19:19:48 -0700 Subject: [PATCH 202/397] [SPARK-1419] Bumped parent POM to apache 14 Keeping up-to-date with the parent, which includes some bugfixes. Author: Mark Hamstra Closes #328 from markhamstra/Apache14 and squashes the following commits: 3f19975 [Mark Hamstra] Bumped parent POM to apache 14 --- pom.xml | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/pom.xml b/pom.xml index 7d58060cba606..01341d21b7f23 100644 --- a/pom.xml +++ b/pom.xml @@ -21,7 +21,7 @@ org.apache apache - 13 + 14 org.apache.spark spark-parent From b50ddfde0342990979979e58348f54c10b500c90 Mon Sep 17 00:00:00 2001 From: Haoyuan Li Date: Fri, 4 Apr 2014 20:36:24 -0700 Subject: [PATCH 203/397] SPARK-1305: Support persisting RDD's directly to Tachyon Move the PR#468 of apache-incubator-spark to the apache-spark "Adding an option to persist Spark RDD blocks into Tachyon." Author: Haoyuan Li Author: RongGu Closes #158 from RongGu/master and squashes the following commits: 72b7768 [Haoyuan Li] merge master 9f7fa1b [Haoyuan Li] fix code style ae7834b [Haoyuan Li] minor cleanup a8b3ec6 [Haoyuan Li] merge master branch e0f4891 [Haoyuan Li] better check offheap. 55b5918 [RongGu] address matei's comment on the replication of offHeap storagelevel 7cd4600 [RongGu] remove some logic code for tachyonstore's replication 51149e7 [RongGu] address aaron's comment on returning value of the remove() function in tachyonstore 8adfcfa [RongGu] address arron's comment on inTachyonSize 120e48a [RongGu] changed the root-level dir name in Tachyon 5cc041c [Haoyuan Li] address aaron's comments 9b97935 [Haoyuan Li] address aaron's comments d9a6438 [Haoyuan Li] fix for pspark 77d2703 [Haoyuan Li] change python api.git status 3dcace4 [Haoyuan Li] address matei's comments 91fa09d [Haoyuan Li] address patrick's comments 589eafe [Haoyuan Li] use TRY_CACHE instead of MUST_CACHE 64348b2 [Haoyuan Li] update conf docs. ed73e19 [Haoyuan Li] Merge branch 'master' of github.com:RongGu/spark-1 619a9a8 [RongGu] set number of directories in TachyonStore back to 64; added a TODO tag for duplicated code from the DiskStore be79d77 [RongGu] find a way to clean up some unnecessay metods and classed to make the code simpler 49cc724 [Haoyuan Li] update docs with off_headp option 4572f9f [RongGu] reserving the old apply function API of StorageLevel 04301d3 [RongGu] rename StorageLevel.TACHYON to Storage.OFF_HEAP c9aeabf [RongGu] rename the StorgeLevel.TACHYON as StorageLevel.OFF_HEAP 76805aa [RongGu] unifies the config properties name prefix; add the configs into docs/configuration.md e700d9c [RongGu] add the SparkTachyonHdfsLR example and some comments fd84156 [RongGu] use randomUUID to generate sparkapp directory name on tachyon;minor code style fix 939e467 [Haoyuan Li] 0.4.1-thrift from maven central 86a2eab [Haoyuan Li] tachyon 0.4.1-thrift is in the staging repo. but jenkins failed to download it. temporarily revert it back to 0.4.1 16c5798 [RongGu] make the dependency on tachyon as tachyon-0.4.1-thrift eacb2e8 [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 bbeb4de [RongGu] fix the JsonProtocolSuite test failure problem 6adb58f [RongGu] Merge branch 'master' of https://github.com/RongGu/spark-1 d827250 [RongGu] fix JsonProtocolSuie test failure 716e93b [Haoyuan Li] revert the version ca14469 [Haoyuan Li] bump tachyon version to 0.4.1-thrift 2825a13 [RongGu] up-merging to the current master branch of the apache spark 6a22c1a [Haoyuan Li] fix scalastyle 8968b67 [Haoyuan Li] exclude more libraries from tachyon dependency to be the same as referencing tachyon-client. 77be7e8 [RongGu] address mateiz's comment about the temp folder name problem. The implementation followed mateiz's advice. 1dcadf9 [Haoyuan Li] typo bf278fa [Haoyuan Li] fix python tests e82909c [Haoyuan Li] minor cleanup 776a56c [Haoyuan Li] address patrick's and ali's comments from the previous PR 8859371 [Haoyuan Li] various minor fixes and clean up e3ddbba [Haoyuan Li] add doc to use Tachyon cache mode. fcaeab2 [Haoyuan Li] address Aaron's comment e554b1e [Haoyuan Li] add python code 47304b3 [Haoyuan Li] make tachyonStore in BlockMananger lazy val; add more comments StorageLevels. dc8ef24 [Haoyuan Li] add old storelevel constructor e01a271 [Haoyuan Li] update tachyon 0.4.1 8011a96 [RongGu] fix a brought-in mistake in StorageLevel 70ca182 [RongGu] a bit change in comment 556978b [RongGu] fix the scalastyle errors 791189b [RongGu] "Adding an option to persist Spark RDD blocks into Tachyon." move the PR#468 of apache-incubator-spark to the apache-spark --- core/pom.xml | 47 ++++++ .../apache/spark/api/java/StorageLevels.java | 46 ++++-- .../scala/org/apache/spark/SparkContext.scala | 10 +- .../CoarseGrainedExecutorBackend.scala | 6 +- .../spark/executor/ExecutorExitCode.scala | 9 + .../apache/spark/storage/BlockManager.scala | 86 ++++++++-- .../spark/storage/BlockManagerMaster.scala | 5 +- .../storage/BlockManagerMasterActor.scala | 37 +++-- .../spark/storage/BlockManagerMessages.scala | 17 +- .../apache/spark/storage/StorageLevel.scala | 72 +++++--- .../spark/storage/StorageStatusListener.scala | 2 +- .../apache/spark/storage/StorageUtils.scala | 23 ++- .../spark/storage/TachyonBlockManager.scala | 155 ++++++++++++++++++ .../spark/storage/TachyonFileSegment.scala | 28 ++++ .../apache/spark/storage/TachyonStore.scala | 142 ++++++++++++++++ .../apache/spark/ui/storage/IndexPage.scala | 3 + .../org/apache/spark/util/JsonProtocol.scala | 11 +- .../scala/org/apache/spark/util/Utils.scala | 46 +++++- .../spark/storage/BlockManagerSuite.scala | 25 ++- .../apache/spark/util/JsonProtocolSuite.scala | 20 +-- docs/configuration.md | 39 +++-- docs/scala-programming-guide.md | 127 ++++++++++---- .../org/apache/spark/examples/SparkPi.scala | 2 +- .../spark/examples/SparkTachyonHdfsLR.scala | 80 +++++++++ .../spark/examples/SparkTachyonPi.scala | 52 ++++++ project/SparkBuild.scala | 17 +- python/pyspark/context.py | 7 +- python/pyspark/rdd.py | 3 +- python/pyspark/storagelevel.py | 28 ++-- 29 files changed, 976 insertions(+), 169 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala create mode 100644 core/src/main/scala/org/apache/spark/storage/TachyonStore.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala diff --git a/core/pom.xml b/core/pom.xml index e4c32eff0cd77..66f9fc4961b03 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -200,6 +200,53 @@ derby test + + org.tachyonproject + tachyon + 0.4.1-thrift + + + org.apache.hadoop + hadoop-client + + + org.apache.curator + curator-recipes + + + org.eclipse.jetty + jetty-jsp + + + org.eclipse.jetty + jetty-webapp + + + org.eclipse.jetty + jetty-server + + + org.eclipse.jetty + jetty-servlet + + + junit + junit + + + org.powermock + powermock-module-junit4 + + + org.powermock + powermock-api-mockito + + + org.apache.curator + curator-test + + + org.scalatest scalatest_${scala.binary.version} diff --git a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java index 9f13b39909481..840a1bd93bfbb 100644 --- a/core/src/main/java/org/apache/spark/api/java/StorageLevels.java +++ b/core/src/main/java/org/apache/spark/api/java/StorageLevels.java @@ -23,17 +23,18 @@ * Expose some commonly useful storage level constants. */ public class StorageLevels { - public static final StorageLevel NONE = create(false, false, false, 1); - public static final StorageLevel DISK_ONLY = create(true, false, false, 1); - public static final StorageLevel DISK_ONLY_2 = create(true, false, false, 2); - public static final StorageLevel MEMORY_ONLY = create(false, true, true, 1); - public static final StorageLevel MEMORY_ONLY_2 = create(false, true, true, 2); - public static final StorageLevel MEMORY_ONLY_SER = create(false, true, false, 1); - public static final StorageLevel MEMORY_ONLY_SER_2 = create(false, true, false, 2); - public static final StorageLevel MEMORY_AND_DISK = create(true, true, true, 1); - public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, true, 2); - public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, 1); - public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, 2); + public static final StorageLevel NONE = create(false, false, false, false, 1); + public static final StorageLevel DISK_ONLY = create(true, false, false, false, 1); + public static final StorageLevel DISK_ONLY_2 = create(true, false, false, false, 2); + public static final StorageLevel MEMORY_ONLY = create(false, true, false, true, 1); + public static final StorageLevel MEMORY_ONLY_2 = create(false, true, false, true, 2); + public static final StorageLevel MEMORY_ONLY_SER = create(false, true, false, false, 1); + public static final StorageLevel MEMORY_ONLY_SER_2 = create(false, true, false, false, 2); + public static final StorageLevel MEMORY_AND_DISK = create(true, true, false, true, 1); + public static final StorageLevel MEMORY_AND_DISK_2 = create(true, true, false, true, 2); + public static final StorageLevel MEMORY_AND_DISK_SER = create(true, true, false, false, 1); + public static final StorageLevel MEMORY_AND_DISK_SER_2 = create(true, true, false, false, 2); + public static final StorageLevel OFF_HEAP = create(false, false, true, false, 1); /** * Create a new StorageLevel object. @@ -42,7 +43,26 @@ public class StorageLevels { * @param deserialized saved as deserialized objects, if true * @param replication replication factor */ - public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, int replication) { - return StorageLevel.apply(useDisk, useMemory, deserialized, replication); + @Deprecated + public static StorageLevel create(boolean useDisk, boolean useMemory, boolean deserialized, + int replication) { + return StorageLevel.apply(useDisk, useMemory, false, deserialized, replication); + } + + /** + * Create a new StorageLevel object. + * @param useDisk saved to disk, if true + * @param useMemory saved to memory, if true + * @param useOffHeap saved to Tachyon, if true + * @param deserialized saved as deserialized objects, if true + * @param replication replication factor + */ + public static StorageLevel create( + boolean useDisk, + boolean useMemory, + boolean useOffHeap, + boolean deserialized, + int replication) { + return StorageLevel.apply(useDisk, useMemory, useOffHeap, deserialized, replication); } } diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 835cffe37a938..fcf16ce1b278e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -19,14 +19,13 @@ package org.apache.spark import java.io._ import java.net.URI -import java.util.{Properties, UUID} import java.util.concurrent.atomic.AtomicInteger - +import java.util.{Properties, UUID} +import java.util.UUID.randomUUID import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.reflect.{ClassTag, classTag} - import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{ArrayWritable, BooleanWritable, BytesWritable, DoubleWritable, FloatWritable, IntWritable, LongWritable, NullWritable, Text, Writable} @@ -130,6 +129,11 @@ class SparkContext( val master = conf.get("spark.master") val appName = conf.get("spark.app.name") + // Generate the random name for a temp folder in Tachyon + // Add a timestamp as the suffix here to make it more safe + val tachyonFolderName = "spark-" + randomUUID.toString() + conf.set("spark.tachyonStore.folderName", tachyonFolderName) + val isLocal = (master == "local" || master.startsWith("local[")) if (master == "yarn-client") System.setProperty("SPARK_YARN_MODE", "true") diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 3486092a140fb..16887d8892b31 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -53,7 +53,8 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? - executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties) + executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, + false) case RegisterExecutorFailed(message) => logError("Slave registration failed: " + message) @@ -105,7 +106,8 @@ private[spark] object CoarseGrainedExecutorBackend { // set it val sparkHostPort = hostname + ":" + boundPort actorSystem.actorOf( - Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, sparkHostPort, cores), + Props(classOf[CoarseGrainedExecutorBackend], driverUrl, executorId, + sparkHostPort, cores), name = "Executor") workerUrl.foreach{ url => actorSystem.actorOf(Props(classOf[WorkerWatcher], url), name = "WorkerWatcher") diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index 210f3dbeebaca..ceff3a067d72a 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -41,6 +41,12 @@ object ExecutorExitCode { /** DiskStore failed to create a local temporary directory after many attempts. */ val DISK_STORE_FAILED_TO_CREATE_DIR = 53 + /** TachyonStore failed to initialize after many attempts. */ + val TACHYON_STORE_FAILED_TO_INITIALIZE = 54 + + /** TachyonStore failed to create a local temporary directory after many attempts. */ + val TACHYON_STORE_FAILED_TO_CREATE_DIR = 55 + def explainExitCode(exitCode: Int): String = { exitCode match { case UNCAUGHT_EXCEPTION => "Uncaught exception" @@ -48,6 +54,9 @@ object ExecutorExitCode { case OOM => "OutOfMemoryError" case DISK_STORE_FAILED_TO_CREATE_DIR => "Failed to create local directory (bad spark.local.dir?)" + case TACHYON_STORE_FAILED_TO_INITIALIZE => "TachyonStore failed to initialize." + case TACHYON_STORE_FAILED_TO_CREATE_DIR => + "TachyonStore failed to create a local temporary directory." case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 71584b6eb102a..19138d9dde697 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -19,22 +19,20 @@ package org.apache.spark.storage import java.io.{File, InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} - import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.util.Random - import akka.actor.{ActorSystem, Cancellable, Props} import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer - import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ + sealed trait Values case class ByteBufferValues(buffer: ByteBuffer) extends Values @@ -59,6 +57,17 @@ private[spark] class BlockManager( private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) private[storage] val diskStore = new DiskStore(this, diskBlockManager) + var tachyonInitialized = false + private[storage] lazy val tachyonStore: TachyonStore = { + val storeDir = conf.get("spark.tachyonStore.baseDir", "/tmp_spark_tachyon") + val appFolderName = conf.get("spark.tachyonStore.folderName") + val tachyonStorePath = s"${storeDir}/${appFolderName}/${this.executorId}" + val tachyonMaster = conf.get("spark.tachyonStore.url", "tachyon://localhost:19998") + val tachyonBlockManager = new TachyonBlockManager( + shuffleBlockManager, tachyonStorePath, tachyonMaster) + tachyonInitialized = true + new TachyonStore(this, tachyonBlockManager) + } // If we use Netty for shuffle, start a new Netty-based shuffle sender service. private val nettyPort: Int = { @@ -248,8 +257,10 @@ private[spark] class BlockManager( if (info.tellMaster) { val storageLevel = status.storageLevel val inMemSize = Math.max(status.memSize, droppedMemorySize) + val inTachyonSize = status.tachyonSize val onDiskSize = status.diskSize - master.updateBlockInfo(blockManagerId, blockId, storageLevel, inMemSize, onDiskSize) + master.updateBlockInfo( + blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inTachyonSize) } else true } @@ -259,22 +270,24 @@ private[spark] class BlockManager( * and the updated in-memory and on-disk sizes. */ private def getCurrentBlockStatus(blockId: BlockId, info: BlockInfo): BlockStatus = { - val (newLevel, inMemSize, onDiskSize) = info.synchronized { + val (newLevel, inMemSize, onDiskSize, inTachyonSize) = info.synchronized { info.level match { case null => - (StorageLevel.NONE, 0L, 0L) + (StorageLevel.NONE, 0L, 0L, 0L) case level => val inMem = level.useMemory && memoryStore.contains(blockId) + val inTachyon = level.useOffHeap && tachyonStore.contains(blockId) val onDisk = level.useDisk && diskStore.contains(blockId) val deserialized = if (inMem) level.deserialized else false - val replication = if (inMem || onDisk) level.replication else 1 - val storageLevel = StorageLevel(onDisk, inMem, deserialized, replication) + val replication = if (inMem || inTachyon || onDisk) level.replication else 1 + val storageLevel = StorageLevel(onDisk, inMem, inTachyon, deserialized, replication) val memSize = if (inMem) memoryStore.getSize(blockId) else 0L + val tachyonSize = if (inTachyon) tachyonStore.getSize(blockId) else 0L val diskSize = if (onDisk) diskStore.getSize(blockId) else 0L - (storageLevel, memSize, diskSize) + (storageLevel, memSize, diskSize, tachyonSize) } } - BlockStatus(newLevel, inMemSize, onDiskSize) + BlockStatus(newLevel, inMemSize, onDiskSize, inTachyonSize) } /** @@ -354,6 +367,24 @@ private[spark] class BlockManager( logDebug("Block " + blockId + " not found in memory") } } + + // Look for the block in Tachyon + if (level.useOffHeap) { + logDebug("Getting block " + blockId + " from tachyon") + if (tachyonStore.contains(blockId)) { + tachyonStore.getBytes(blockId) match { + case Some(bytes) => { + if (!asValues) { + return Some(bytes) + } else { + return Some(dataDeserialize(blockId, bytes)) + } + } + case None => + logDebug("Block " + blockId + " not found in tachyon") + } + } + } // Look for block on disk, potentially storing it back into memory if required: if (level.useDisk) { @@ -620,6 +651,23 @@ private[spark] class BlockManager( } // Keep track of which blocks are dropped from memory res.droppedBlocks.foreach { block => updatedBlocks += block } + } else if (level.useOffHeap) { + // Save to Tachyon. + val res = data match { + case IteratorValues(iterator) => + tachyonStore.putValues(blockId, iterator, level, false) + case ArrayBufferValues(array) => + tachyonStore.putValues(blockId, array, level, false) + case ByteBufferValues(bytes) => { + bytes.rewind(); + tachyonStore.putBytes(blockId, bytes, level) + } + } + size = res.size + res.data match { + case Right(newBytes) => bytesAfterPut = newBytes + case _ => + } } else { // Save directly to disk. // Don't get back the bytes unless we replicate them. @@ -644,8 +692,8 @@ private[spark] class BlockManager( val putBlockStatus = getCurrentBlockStatus(blockId, putBlockInfo) if (putBlockStatus.storageLevel != StorageLevel.NONE) { - // Now that the block is in either the memory or disk store, let other threads read it, - // and tell the master about it. + // Now that the block is in either the memory, tachyon, or disk store, + // let other threads read it, and tell the master about it. marked = true putBlockInfo.markReady(size) if (tellMaster) { @@ -707,7 +755,8 @@ private[spark] class BlockManager( */ var cachedPeers: Seq[BlockManagerId] = null private def replicate(blockId: BlockId, data: ByteBuffer, level: StorageLevel) { - val tLevel = StorageLevel(level.useDisk, level.useMemory, level.deserialized, 1) + val tLevel = StorageLevel( + level.useDisk, level.useMemory, level.useOffHeap, level.deserialized, 1) if (cachedPeers == null) { cachedPeers = master.getPeers(blockManagerId, level.replication - 1) } @@ -832,9 +881,10 @@ private[spark] class BlockManager( // Removals are idempotent in disk store and memory store. At worst, we get a warning. val removedFromMemory = memoryStore.remove(blockId) val removedFromDisk = diskStore.remove(blockId) - if (!removedFromMemory && !removedFromDisk) { + val removedFromTachyon = if (tachyonInitialized) tachyonStore.remove(blockId) else false + if (!removedFromMemory && !removedFromDisk && !removedFromTachyon) { logWarning("Block " + blockId + " could not be removed as it was not found in either " + - "the disk or memory store") + "the disk, memory, or tachyon store") } blockInfo.remove(blockId) if (tellMaster && info.tellMaster) { @@ -871,6 +921,9 @@ private[spark] class BlockManager( if (level.useDisk) { diskStore.remove(id) } + if (level.useOffHeap) { + tachyonStore.remove(id) + } iterator.remove() logInfo("Dropped block " + id) } @@ -946,6 +999,9 @@ private[spark] class BlockManager( blockInfo.clear() memoryStore.clear() diskStore.clear() + if (tachyonInitialized) { + tachyonStore.clear() + } metadataCleaner.cancel() broadcastCleaner.cancel() logInfo("BlockManager stopped") diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index ed6937851b836..4bc1b407ad106 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -63,9 +63,10 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long): Boolean = { + diskSize: Long, + tachyonSize: Long): Boolean = { val res = askDriverWithReply[Boolean]( - UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize)) + UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize)) logInfo("Updated info of block " + blockId) res } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index ff2652b640272..378f4cadc17d7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -73,10 +73,11 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus register(blockManagerId, maxMemSize, slaveActor) sender ! true - case UpdateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) => + case UpdateBlockInfo( + blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) => // TODO: Ideally we want to handle all the message replies in receive instead of in the // individual private methods. - updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size) + updateBlockInfo(blockManagerId, blockId, storageLevel, deserializedSize, size, tachyonSize) case GetLocations(blockId) => sender ! getLocations(blockId) @@ -246,7 +247,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + diskSize: Long, + tachyonSize: Long) { if (!blockManagerInfo.contains(blockManagerId)) { if (blockManagerId.executorId == "" && !isLocal) { @@ -265,7 +267,8 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus return } - blockManagerInfo(blockManagerId).updateBlockInfo(blockId, storageLevel, memSize, diskSize) + blockManagerInfo(blockManagerId).updateBlockInfo( + blockId, storageLevel, memSize, diskSize, tachyonSize) var locations: mutable.HashSet[BlockManagerId] = null if (blockLocations.containsKey(blockId)) { @@ -309,8 +312,11 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus } } - -private[spark] case class BlockStatus(storageLevel: StorageLevel, memSize: Long, diskSize: Long) +private[spark] case class BlockStatus( + storageLevel: StorageLevel, + memSize: Long, + diskSize: Long, + tachyonSize: Long) private[spark] class BlockManagerInfo( val blockManagerId: BlockManagerId, @@ -336,7 +342,8 @@ private[spark] class BlockManagerInfo( blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long) { + diskSize: Long, + tachyonSize: Long) { updateLastSeenMs() @@ -350,23 +357,29 @@ private[spark] class BlockManagerInfo( } if (storageLevel.isValid) { - /* isValid means it is either stored in-memory or on-disk. + /* isValid means it is either stored in-memory, on-disk or on-Tachyon. * But the memSize here indicates the data size in or dropped from memory, + * tachyonSize here indicates the data size in or dropped from Tachyon, * and the diskSize here indicates the data size in or dropped to disk. * They can be both larger than 0, when a block is dropped from memory to disk. * Therefore, a safe way to set BlockStatus is to set its info in accurate modes. */ if (storageLevel.useMemory) { - _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0)) + _blocks.put(blockId, BlockStatus(storageLevel, memSize, 0, 0)) _remainingMem -= memSize logInfo("Added %s in memory on %s (size: %s, free: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(memSize), Utils.bytesToString(_remainingMem))) } if (storageLevel.useDisk) { - _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize)) + _blocks.put(blockId, BlockStatus(storageLevel, 0, diskSize, 0)) logInfo("Added %s on disk on %s (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(diskSize))) } + if (storageLevel.useOffHeap) { + _blocks.put(blockId, BlockStatus(storageLevel, 0, 0, tachyonSize)) + logInfo("Added %s on tachyon on %s (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(tachyonSize))) + } } else if (_blocks.containsKey(blockId)) { // If isValid is not true, drop the block. val blockStatus: BlockStatus = _blocks.get(blockId) @@ -381,6 +394,10 @@ private[spark] class BlockManagerInfo( logInfo("Removed %s on %s on disk (size: %s)".format( blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.diskSize))) } + if (blockStatus.storageLevel.useOffHeap) { + logInfo("Removed %s on %s on tachyon (size: %s)".format( + blockId, blockManagerId.hostPort, Utils.bytesToString(blockStatus.tachyonSize))) + } } } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index bbb9529b5a0ca..8a36b5cc42dfd 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -53,11 +53,12 @@ private[storage] object BlockManagerMessages { var blockId: BlockId, var storageLevel: StorageLevel, var memSize: Long, - var diskSize: Long) + var diskSize: Long, + var tachyonSize: Long) extends ToBlockManagerMaster with Externalizable { - def this() = this(null, null, null, 0, 0) // For deserialization only + def this() = this(null, null, null, 0, 0, 0) // For deserialization only override def writeExternal(out: ObjectOutput) { blockManagerId.writeExternal(out) @@ -65,6 +66,7 @@ private[storage] object BlockManagerMessages { storageLevel.writeExternal(out) out.writeLong(memSize) out.writeLong(diskSize) + out.writeLong(tachyonSize) } override def readExternal(in: ObjectInput) { @@ -73,6 +75,7 @@ private[storage] object BlockManagerMessages { storageLevel = StorageLevel(in) memSize = in.readLong() diskSize = in.readLong() + tachyonSize = in.readLong() } } @@ -81,13 +84,15 @@ private[storage] object BlockManagerMessages { blockId: BlockId, storageLevel: StorageLevel, memSize: Long, - diskSize: Long): UpdateBlockInfo = { - new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize) + diskSize: Long, + tachyonSize: Long): UpdateBlockInfo = { + new UpdateBlockInfo(blockManagerId, blockId, storageLevel, memSize, diskSize, tachyonSize) } // For pattern-matching - def unapply(h: UpdateBlockInfo): Option[(BlockManagerId, BlockId, StorageLevel, Long, Long)] = { - Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize)) + def unapply(h: UpdateBlockInfo) + : Option[(BlockManagerId, BlockId, StorageLevel, Long, Long, Long)] = { + Some((h.blockManagerId, h.blockId, h.storageLevel, h.memSize, h.diskSize, h.tachyonSize)) } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 4212a539dab4b..95e71de2d3f1d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -21,8 +21,9 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, - * whether to drop the RDD to disk if it falls out of memory, whether to keep the data in memory - * in a serialized format, and whether to replicate the RDD partitions on multiple nodes. + * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to + * keep the data in memory in a serialized format, and whether to replicate the RDD partitions on + * multiple nodes. * The [[org.apache.spark.storage.StorageLevel$]] singleton object contains some static constants * for commonly useful storage levels. To create your own storage level object, use the * factory method of the singleton object (`StorageLevel(...)`). @@ -30,45 +31,58 @@ import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} class StorageLevel private( private var useDisk_ : Boolean, private var useMemory_ : Boolean, + private var useOffHeap_ : Boolean, private var deserialized_ : Boolean, private var replication_ : Int = 1) extends Externalizable { // TODO: Also add fields for caching priority, dataset ID, and flushing. private def this(flags: Int, replication: Int) { - this((flags & 4) != 0, (flags & 2) != 0, (flags & 1) != 0, replication) + this((flags & 8) != 0, (flags & 4) != 0, (flags & 2) != 0, (flags & 1) != 0, replication) } - def this() = this(false, true, false) // For deserialization + def this() = this(false, true, false, false) // For deserialization def useDisk = useDisk_ def useMemory = useMemory_ + def useOffHeap = useOffHeap_ def deserialized = deserialized_ def replication = replication_ assert(replication < 40, "Replication restricted to be less than 40 for calculating hashcodes") + if (useOffHeap) { + require(useDisk == false, "Off-heap storage level does not support using disk") + require(useMemory == false, "Off-heap storage level does not support using heap memory") + require(deserialized == false, "Off-heap storage level does not support deserialized storage") + require(replication == 1, "Off-heap storage level does not support multiple replication") + } + override def clone(): StorageLevel = new StorageLevel( - this.useDisk, this.useMemory, this.deserialized, this.replication) + this.useDisk, this.useMemory, this.useOffHeap, this.deserialized, this.replication) override def equals(other: Any): Boolean = other match { case s: StorageLevel => s.useDisk == useDisk && s.useMemory == useMemory && + s.useOffHeap == useOffHeap && s.deserialized == deserialized && s.replication == replication case _ => false } - def isValid = ((useMemory || useDisk) && (replication > 0)) + def isValid = ((useMemory || useDisk || useOffHeap) && (replication > 0)) def toInt: Int = { var ret = 0 if (useDisk_) { - ret |= 4 + ret |= 8 } if (useMemory_) { + ret |= 4 + } + if (useOffHeap_) { ret |= 2 } if (deserialized_) { @@ -84,8 +98,9 @@ class StorageLevel private( override def readExternal(in: ObjectInput) { val flags = in.readByte() - useDisk_ = (flags & 4) != 0 - useMemory_ = (flags & 2) != 0 + useDisk_ = (flags & 8) != 0 + useMemory_ = (flags & 4) != 0 + useOffHeap_ = (flags & 2) != 0 deserialized_ = (flags & 1) != 0 replication_ = in.readByte() } @@ -93,14 +108,15 @@ class StorageLevel private( @throws(classOf[IOException]) private def readResolve(): Object = StorageLevel.getCachedStorageLevel(this) - override def toString: String = - "StorageLevel(%b, %b, %b, %d)".format(useDisk, useMemory, deserialized, replication) + override def toString: String = "StorageLevel(%b, %b, %b, %b, %d)".format( + useDisk, useMemory, useOffHeap, deserialized, replication) override def hashCode(): Int = toInt * 41 + replication def description : String = { var result = "" result += (if (useDisk) "Disk " else "") result += (if (useMemory) "Memory " else "") + result += (if (useOffHeap) "Tachyon " else "") result += (if (deserialized) "Deserialized " else "Serialized ") result += "%sx Replicated".format(replication) result @@ -113,22 +129,28 @@ class StorageLevel private( * new storage levels. */ object StorageLevel { - val NONE = new StorageLevel(false, false, false) - val DISK_ONLY = new StorageLevel(true, false, false) - val DISK_ONLY_2 = new StorageLevel(true, false, false, 2) - val MEMORY_ONLY = new StorageLevel(false, true, true) - val MEMORY_ONLY_2 = new StorageLevel(false, true, true, 2) - val MEMORY_ONLY_SER = new StorageLevel(false, true, false) - val MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, 2) - val MEMORY_AND_DISK = new StorageLevel(true, true, true) - val MEMORY_AND_DISK_2 = new StorageLevel(true, true, true, 2) - val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false) - val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, 2) + val NONE = new StorageLevel(false, false, false, false) + val DISK_ONLY = new StorageLevel(true, false, false, false) + val DISK_ONLY_2 = new StorageLevel(true, false, false, false, 2) + val MEMORY_ONLY = new StorageLevel(false, true, false, true) + val MEMORY_ONLY_2 = new StorageLevel(false, true, false, true, 2) + val MEMORY_ONLY_SER = new StorageLevel(false, true, false, false) + val MEMORY_ONLY_SER_2 = new StorageLevel(false, true, false, false, 2) + val MEMORY_AND_DISK = new StorageLevel(true, true, false, true) + val MEMORY_AND_DISK_2 = new StorageLevel(true, true, false, true, 2) + val MEMORY_AND_DISK_SER = new StorageLevel(true, true, false, false) + val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) + val OFF_HEAP = new StorageLevel(false, false, true, false) + + /** Create a new StorageLevel object without setting useOffHeap */ + def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, + deserialized: Boolean, replication: Int) = getCachedStorageLevel( + new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) /** Create a new StorageLevel object */ - def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, - replication: Int = 1): StorageLevel = - getCachedStorageLevel(new StorageLevel(useDisk, useMemory, deserialized, replication)) + def apply(useDisk: Boolean, useMemory: Boolean, + deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( + new StorageLevel(useDisk, useMemory, false, deserialized, replication)) /** Create a new StorageLevel object from its integer representation */ def apply(flags: Int, replication: Int): StorageLevel = diff --git a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala index 26565f56ad858..7a174959037be 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageStatusListener.scala @@ -44,7 +44,7 @@ private[spark] class StorageStatusListener extends SparkListener { storageStatusList.foreach { storageStatus => val unpersistedBlocksIds = storageStatus.rddBlocks.keys.filter(_.rddId == unpersistedRDDId) unpersistedBlocksIds.foreach { blockId => - storageStatus.blocks(blockId) = BlockStatus(StorageLevel.NONE, 0L, 0L) + storageStatus.blocks(blockId) = BlockStatus(StorageLevel.NONE, 0L, 0L, 0L) } } } diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 6153dfe0b7e13..ff6e84cf9819a 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -48,17 +48,23 @@ class StorageStatus( } private[spark] -class RDDInfo(val id: Int, val name: String, val numPartitions: Int, val storageLevel: StorageLevel) - extends Ordered[RDDInfo] { +class RDDInfo( + val id: Int, + val name: String, + val numPartitions: Int, + val storageLevel: StorageLevel) extends Ordered[RDDInfo] { var numCachedPartitions = 0 var memSize = 0L var diskSize = 0L + var tachyonSize= 0L override def toString = { - ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + - "DiskSize: %s").format(name, id, storageLevel.toString, numCachedPartitions, - numPartitions, Utils.bytesToString(memSize), Utils.bytesToString(diskSize)) + import Utils.bytesToString + ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s;" + + "TachyonSize: %s; DiskSize: %s").format( + name, id, storageLevel.toString, numCachedPartitions, numPartitions, + bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) } override def compare(that: RDDInfo) = { @@ -105,14 +111,17 @@ object StorageUtils { val rddInfoMap = rddInfos.map { info => (info.id, info) }.toMap val rddStorageInfos = blockStatusMap.flatMap { case (rddId, blocks) => - // Add up memory and disk sizes - val persistedBlocks = blocks.filter { status => status.memSize + status.diskSize > 0 } + // Add up memory, disk and Tachyon sizes + val persistedBlocks = + blocks.filter { status => status.memSize + status.diskSize + status.tachyonSize > 0 } val memSize = persistedBlocks.map(_.memSize).reduceOption(_ + _).getOrElse(0L) val diskSize = persistedBlocks.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + val tachyonSize = persistedBlocks.map(_.tachyonSize).reduceOption(_ + _).getOrElse(0L) rddInfoMap.get(rddId).map { rddInfo => rddInfo.numCachedPartitions = persistedBlocks.length rddInfo.memSize = memSize rddInfo.diskSize = diskSize + rddInfo.tachyonSize = tachyonSize rddInfo } }.toArray diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala new file mode 100644 index 0000000000000..b0b9674856568 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TachyonBlockManager.scala @@ -0,0 +1,155 @@ +/* + * 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.storage + +import java.text.SimpleDateFormat +import java.util.{Date, Random} + +import tachyon.client.TachyonFS +import tachyon.client.TachyonFile + +import org.apache.spark.Logging +import org.apache.spark.executor.ExecutorExitCode +import org.apache.spark.network.netty.ShuffleSender +import org.apache.spark.util.Utils + + +/** + * Creates and maintains the logical mapping between logical blocks and tachyon fs locations. By + * default, one block is mapped to one file with a name given by its BlockId. + * + * @param rootDirs The directories to use for storing block files. Data will be hashed among these. + */ +private[spark] class TachyonBlockManager( + shuffleManager: ShuffleBlockManager, + rootDirs: String, + val master: String) + extends Logging { + + val client = if (master != null && master != "") TachyonFS.get(master) else null + + if (client == null) { + logError("Failed to connect to the Tachyon as the master address is not configured") + System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_INITIALIZE) + } + + private val MAX_DIR_CREATION_ATTEMPTS = 10 + private val subDirsPerTachyonDir = + shuffleManager.conf.get("spark.tachyonStore.subDirectories", "64").toInt + + // Create one Tachyon directory for each path mentioned in spark.tachyonStore.folderName; + // then, inside this directory, create multiple subdirectories that we will hash files into, + // in order to avoid having really large inodes at the top level in Tachyon. + private val tachyonDirs: Array[TachyonFile] = createTachyonDirs() + private val subDirs = Array.fill(tachyonDirs.length)(new Array[TachyonFile](subDirsPerTachyonDir)) + + addShutdownHook() + + def removeFile(file: TachyonFile): Boolean = { + client.delete(file.getPath(), false) + } + + def fileExists(file: TachyonFile): Boolean = { + client.exist(file.getPath()) + } + + def getFile(filename: String): TachyonFile = { + // Figure out which tachyon directory it hashes to, and which subdirectory in that + val hash = Utils.nonNegativeHash(filename) + val dirId = hash % tachyonDirs.length + val subDirId = (hash / tachyonDirs.length) % subDirsPerTachyonDir + + // Create the subdirectory if it doesn't already exist + var subDir = subDirs(dirId)(subDirId) + if (subDir == null) { + subDir = subDirs(dirId).synchronized { + val old = subDirs(dirId)(subDirId) + if (old != null) { + old + } else { + val path = tachyonDirs(dirId) + "/" + "%02x".format(subDirId) + client.mkdir(path) + val newDir = client.getFile(path) + subDirs(dirId)(subDirId) = newDir + newDir + } + } + } + val filePath = subDir + "/" + filename + if(!client.exist(filePath)) { + client.createFile(filePath) + } + val file = client.getFile(filePath) + file + } + + def getFile(blockId: BlockId): TachyonFile = getFile(blockId.name) + + // TODO: Some of the logic here could be consolidated/de-duplicated with that in the DiskStore. + private def createTachyonDirs(): Array[TachyonFile] = { + logDebug("Creating tachyon directories at root dirs '" + rootDirs + "'") + val dateFormat = new SimpleDateFormat("yyyyMMddHHmmss") + rootDirs.split(",").map { rootDir => + var foundLocalDir = false + var tachyonDir: TachyonFile = null + var tachyonDirId: String = null + var tries = 0 + val rand = new Random() + while (!foundLocalDir && tries < MAX_DIR_CREATION_ATTEMPTS) { + tries += 1 + try { + tachyonDirId = "%s-%04x".format(dateFormat.format(new Date), rand.nextInt(65536)) + val path = rootDir + "/" + "spark-tachyon-" + tachyonDirId + if (!client.exist(path)) { + foundLocalDir = client.mkdir(path) + tachyonDir = client.getFile(path) + } + } catch { + case e: Exception => + logWarning("Attempt " + tries + " to create tachyon dir " + tachyonDir + " failed", e) + } + } + if (!foundLocalDir) { + logError("Failed " + MAX_DIR_CREATION_ATTEMPTS + " attempts to create tachyon dir in " + + rootDir) + System.exit(ExecutorExitCode.TACHYON_STORE_FAILED_TO_CREATE_DIR) + } + logInfo("Created tachyon directory at " + tachyonDir) + tachyonDir + } + } + + private def addShutdownHook() { + tachyonDirs.foreach(tachyonDir => Utils.registerShutdownDeleteDir(tachyonDir)) + Runtime.getRuntime.addShutdownHook(new Thread("delete Spark tachyon dirs") { + override def run() { + logDebug("Shutdown hook called") + tachyonDirs.foreach { tachyonDir => + try { + if (!Utils.hasRootAsShutdownDeleteDir(tachyonDir)) { + Utils.deleteRecursively(tachyonDir, client) + } + } catch { + case t: Throwable => + logError("Exception while deleting tachyon spark dir: " + tachyonDir, t) + } + } + } + }) + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala new file mode 100644 index 0000000000000..b86abbda1d3e7 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TachyonFileSegment.scala @@ -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. + */ + +package org.apache.spark.storage + +import tachyon.client.TachyonFile + +/** + * References a particular segment of a file (potentially the entire file), based off an offset and + * a length. + */ +private[spark] class TachyonFileSegment(val file: TachyonFile, val offset: Long, val length: Long) { + override def toString = "(name=%s, offset=%d, length=%d)".format(file.getPath(), offset, length) +} diff --git a/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala new file mode 100644 index 0000000000000..c37e76f893605 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/TachyonStore.scala @@ -0,0 +1,142 @@ +/* + * 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.storage + +import java.io.IOException +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import tachyon.client.{WriteType, ReadType} + +import org.apache.spark.Logging +import org.apache.spark.util.Utils +import org.apache.spark.serializer.Serializer + + +private class Entry(val size: Long) + + +/** + * Stores BlockManager blocks on Tachyon. + */ +private class TachyonStore( + blockManager: BlockManager, + tachyonManager: TachyonBlockManager) + extends BlockStore(blockManager: BlockManager) with Logging { + + logInfo("TachyonStore started") + + override def getSize(blockId: BlockId): Long = { + tachyonManager.getFile(blockId.name).length + } + + override def putBytes(blockId: BlockId, bytes: ByteBuffer, level: StorageLevel): PutResult = { + putToTachyonStore(blockId, bytes, true) + } + + override def putValues( + blockId: BlockId, + values: ArrayBuffer[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { + return putValues(blockId, values.toIterator, level, returnValues) + } + + override def putValues( + blockId: BlockId, + values: Iterator[Any], + level: StorageLevel, + returnValues: Boolean): PutResult = { + logDebug("Attempting to write values for block " + blockId) + val _bytes = blockManager.dataSerialize(blockId, values) + putToTachyonStore(blockId, _bytes, returnValues) + } + + private def putToTachyonStore( + blockId: BlockId, + bytes: ByteBuffer, + returnValues: Boolean): PutResult = { + // So that we do not modify the input offsets ! + // duplicate does not copy buffer, so inexpensive + val byteBuffer = bytes.duplicate() + byteBuffer.rewind() + logDebug("Attempting to put block " + blockId + " into Tachyon") + val startTime = System.currentTimeMillis + val file = tachyonManager.getFile(blockId) + val os = file.getOutStream(WriteType.TRY_CACHE) + os.write(byteBuffer.array()) + os.close() + val finishTime = System.currentTimeMillis + logDebug("Block %s stored as %s file in Tachyon in %d ms".format( + blockId, Utils.bytesToString(byteBuffer.limit), (finishTime - startTime))) + + if (returnValues) { + PutResult(bytes.limit(), Right(bytes.duplicate())) + } else { + PutResult(bytes.limit(), null) + } + } + + override def remove(blockId: BlockId): Boolean = { + val file = tachyonManager.getFile(blockId) + if (tachyonManager.fileExists(file)) { + tachyonManager.removeFile(file) + } else { + false + } + } + + override def getValues(blockId: BlockId): Option[Iterator[Any]] = { + getBytes(blockId).map(buffer => blockManager.dataDeserialize(blockId, buffer)) + } + + + override def getBytes(blockId: BlockId): Option[ByteBuffer] = { + val file = tachyonManager.getFile(blockId) + if (file == null || file.getLocationHosts().size == 0) { + return None + } + val is = file.getInStream(ReadType.CACHE) + var buffer: ByteBuffer = null + try { + if (is != null) { + val size = file.length + val bs = new Array[Byte](size.asInstanceOf[Int]) + val fetchSize = is.read(bs, 0, size.asInstanceOf[Int]) + buffer = ByteBuffer.wrap(bs) + if (fetchSize != size) { + logWarning("Failed to fetch the block " + blockId + " from Tachyon : Size " + size + + " is not equal to fetched size " + fetchSize) + return None + } + } + } catch { + case ioe: IOException => { + logWarning("Failed to fetch the block " + blockId + " from Tachyon", ioe) + return None + } + } + Some(buffer) + } + + override def contains(blockId: BlockId): Boolean = { + val file = tachyonManager.getFile(blockId) + tachyonManager.fileExists(file) + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index b2732de51058a..0fa461e5e9d27 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -33,6 +33,7 @@ private[ui] class IndexPage(parent: BlockManagerUI) { private lazy val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { + val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) @@ -45,6 +46,7 @@ private[ui] class IndexPage(parent: BlockManagerUI) { "Cached Partitions", "Fraction Cached", "Size in Memory", + "Size in Tachyon", "Size on Disk") /** Render an HTML row representing an RDD */ @@ -60,6 +62,7 @@ private[ui] class IndexPage(parent: BlockManagerUI) { {rdd.numCachedPartitions} {"%.0f%%".format(rdd.numCachedPartitions * 100.0 / rdd.numPartitions)} {Utils.bytesToString(rdd.memSize)} + {Utils.bytesToString(rdd.tachyonSize)} {Utils.bytesToString(rdd.diskSize)} } diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index d9a6af61872d1..2155a8888c85c 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -274,12 +274,14 @@ private[spark] object JsonProtocol { ("Number of Partitions" -> rddInfo.numPartitions) ~ ("Number of Cached Partitions" -> rddInfo.numCachedPartitions) ~ ("Memory Size" -> rddInfo.memSize) ~ + ("Tachyon Size" -> rddInfo.tachyonSize) ~ ("Disk Size" -> rddInfo.diskSize) } def storageLevelToJson(storageLevel: StorageLevel): JValue = { ("Use Disk" -> storageLevel.useDisk) ~ ("Use Memory" -> storageLevel.useMemory) ~ + ("Use Tachyon" -> storageLevel.useOffHeap) ~ ("Deserialized" -> storageLevel.deserialized) ~ ("Replication" -> storageLevel.replication) } @@ -288,6 +290,7 @@ private[spark] object JsonProtocol { val storageLevel = storageLevelToJson(blockStatus.storageLevel) ("Storage Level" -> storageLevel) ~ ("Memory Size" -> blockStatus.memSize) ~ + ("Tachyon Size" -> blockStatus.tachyonSize) ~ ("Disk Size" -> blockStatus.diskSize) } @@ -570,11 +573,13 @@ private[spark] object JsonProtocol { val numPartitions = (json \ "Number of Partitions").extract[Int] val numCachedPartitions = (json \ "Number of Cached Partitions").extract[Int] val memSize = (json \ "Memory Size").extract[Long] + val tachyonSize = (json \ "Tachyon Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize + rddInfo.tachyonSize = tachyonSize rddInfo.diskSize = diskSize rddInfo } @@ -582,16 +587,18 @@ private[spark] object JsonProtocol { def storageLevelFromJson(json: JValue): StorageLevel = { val useDisk = (json \ "Use Disk").extract[Boolean] val useMemory = (json \ "Use Memory").extract[Boolean] + val useTachyon = (json \ "Use Tachyon").extract[Boolean] val deserialized = (json \ "Deserialized").extract[Boolean] val replication = (json \ "Replication").extract[Int] - StorageLevel(useDisk, useMemory, deserialized, replication) + StorageLevel(useDisk, useMemory, useTachyon, deserialized, replication) } def blockStatusFromJson(json: JValue): BlockStatus = { val storageLevel = storageLevelFromJson(json \ "Storage Level") val memorySize = (json \ "Memory Size").extract[Long] val diskSize = (json \ "Disk Size").extract[Long] - BlockStatus(storageLevel, memorySize, diskSize) + val tachyonSize = (json \ "Tachyon Size").extract[Long] + BlockStatus(storageLevel, memorySize, diskSize, tachyonSize) } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 737b765e2aed6..d3c39dee330b2 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -34,11 +34,13 @@ import com.google.common.io.Files import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ +import tachyon.client.{TachyonFile,TachyonFS} import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.serializer.{DeserializationStream, SerializationStream, SerializerInstance} + /** * Various utility methods used by Spark. */ @@ -153,6 +155,7 @@ private[spark] object Utils extends Logging { } private val shutdownDeletePaths = new scala.collection.mutable.HashSet[String]() + private val shutdownDeleteTachyonPaths = new scala.collection.mutable.HashSet[String]() // Register the path to be deleted via shutdown hook def registerShutdownDeleteDir(file: File) { @@ -162,6 +165,14 @@ private[spark] object Utils extends Logging { } } + // Register the tachyon path to be deleted via shutdown hook + def registerShutdownDeleteDir(tachyonfile: TachyonFile) { + val absolutePath = tachyonfile.getPath() + shutdownDeleteTachyonPaths.synchronized { + shutdownDeleteTachyonPaths += absolutePath + } + } + // Is the path already registered to be deleted via a shutdown hook ? def hasShutdownDeleteDir(file: File): Boolean = { val absolutePath = file.getAbsolutePath() @@ -170,6 +181,14 @@ private[spark] object Utils extends Logging { } } + // Is the path already registered to be deleted via a shutdown hook ? + def hasShutdownDeleteTachyonDir(file: TachyonFile): Boolean = { + val absolutePath = file.getPath() + shutdownDeletePaths.synchronized { + shutdownDeletePaths.contains(absolutePath) + } + } + // Note: if file is child of some registered path, while not equal to it, then return true; // else false. This is to ensure that two shutdown hooks do not try to delete each others // paths - resulting in IOException and incomplete cleanup. @@ -186,6 +205,22 @@ private[spark] object Utils extends Logging { retval } + // Note: if file is child of some registered path, while not equal to it, then return true; + // else false. This is to ensure that two shutdown hooks do not try to delete each others + // paths - resulting in Exception and incomplete cleanup. + def hasRootAsShutdownDeleteDir(file: TachyonFile): Boolean = { + val absolutePath = file.getPath() + val retval = shutdownDeletePaths.synchronized { + shutdownDeletePaths.find { path => + !absolutePath.equals(path) && absolutePath.startsWith(path) + }.isDefined + } + if (retval) { + logInfo("path = " + file + ", already present as root for deletion.") + } + retval + } + /** Create a temporary directory inside the given parent directory */ def createTempDir(root: String = System.getProperty("java.io.tmpdir")): File = { var attempts = 0 @@ -541,7 +576,16 @@ private[spark] object Utils extends Logging { } /** - * Check to see if file is a symbolic link. + * Delete a file or directory and its contents recursively. + */ + def deleteRecursively(dir: TachyonFile, client: TachyonFS) { + if (!client.delete(dir.getPath(), true)) { + throw new IOException("Failed to delete the tachyon dir: " + dir) + } + } + + /** + * Check to see if file is a symbolic link. */ def isSymlink(file: File): Boolean = { if (file == null) throw new NullPointerException("File must not be null") diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index e83cd55e73691..b6dd0526105a0 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -96,9 +96,9 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("StorageLevel object caching") { - val level1 = StorageLevel(false, false, false, 3) - val level2 = StorageLevel(false, false, false, 3) // this should return the same object as level1 - val level3 = StorageLevel(false, false, false, 2) // this should return a different object + val level1 = StorageLevel(false, false, false, false, 3) + val level2 = StorageLevel(false, false, false, false, 3) // this should return the same object as level1 + val level3 = StorageLevel(false, false, false, false, 2) // this should return a different object assert(level2 === level1, "level2 is not same as level1") assert(level2.eq(level1), "level2 is not the same object as level1") assert(level3 != level1, "level3 is same as level1") @@ -410,6 +410,25 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(store.memoryStore.contains(rdd(0, 3)), "rdd_0_3 was not in store") } + test("tachyon storage") { + // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. + val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) + if (tachyonUnitTestEnabled) { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + store.putSingle("a1", a1, StorageLevel.OFF_HEAP) + store.putSingle("a2", a2, StorageLevel.OFF_HEAP) + store.putSingle("a3", a3, StorageLevel.OFF_HEAP) + assert(store.getSingle("a3").isDefined, "a3 was in store") + assert(store.getSingle("a2").isDefined, "a2 was in store") + assert(store.getSingle("a1").isDefined, "a1 was in store") + } else { + info("tachyon storage test disabled.") + } + } + test("on-disk storage") { store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) val a1 = new Array[Byte](400) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 40c29014c4b59..054eb01a64c11 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -456,7 +456,7 @@ class JsonProtocolSuite extends FunSuite { t.shuffleWriteMetrics = Some(sw) // Make at most 6 blocks t.updatedBlocks = Some((1 to (e % 5 + 1)).map { i => - (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i)) + (RDDBlockId(e % i, f % i), BlockStatus(StorageLevel.MEMORY_AND_DISK_SER_2, a % i, b % i, c%i)) }.toSeq) t } @@ -470,19 +470,19 @@ class JsonProtocolSuite extends FunSuite { """ {"Event":"SparkListenerStageSubmitted","Stage Info":{"Stage ID":100,"Stage Name": "greetings","Number of Tasks":200,"RDD Info":{"RDD ID":100,"Name":"mayor","Storage - Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, - "Number of Partitions":200,"Number of Cached Partitions":300,"Memory Size":400, - "Disk Size":500},"Emitted Task Size Warning":false},"Properties":{"France":"Paris", - "Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} + Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, + "Replication":1},"Number of Partitions":200,"Number of Cached Partitions":300, + "Memory Size":400,"Disk Size":500,"Tachyon Size":0},"Emitted Task Size Warning":false}, + "Properties":{"France":"Paris","Germany":"Berlin","Russia":"Moscow","Ukraine":"Kiev"}} """ private val stageCompletedJsonString = """ {"Event":"SparkListenerStageCompleted","Stage Info":{"Stage ID":101,"Stage Name": "greetings","Number of Tasks":201,"RDD Info":{"RDD ID":101,"Name":"mayor","Storage - Level":{"Use Disk":true,"Use Memory":true,"Deserialized":true,"Replication":1}, - "Number of Partitions":201,"Number of Cached Partitions":301,"Memory Size":401, - "Disk Size":501},"Emitted Task Size Warning":false}} + Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":true, + "Replication":1},"Number of Partitions":201,"Number of Cached Partitions":301, + "Memory Size":401,"Disk Size":501,"Tachyon Size":0},"Emitted Task Size Warning":false}} """ private val taskStartJsonString = @@ -515,8 +515,8 @@ class JsonProtocolSuite extends FunSuite { 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": - {"Storage Level":{"Use Disk":true,"Use Memory":true,"Deserialized":false, - "Replication":2},"Memory Size":0,"Disk Size":0}}]}} + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, + "Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} """ private val jobStartJsonString = diff --git a/docs/configuration.md b/docs/configuration.md index 1ff0150567255..b6005acac8b93 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -122,6 +122,21 @@ Apart from these, the following properties are also available, and may be useful spark.storage.memoryFraction. + + spark.tachyonStore.baseDir + System.getProperty("java.io.tmpdir") + + Directories of the Tachyon File System that store RDDs. The Tachyon file system's URL is set by spark.tachyonStore.url. + It can also be a comma-separated list of multiple directories on Tachyon file system. + + + + spark.tachyonStore.url + tachyon://localhost:19998 + + The URL of the underlying Tachyon file system in the TachyonStore. + + spark.mesos.coarse false @@ -161,13 +176,13 @@ Apart from these, the following properties are also available, and may be useful spark.ui.acls.enable false - Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has + Whether spark web ui acls should are enabled. If enabled, this checks to see if the user has access permissions to view the web ui. See spark.ui.view.acls for more details. Also note this requires the user to be known, if the user comes across as null no checks are done. Filters can be used to authenticate and set the user. - + spark.ui.view.acls Empty @@ -276,10 +291,10 @@ Apart from these, the following properties are also available, and may be useful spark.serializer.objectStreamReset 10000 - When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches - objects to prevent writing redundant data, however that stops garbage collection of those - objects. By calling 'reset' you flush that info from the serializer, and allow old - objects to be collected. To turn off this periodic reset set it to a value of <= 0. + When serializing using org.apache.spark.serializer.JavaSerializer, the serializer caches + objects to prevent writing redundant data, however that stops garbage collection of those + objects. By calling 'reset' you flush that info from the serializer, and allow old + objects to be collected. To turn off this periodic reset set it to a value of <= 0. By default it will reset the serializer every 10,000 objects. @@ -375,7 +390,7 @@ Apart from these, the following properties are also available, and may be useful spark.akka.heartbeat.interval 1000 - This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those. + This is set to a larger value to disable failure detector that comes inbuilt akka. It can be enabled again, if you plan to use this feature (Not recommended). A larger interval value in seconds reduces network overhead and a smaller value ( ~ 1 s) might be more informative for akka's failure detector. Tune this in combination of `spark.akka.heartbeat.pauses` and `spark.akka.failure-detector.threshold` if you need to. Only positive use case for using failure detector can be, a sensistive failure detector can help evict rogue executors really quick. However this is usually not the case as gc pauses and network lags are expected in a real spark cluster. Apart from that enabling this leads to a lot of exchanges of heart beats between nodes leading to flooding the network with those. @@ -430,7 +445,7 @@ Apart from these, the following properties are also available, and may be useful spark.broadcast.blockSize 4096 - Size of each piece of a block in kilobytes for TorrentBroadcastFactory. + Size of each piece of a block in kilobytes for TorrentBroadcastFactory. Too large a value decreases parallelism during broadcast (makes it slower); however, if it is too small, BlockManager might take a performance hit. @@ -555,7 +570,7 @@ Apart from these, the following properties are also available, and may be useful the driver. - + spark.authenticate false @@ -563,7 +578,7 @@ Apart from these, the following properties are also available, and may be useful running on Yarn. - + spark.authenticate.secret None @@ -571,12 +586,12 @@ Apart from these, the following properties are also available, and may be useful not running on Yarn and authentication is enabled. - + spark.core.connection.auth.wait.timeout 30 Number of seconds for the connection to wait for authentication to occur before timing - out and giving up. + out and giving up. diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 99412733d4268..77373890eead7 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -23,7 +23,7 @@ To write a Spark application, you need to add a dependency on Spark. If you use groupId = org.apache.spark artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} - version = {{site.SPARK_VERSION}} + version = {{site.SPARK_VERSION}} In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS: @@ -73,14 +73,14 @@ The master URL passed to Spark can be in one of the following formats: - - -
    Master URLMeaning
    local Run Spark locally with one worker thread (i.e. no parallelism at all).
    local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). +
    local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine).
    spark://HOST:PORT Connect to the given Spark standalone - cluster master. The port must be whichever one your master is configured to use, which is 7077 by default. +
    spark://HOST:PORT Connect to the given Spark standalone + cluster master. The port must be whichever one your master is configured to use, which is 7077 by default.
    mesos://HOST:PORT Connect to the given Mesos cluster. - The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, - which is 5050 by default. +
    mesos://HOST:PORT Connect to the given Mesos cluster. + The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, + which is 5050 by default.
    @@ -265,11 +265,25 @@ A complete list of actions is available in the [RDD API doc](api/core/index.html ## RDD Persistence -One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory across operations. When you persist an RDD, each node stores any slices of it that it computes in memory and reuses them in other actions on that dataset (or datasets derived from it). This allows future actions to be much faster (often by more than 10x). Caching is a key tool for building iterative algorithms with Spark and for interactive use from the interpreter. - -You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- if any partition of an RDD is lost, it will automatically be recomputed using the transformations that originally created it. - -In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or even replicate it across nodes. These levels are chosen by passing a [`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is: +One of the most important capabilities in Spark is *persisting* (or *caching*) a dataset in memory +across operations. When you persist an RDD, each node stores any slices of it that it computes in +memory and reuses them in other actions on that dataset (or datasets derived from it). This allows +future actions to be much faster (often by more than 10x). Caching is a key tool for building +iterative algorithms with Spark and for interactive use from the interpreter. + +You can mark an RDD to be persisted using the `persist()` or `cache()` methods on it. The first time +it is computed in an action, it will be kept in memory on the nodes. The cache is fault-tolerant -- +if any partition of an RDD is lost, it will automatically be recomputed using the transformations +that originally created it. + +In addition, each RDD can be stored using a different *storage level*, allowing you, for example, to +persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), +or replicate it across nodes, or store the data in off-heap memory in [Tachyon](http://tachyon-project.org/). +These levels are chosen by passing a +[`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) +object to `persist()`. The `cache()` method is a shorthand for using the default storage level, +which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of +available storage levels is: @@ -292,8 +306,16 @@ In addition, each RDD can be stored using a different *storage level*, allowing - + + + + + @@ -307,30 +329,59 @@ In addition, each RDD can be stored using a different *storage level*, allowing ### Which Storage Level to Choose? -Spark's storage levels are meant to provide different tradeoffs between memory usage and CPU efficiency. -We recommend going through the following process to select one: - -* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. This is the most - CPU-efficient option, allowing operations on the RDDs to run as fast as possible. -* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to make the objects - much more space-efficient, but still reasonably fast to access. -* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter a large - amount of the data. Otherwise, recomputing a partition is about as fast as reading it from disk. -* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve requests from a web - application). *All* the storage levels provide full fault tolerance by recomputing lost data, but the replicated ones - let you continue running tasks on the RDD without waiting to recompute a lost partition. - -If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the [`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. +Spark's storage levels are meant to provide different trade-offs between memory usage and CPU +efficiency. It allows uses to choose memory, disk, or Tachyon for storing data. We recommend going +through the following process to select one: + +* If your RDDs fit comfortably with the default storage level (`MEMORY_ONLY`), leave them that way. + This is the most CPU-efficient option, allowing operations on the RDDs to run as fast as possible. + +* If not, try using `MEMORY_ONLY_SER` and [selecting a fast serialization library](tuning.html) to +make the objects much more space-efficient, but still reasonably fast to access. You can also use +`OFF_HEAP` mode to store the data off the heap in [Tachyon](http://tachyon-project.org/). This will +significantly reduce JVM GC overhead. + +* Don't spill to disk unless the functions that computed your datasets are expensive, or they filter +a large amount of the data. Otherwise, recomputing a partition is about as fast as reading it from +disk. + +* Use the replicated storage levels if you want fast fault recovery (e.g. if using Spark to serve +requests from a web application). *All* the storage levels provide full fault tolerance by +recomputing lost data, but the replicated ones let you continue running tasks on the RDD without +waiting to recompute a lost partition. + +If you want to define your own storage level (say, with replication factor of 3 instead of 2), then +use the function factor method `apply()` of the +[`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. + +Spark has a block manager inside the Executors that let you chose memory, disk, or off-heap. The +latter is for storing RDDs off-heap outside the Executor JVM on top of the memory management system +[Tachyon](http://tachyon-project.org/). This mode has the following advantages: + +* Cached data will not be lost if individual executors crash. +* Executors can have a smaller memory footprint, allowing you to run more executors on the same +machine as the bulk of the memory will be inside Tachyon. +* Reduced GC overhead since data is stored in Tachyon. # Shared Variables -Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a remote cluster node, it works on separate copies of all the variables used in the function. These variables are copied to each machine, and no updates to the variables on the remote machine are propagated back to the driver program. Supporting general, read-write shared variables across tasks would be inefficient. However, Spark does provide two limited types of *shared variables* for two common usage patterns: broadcast variables and accumulators. +Normally, when a function passed to a Spark operation (such as `map` or `reduce`) is executed on a +remote cluster node, it works on separate copies of all the variables used in the function. These +variables are copied to each machine, and no updates to the variables on the remote machine are +propagated back to the driver program. Supporting general, read-write shared variables across tasks +would be inefficient. However, Spark does provide two limited types of *shared variables* for two +common usage patterns: broadcast variables and accumulators. ## Broadcast Variables -Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables using efficient broadcast algorithms to reduce communication cost. +Broadcast variables allow the programmer to keep a read-only variable cached on each machine rather +than shipping a copy of it with tasks. They can be used, for example, to give every node a copy of a +large input dataset in an efficient manner. Spark also attempts to distribute broadcast variables +using efficient broadcast algorithms to reduce communication cost. -Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` method. The interpreter session below shows this: +Broadcast variables are created from a variable `v` by calling `SparkContext.broadcast(v)`. The +broadcast variable is a wrapper around `v`, and its value can be accessed by calling the `value` +method. The interpreter session below shows this: {% highlight scala %} scala> val broadcastVar = sc.broadcast(Array(1, 2, 3)) @@ -340,13 +391,21 @@ scala> broadcastVar.value res0: Array[Int] = Array(1, 2, 3) {% endhighlight %} -After the broadcast variable is created, it should be used instead of the value `v` in any functions run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object `v` should not be modified after it is broadcast in order to ensure that all nodes get the same value of the broadcast variable (e.g. if the variable is shipped to a new node later). +After the broadcast variable is created, it should be used instead of the value `v` in any functions +run on the cluster so that `v` is not shipped to the nodes more than once. In addition, the object +`v` should not be modified after it is broadcast in order to ensure that all nodes get the same +value of the broadcast variable (e.g. if the variable is shipped to a new node later). ## Accumulators -Accumulators are variables that are only "added" to through an associative operation and can therefore be efficiently supported in parallel. They can be used to implement counters (as in MapReduce) or sums. Spark natively supports accumulators of numeric value types and standard mutable collections, and programmers can add support for new types. +Accumulators are variables that are only "added" to through an associative operation and can +therefore be efficiently supported in parallel. They can be used to implement counters (as in +MapReduce) or sums. Spark natively supports accumulators of numeric value types and standard mutable +collections, and programmers can add support for new types. -An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks running on the cluster can then add to it using the `+=` operator. However, they cannot read its value. Only the driver program can read the accumulator's value, using its `value` method. +An accumulator is created from an initial value `v` by calling `SparkContext.accumulator(v)`. Tasks +running on the cluster can then add to it using the `+=` operator. However, they cannot read its +value. Only the driver program can read the accumulator's value, using its `value` method. The interpreter session below shows an accumulator being used to add up the elements of an array: diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index e5a09ecec006f..d3babc3ed12c8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -18,8 +18,8 @@ package org.apache.spark.examples import scala.math.random + import org.apache.spark._ -import SparkContext._ /** Computes an approximation to pi */ object SparkPi { diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala new file mode 100644 index 0000000000000..53b303d658386 --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -0,0 +1,80 @@ +/* + * 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.examples + +import java.util.Random +import scala.math.exp +import org.apache.spark.util.Vector +import org.apache.spark._ +import org.apache.spark.deploy.SparkHadoopUtil +import org.apache.spark.scheduler.InputFormatInfo +import org.apache.spark.storage.StorageLevel + +/** + * Logistic regression based classification. + * This example uses Tachyon to persist rdds during computation. + */ +object SparkTachyonHdfsLR { + val D = 10 // Numer of dimensions + val rand = new Random(42) + + case class DataPoint(x: Vector, y: Double) + + def parsePoint(line: String): DataPoint = { + val tok = new java.util.StringTokenizer(line, " ") + var y = tok.nextToken.toDouble + var x = new Array[Double](D) + var i = 0 + while (i < D) { + x(i) = tok.nextToken.toDouble; i += 1 + } + DataPoint(new Vector(x), y) + } + + def main(args: Array[String]) { + if (args.length < 3) { + System.err.println("Usage: SparkTachyonHdfsLR ") + System.exit(1) + } + val inputPath = args(1) + val conf = SparkHadoopUtil.get.newConfiguration() + val sc = new SparkContext(args(0), "SparkTachyonHdfsLR", + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + InputFormatInfo.computePreferredLocations( + Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) + )) + val lines = sc.textFile(inputPath) + val points = lines.map(parsePoint _).persist(StorageLevel.OFF_HEAP) + val ITERATIONS = args(2).toInt + + // Initialize w to a random value + var w = Vector(D, _ => 2 * rand.nextDouble - 1) + println("Initial w: " + w) + + for (i <- 1 to ITERATIONS) { + println("On iteration " + i) + val gradient = points.map { p => + (1 / (1 + exp(-p.y * (w dot p.x))) - 1) * p.y * p.x + }.reduce(_ + _) + w -= gradient + } + + println("Final w: " + w) + System.exit(0) + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala new file mode 100644 index 0000000000000..ce78f0876ed7c --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -0,0 +1,52 @@ +/* + * 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.examples + +import scala.math.random + +import org.apache.spark._ +import org.apache.spark.storage.StorageLevel + +/** + * Computes an approximation to pi + * This example uses Tachyon to persist rdds during computation. + */ +object SparkTachyonPi { + def main(args: Array[String]) { + if (args.length == 0) { + System.err.println("Usage: SparkTachyonPi []") + System.exit(1) + } + val spark = new SparkContext(args(0), "SparkTachyonPi", + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + + val slices = if (args.length > 1) args(1).toInt else 2 + val n = 100000 * slices + + val rdd = spark.parallelize(1 to n, slices) + rdd.persist(StorageLevel.OFF_HEAP) + val count = rdd.map { i => + val x = random * 2 - 1 + val y = random * 2 - 1 + if (x * x + y * y < 1) 1 else 0 + }.reduce(_ + _) + println("Pi is roughly " + 4.0 * count / n) + + spark.stop() + } +} diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index c5c697e8e2427..843a874fbfdb0 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -30,7 +30,7 @@ import scala.collection.JavaConversions._ // import com.jsuereth.pgp.sbtplugin.PgpKeys._ object SparkBuild extends Build { - val SPARK_VERSION = "1.0.0-SNAPSHOT" + val SPARK_VERSION = "1.0.0-SNAPSHOT" // Hadoop version to build against. For example, "1.0.4" for Apache releases, or // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set @@ -185,15 +185,14 @@ object SparkBuild extends Build { concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), resolvers ++= Seq( - // HTTPS is unavailable for Maven Central "Maven Repository" at "http://repo.maven.apache.org/maven2", "Apache Repository" at "https://repository.apache.org/content/repositories/releases", "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", "MQTT Repository" at "https://repo.eclipse.org/content/repositories/paho-releases/", - "Cloudera Repository" at "https://repository.cloudera.com/artifactory/cloudera-repos/", + "Cloudera Repository" at "http://repository.cloudera.com/artifactory/cloudera-repos/", // For Sonatype publishing - //"sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", - //"sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", + // "sonatype-snapshots" at "https://oss.sonatype.org/content/repositories/snapshots", + // "sonatype-staging" at "https://oss.sonatype.org/service/local/staging/deploy/maven2/", // also check the local Maven repository ~/.m2 Resolver.mavenLocal ), @@ -280,13 +279,18 @@ object SparkBuild extends Build { val slf4jVersion = "1.7.5" val excludeNetty = ExclusionRule(organization = "org.jboss.netty") + val excludeEclipseJetty = ExclusionRule(organization = "org.eclipse.jetty") val excludeAsm = ExclusionRule(organization = "org.ow2.asm") val excludeOldAsm = ExclusionRule(organization = "asm") val excludeCommonsLogging = ExclusionRule(organization = "commons-logging") val excludeSLF4J = ExclusionRule(organization = "org.slf4j") val excludeScalap = ExclusionRule(organization = "org.scala-lang", artifact = "scalap") + val excludeHadoop = ExclusionRule(organization = "org.apache.hadoop") + val excludeCurator = ExclusionRule(organization = "org.apache.curator") + val excludePowermock = ExclusionRule(organization = "org.powermock") - def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", + + def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { val fullId = if (crossVersion.isEmpty) id else id + "_" + crossVersion Some(organization % fullId % version) // the artifact to compare binary compatibility with @@ -323,6 +327,7 @@ object SparkBuild extends Build { "com.codahale.metrics" % "metrics-graphite" % "3.0.0", "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), + "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), libraryDependencies ++= maybeAvro diff --git a/python/pyspark/context.py b/python/pyspark/context.py index ff1023bbfa539..d8667e84fedff 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -423,8 +423,11 @@ def _getJavaStorageLevel(self, storageLevel): raise Exception("storageLevel must be of type pyspark.StorageLevel") newStorageLevel = self._jvm.org.apache.spark.storage.StorageLevel - return newStorageLevel(storageLevel.useDisk, storageLevel.useMemory, - storageLevel.deserialized, storageLevel.replication) + return newStorageLevel(storageLevel.useDisk, + storageLevel.useMemory, + storageLevel.useOffHeap, + storageLevel.deserialized, + storageLevel.replication) def setJobGroup(self, groupId, description): """ diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 9943296b927dc..fb27863e07f55 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -1302,11 +1302,12 @@ def getStorageLevel(self): Get the RDD's current storage level. >>> rdd1 = sc.parallelize([1,2]) >>> rdd1.getStorageLevel() - StorageLevel(False, False, False, 1) + StorageLevel(False, False, False, False, 1) """ java_storage_level = self._jrdd.getStorageLevel() storage_level = StorageLevel(java_storage_level.useDisk(), java_storage_level.useMemory(), + java_storage_level.useOffHeap(), java_storage_level.deserialized(), java_storage_level.replication()) return storage_level diff --git a/python/pyspark/storagelevel.py b/python/pyspark/storagelevel.py index c3e3a44e8e7ab..7b6660eab231b 100644 --- a/python/pyspark/storagelevel.py +++ b/python/pyspark/storagelevel.py @@ -25,23 +25,25 @@ class StorageLevel: Also contains static constants for some commonly used storage levels, such as MEMORY_ONLY. """ - def __init__(self, useDisk, useMemory, deserialized, replication = 1): + def __init__(self, useDisk, useMemory, useOffHeap, deserialized, replication = 1): self.useDisk = useDisk self.useMemory = useMemory + self.useOffHeap = useOffHeap self.deserialized = deserialized self.replication = replication def __repr__(self): - return "StorageLevel(%s, %s, %s, %s)" % ( - self.useDisk, self.useMemory, self.deserialized, self.replication) + return "StorageLevel(%s, %s, %s, %s, %s)" % ( + self.useDisk, self.useMemory, self.useOffHeap, self.deserialized, self.replication) -StorageLevel.DISK_ONLY = StorageLevel(True, False, False) -StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, 2) -StorageLevel.MEMORY_ONLY = StorageLevel(False, True, True) -StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, True, 2) -StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False) -StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, 2) -StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, True) -StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, True, 2) -StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False) -StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, 2) +StorageLevel.DISK_ONLY = StorageLevel(True, False, False, False) +StorageLevel.DISK_ONLY_2 = StorageLevel(True, False, False, False, 2) +StorageLevel.MEMORY_ONLY = StorageLevel(False, True, False, True) +StorageLevel.MEMORY_ONLY_2 = StorageLevel(False, True, False, True, 2) +StorageLevel.MEMORY_ONLY_SER = StorageLevel(False, True, False, False) +StorageLevel.MEMORY_ONLY_SER_2 = StorageLevel(False, True, False, False, 2) +StorageLevel.MEMORY_AND_DISK = StorageLevel(True, True, False, True) +StorageLevel.MEMORY_AND_DISK_2 = StorageLevel(True, True, False, True, 2) +StorageLevel.MEMORY_AND_DISK_SER = StorageLevel(True, True, False, False) +StorageLevel.MEMORY_AND_DISK_SER_2 = StorageLevel(True, True, False, False, 2) +StorageLevel.OFF_HEAP = StorageLevel(False, False, True, False, 1) \ No newline at end of file From 8de038eb366ded2ac74f72517e40545dbbab8cdd Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Fri, 4 Apr 2014 21:15:33 -0700 Subject: [PATCH 204/397] [SQL] SPARK-1366 Consistent sql function across different types of SQLContexts Now users who want to use HiveQL should explicitly say `hiveql` or `hql`. Author: Michael Armbrust Closes #319 from marmbrus/standardizeSqlHql and squashes the following commits: de68d0e [Michael Armbrust] Fix sampling test. fbe4a54 [Michael Armbrust] Make `sql` always use spark sql parser, users of hive context can now use hql or hiveql to run queries using HiveQL instead. --- .../spark/sql/examples/HiveFromSpark.scala | 12 ++--- .../apache/spark/sql/hive/HiveContext.scala | 17 ++++--- .../org/apache/spark/sql/hive/TestHive.scala | 12 ++--- .../hive/execution/HiveComparisonTest.scala | 10 ++-- .../sql/hive/execution/HiveQuerySuite.scala | 12 ++++- .../hive/execution/HiveResolutionSuite.scala | 2 +- .../sql/hive/execution/PruningSuite.scala | 2 +- .../spark/sql/parquet/HiveParquetSuite.scala | 46 +++++++++---------- 8 files changed, 63 insertions(+), 50 deletions(-) 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 index abcc1f04d4279..62329bde84481 100644 --- a/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala +++ b/examples/src/main/scala/org/apache/spark/sql/examples/HiveFromSpark.scala @@ -33,20 +33,20 @@ object HiveFromSpark { 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") + hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") + hql("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) + hql("SELECT * FROM src").collect.foreach(println) // Aggregation queries are also supported. - val count = sql("SELECT COUNT(*) FROM src").collect().head.getInt(0) + val count = hql("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") + val rddFromSql = hql("SELECT key, value FROM src WHERE key < 10 ORDER BY key") println("Result of RDD.map:") val rddAsStrings = rddFromSql.map { @@ -59,6 +59,6 @@ object HiveFromSpark { // 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) + hql("SELECT * FROM records r JOIN src s ON r.key = s.key").collect().foreach(println) } } 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 ff8eaacded4c8..f66a667c0a942 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 @@ -67,14 +67,13 @@ class LocalHiveContext(sc: SparkContext) extends HiveContext(sc) { class HiveContext(sc: SparkContext) extends SQLContext(sc) { self => - override def parseSql(sql: String): LogicalPlan = HiveQl.parseSql(sql) - override def executePlan(plan: LogicalPlan): this.QueryExecution = + override protected[sql] def executePlan(plan: LogicalPlan): this.QueryExecution = new this.QueryExecution { val logical = plan } /** * Executes a query expressed in HiveQL using Spark, returning the result as a SchemaRDD. */ - def hql(hqlQuery: String): SchemaRDD = { + def hiveql(hqlQuery: String): SchemaRDD = { val result = new SchemaRDD(this, HiveQl.parseSql(hqlQuery)) // We force query optimization to happen right away instead of letting it happen lazily like // when using the query DSL. This is so DDL commands behave as expected. This is only @@ -83,6 +82,9 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { result } + /** An alias for `hiveql`. */ + def hql(hqlQuery: String): SchemaRDD = hiveql(hqlQuery) + // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. @transient protected val outputBuffer = new java.io.OutputStream { @@ -120,7 +122,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* A catalyst metadata catalog that points to the Hive Metastore. */ @transient - override lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog { + override protected[sql] lazy val catalog = new HiveMetastoreCatalog(this) with OverrideCatalog { override def lookupRelation( databaseName: Option[String], tableName: String, @@ -132,7 +134,8 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { /* An analyzer that uses the Hive metastore. */ @transient - override lazy val analyzer = new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) + override protected[sql] lazy val analyzer = + new Analyzer(catalog, HiveFunctionRegistry, caseSensitive = false) /** * Runs the specified SQL query using Hive. @@ -214,14 +217,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { } @transient - override val planner = hivePlanner + override protected[sql] val planner = hivePlanner @transient 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 { + protected[sql] 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))) diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 0a6bea0162430..2fea9702954d7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -110,10 +110,10 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { 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 + protected[hive] class HiveQLQueryExecution(hql: String) extends this.QueryExecution { + lazy val logical = HiveQl.parseSql(hql) + def hiveExec() = runSqlHive(hql) + override def toString = hql + "\n" + super.toString } /** @@ -140,8 +140,8 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { case class TestTable(name: String, commands: (()=>Unit)*) - implicit class SqlCmd(sql: String) { - def cmd = () => new SqlQueryExecution(sql).stringResult(): Unit + protected[hive] implicit class SqlCmd(sql: String) { + def cmd = () => new HiveQLQueryExecution(sql).stringResult(): Unit } /** 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 18654b308d234..3cc4562a88d66 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 @@ -125,7 +125,7 @@ abstract class HiveComparisonTest } protected def prepareAnswer( - hiveQuery: TestHive.type#SqlQueryExecution, + hiveQuery: TestHive.type#HiveQLQueryExecution, answer: Seq[String]): Seq[String] = { val orderedAnswer = hiveQuery.logical match { // Clean out non-deterministic time schema info. @@ -227,7 +227,7 @@ abstract class HiveComparisonTest try { // MINOR HACK: You must run a query before calling reset the first time. - TestHive.sql("SHOW TABLES") + TestHive.hql("SHOW TABLES") if (reset) { TestHive.reset() } val hiveCacheFiles = queryList.zipWithIndex.map { @@ -256,7 +256,7 @@ abstract class HiveComparisonTest hiveCachedResults } else { - val hiveQueries = queryList.map(new TestHive.SqlQueryExecution(_)) + val hiveQueries = queryList.map(new TestHive.HiveQLQueryExecution(_)) // Make sure we can at least parse everything before attempting hive execution. hiveQueries.foreach(_.logical) val computedResults = (queryList.zipWithIndex, hiveQueries, hiveCacheFiles).zipped.map { @@ -302,7 +302,7 @@ abstract class HiveComparisonTest // Run w/ catalyst val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => - val query = new TestHive.SqlQueryExecution(queryString) + val query = new TestHive.HiveQLQueryExecution(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { case e: Exception => val errorMessage = @@ -359,7 +359,7 @@ abstract class HiveComparisonTest // 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 TestHive.SqlQueryExecution("SELECT key FROM src").stringResult() + new TestHive.HiveQLQueryExecution("SELECT key FROM src").stringResult() TestHive.runSqlHive("SELECT key FROM src") } catch { case e: Exception => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index c184ebe288af4..0c27498a93507 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -23,6 +23,16 @@ import org.apache.spark.sql.hive.TestHive._ * 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 { + + test("Query expressed in SQL") { + assert(sql("SELECT 1").collect() === Array(Seq(1))) + } + + test("Query expressed in HiveQL") { + hql("FROM src SELECT key").collect() + hiveql("FROM src SELECT key").collect() + } + createQueryTest("Simple Average", "SELECT AVG(key) FROM src") @@ -133,7 +143,7 @@ class HiveQuerySuite extends HiveComparisonTest { "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") + hql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 40c4e23f90fb8..8883e5b16d4da 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -56,7 +56,7 @@ class HiveResolutionSuite extends HiveComparisonTest { TestHive.sparkContext.parallelize(Data(1, 2, Nested(1,2)) :: Nil) .registerAsTable("caseSensitivityTest") - sql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") + hql("SELECT a, b, A, B, n.a, n.b, n.A, n.B FROM caseSensitivityTest") } /** diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index 1318ac1968dad..d9ccb93e23923 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -136,7 +136,7 @@ class PruningSuite extends HiveComparisonTest { expectedScannedColumns: Seq[String], expectedPartValues: Seq[Seq[String]]) = { test(s"$testCaseName - pruning test") { - val plan = new TestHive.SqlQueryExecution(sql).executedPlan + val plan = new TestHive.HiveQLQueryExecution(sql).executedPlan val actualOutputColumns = plan.output.map(_.name) val (actualScannedColumns, actualPartValues) = plan.collect { case p @ HiveTableScan(columns, relation, _) => diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 314ca48ad8f6a..aade62eb8f84e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -57,34 +57,34 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft } test("SELECT on Parquet table") { - val rdd = sql("SELECT * FROM testsource").collect() + val rdd = hql("SELECT * FROM testsource").collect() assert(rdd != null) assert(rdd.forall(_.size == 6)) } test("Simple column projection + filter on Parquet table") { - val rdd = sql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() + val rdd = hql("SELECT myboolean, mylong FROM testsource WHERE myboolean=true").collect() 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 saveAsParquetFile") { - sql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) + hql("SELECT * FROM src").saveAsParquetFile(dirname.getAbsolutePath) parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") - val rddOne = sql("SELECT * FROM src").collect().sortBy(_.getInt(0)) - val rddTwo = sql("SELECT * from ptable").collect().sortBy(_.getInt(0)) + val rddOne = hql("SELECT * FROM src").collect().sortBy(_.getInt(0)) + val rddTwo = hql("SELECT * from ptable").collect().sortBy(_.getInt(0)) compareRDDs(rddOne, rddTwo, "src (Hive)", Seq("key:Int", "value:String")) } test("INSERT OVERWRITE TABLE Parquet table") { - sql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) + hql("SELECT * FROM testsource").saveAsParquetFile(dirname.getAbsolutePath) parquetFile(dirname.getAbsolutePath).registerAsTable("ptable") // let's do three overwrites for good measure - sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - sql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() - val rddCopy = sql("SELECT * FROM ptable").collect() - val rddOrig = sql("SELECT * FROM testsource").collect() + hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + hql("INSERT OVERWRITE TABLE ptable SELECT * FROM testsource").collect() + val rddCopy = hql("SELECT * FROM ptable").collect() + val rddOrig = hql("SELECT * FROM testsource").collect() assert(rddCopy.size === rddOrig.size, "INSERT OVERWRITE changed size of table??") compareRDDs(rddOrig, rddCopy, "testsource", ParquetTestData.testSchemaFieldNames) } @@ -93,13 +93,13 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType)) .registerAsTable("tmp") val rddCopy = - sql("INSERT INTO TABLE tmp SELECT * FROM src") + hql("INSERT INTO TABLE tmp SELECT * FROM src") .collect() .sortBy[Int](_.apply(0) match { case x: Int => x case _ => 0 }) - val rddOrig = sql("SELECT * FROM src") + val rddOrig = hql("SELECT * FROM src") .collect() .sortBy(_.getInt(0)) compareRDDs(rddOrig, rddCopy, "src (Hive)", Seq("key:Int", "value:String")) @@ -108,22 +108,22 @@ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAft test("Appending to Parquet table") { createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType)) .registerAsTable("tmpnew") - sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() - sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() - sql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() - val rddCopies = sql("SELECT * FROM tmpnew").collect() - val rddOrig = sql("SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmpnew SELECT * FROM src").collect() + val rddCopies = hql("SELECT * FROM tmpnew").collect() + val rddOrig = hql("SELECT * FROM src").collect() assert(rddCopies.size === 3 * rddOrig.size, "number of copied rows via INSERT INTO did not match correct number") } test("Appending to and then overwriting Parquet table") { createParquetFile(dirname.getAbsolutePath, ("key", IntegerType), ("value", StringType)) .registerAsTable("tmp") - sql("INSERT INTO TABLE tmp SELECT * FROM src").collect() - sql("INSERT INTO TABLE tmp SELECT * FROM src").collect() - sql("INSERT OVERWRITE TABLE tmp SELECT * FROM src").collect() - val rddCopies = sql("SELECT * FROM tmp").collect() - val rddOrig = sql("SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmp SELECT * FROM src").collect() + hql("INSERT INTO TABLE tmp SELECT * FROM src").collect() + hql("INSERT OVERWRITE TABLE tmp SELECT * FROM src").collect() + val rddCopies = hql("SELECT * FROM tmp").collect() + val rddOrig = hql("SELECT * FROM src").collect() assert(rddCopies.size === rddOrig.size, "INSERT OVERWRITE did not actually overwrite") } From 0acc7a02b4323f4e0b7736bc1999bdcedab41f39 Mon Sep 17 00:00:00 2001 From: Prabeesh K Date: Fri, 4 Apr 2014 21:32:00 -0700 Subject: [PATCH 205/397] small fix ( proogram -> program ) Author: Prabeesh K Closes #331 from prabeesh/patch-3 and squashes the following commits: 9399eb5 [Prabeesh K] small fix(proogram -> program) --- docs/quick-start.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/quick-start.md b/docs/quick-start.md index 13df6beea16e8..60e8b1ba0eb46 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -124,7 +124,7 @@ object SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the proogram. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. +This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on: From 7c18428fac1403eb9c69b61890453964b255c432 Mon Sep 17 00:00:00 2001 From: Prashant Sharma Date: Fri, 4 Apr 2014 22:49:19 -0700 Subject: [PATCH 206/397] HOTFIX for broken CI, by SPARK-1336 Learnt about `set -o pipefail` is very useful. Author: Prashant Sharma Author: Prashant Sharma Closes #321 from ScrapCodes/hf-SPARK-1336 and squashes the following commits: 9d22bc2 [Prashant Sharma] added comment why echo -e q exists. f865951 [Prashant Sharma] made error to match with word boundry so errors does not match. This is there to make sure build fails if provided SparkBuild has compile errors. 7fffdf2 [Prashant Sharma] Removed a stray line. 97379d8 [Prashant Sharma] HOTFIX for broken CI, by SPARK-1336 --- .rat-excludes | 1 + dev/run-tests | 10 +++++----- dev/scalastyle | 4 ++-- 3 files changed, 8 insertions(+), 7 deletions(-) diff --git a/.rat-excludes b/.rat-excludes index 17cf6d0ed1cf3..85bfad60fcadc 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -39,3 +39,4 @@ work .*\.q golden test.out/* +.*iml diff --git a/dev/run-tests b/dev/run-tests index fff949e04fcd7..6ad674a2ba127 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -26,13 +26,12 @@ rm -rf ./work # Fail fast set -e - +set -o pipefail if test -x "$JAVA_HOME/bin/java"; then declare java_cmd="$JAVA_HOME/bin/java" else declare java_cmd=java fi - JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') [ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run because JDK version is < 1.8." @@ -49,7 +48,9 @@ dev/scalastyle echo "=========================================================================" echo "Running Spark unit tests" echo "=========================================================================" -sbt/sbt assembly test +# echo "q" is needed because sbt on encountering a build file with failure (either resolution or compilation) +# prompts the user for input either q, r, etc to quit or retry. This echo is there to make it not block. +echo -e "q\n" | sbt/sbt assembly test | grep -v -e "info.*Resolving" -e "warn.*Merging" -e "info.*Including" echo "=========================================================================" echo "Running PySpark tests" @@ -63,5 +64,4 @@ echo "=========================================================================" echo "Detecting binary incompatibilites with MiMa" echo "=========================================================================" ./bin/spark-class org.apache.spark.tools.GenerateMIMAIgnore -sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" - +echo -e "q\n" | sbt/sbt mima-report-binary-issues | grep -v -e "info.*Resolving" diff --git a/dev/scalastyle b/dev/scalastyle index 5a18f4d672825..19955b9aaaad3 100755 --- a/dev/scalastyle +++ b/dev/scalastyle @@ -17,8 +17,8 @@ # limitations under the License. # -sbt/sbt clean scalastyle > scalastyle.txt -ERRORS=$(cat scalastyle.txt | grep -e "error file") +echo -e "q\n" | sbt/sbt clean scalastyle > scalastyle.txt +ERRORS=$(cat scalastyle.txt | grep -e "\") if test ! -z "$ERRORS"; then echo -e "Scalastyle checks failed at following occurrences:\n$ERRORS" exit 1 From 2d0150c1a2688296346fa279b1f8d14edac935eb Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Sat, 5 Apr 2014 15:17:50 -0700 Subject: [PATCH 207/397] Remove the getStageInfo() method from SparkContext. This method exposes the Stage objects, which are private to Spark and should not be exposed to the user. This method was added in https://github.com/apache/spark/commit/01d77f329f5878b7c8672bbdc1859f3ca95d759d; ccing @squito here in case there's a good reason to keep this! Author: Kay Ousterhout Closes #308 from kayousterhout/remove_public_method and squashes the following commits: 2e2f009 [Kay Ousterhout] Remove the getStageInfo() method from SparkContext. --- core/src/main/scala/org/apache/spark/SparkContext.scala | 4 ---- .../main/scala/org/apache/spark/scheduler/DAGScheduler.scala | 2 +- 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index fcf16ce1b278e..8382dd44f3484 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -731,10 +731,6 @@ class SparkContext( */ def getPersistentRDDs: Map[Int, RDD[_]] = persistentRdds.toMap - def getStageInfo: Map[Stage, StageInfo] = { - dagScheduler.stageToInfos - } - /** * Return information about blocks stored in all of the slaves */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index ef3d24d746829..442a95bb2c44b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -86,7 +86,7 @@ class DAGScheduler( private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] - private[spark] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] + private[scheduler] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] From 6e88583aef7d8caf59d53c9fcb659a62d2cd6051 Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Sat, 5 Apr 2014 15:23:37 -0700 Subject: [PATCH 208/397] [SPARK-1371] fix computePreferredLocations signature to not depend on underlying implementation Change to Map and Set - not mutable HashMap and HashSet Author: Mridul Muralidharan Closes #302 from mridulm/master and squashes the following commits: df747af [Mridul Muralidharan] Address review comments 17e2907 [Mridul Muralidharan] fix computePreferredLocations signature to not depend on underlying implementation --- .../scala/org/apache/spark/scheduler/InputFormatInfo.scala | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index 5555585c8b4cd..b3f2cb346f7da 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -164,8 +164,7 @@ object InputFormatInfo { PS: I know the wording here is weird, hopefully it makes some sense ! */ - def computePreferredLocations(formats: Seq[InputFormatInfo]): HashMap[String, HashSet[SplitInfo]] - = { + def computePreferredLocations(formats: Seq[InputFormatInfo]): Map[String, Set[SplitInfo]] = { val nodeToSplit = new HashMap[String, HashSet[SplitInfo]] for (inputSplit <- formats) { @@ -178,6 +177,6 @@ object InputFormatInfo { } } - nodeToSplit + nodeToSplit.mapValues(_.toSet).toMap } } From 890d63bd4e16296ac70e151b3754727ea42b583c Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sat, 5 Apr 2014 19:08:24 -0700 Subject: [PATCH 209/397] Fix for PR #195 for Java 6 Use Java 6's recommended equivalent of Java 7's Logger.getGlobal() to retain Java 6 compatibility. See PR #195 Author: Sean Owen Closes #334 from srowen/FixPR195ForJava6 and squashes the following commits: f92fbd3 [Sean Owen] Use Java 6's recommended equivalent of Java 7's Logger.getGlobal() to retain Java 6 compatibility --- .../scala/org/apache/spark/sql/parquet/ParquetRelation.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) 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 114bfbb719ee9..505ad0a2c77c1 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 @@ -103,7 +103,7 @@ private[sql] object ParquetRelation { SLF4JBridgeHandler.install() for(name <- loggerNames) { val logger = Logger.getLogger(name) - logger.setParent(Logger.getGlobal) + logger.setParent(Logger.getLogger(Logger.GLOBAL_LOGGER_NAME)) logger.setUseParentHandlers(true) } } From 0b855167818b9afd2d2aa9f617b9861d77b2425d Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Sat, 5 Apr 2014 20:52:05 -0700 Subject: [PATCH 210/397] SPARK-1421. Make MLlib work on Python 2.6 The reason it wasn't working was passing a bytearray to stream.write(), which is not supported in Python 2.6 but is in 2.7. (This array came from NumPy when we converted data to send it over to Java). Now we just convert those bytearrays to strings of bytes, which preserves nonprintable characters as well. Author: Matei Zaharia Closes #335 from mateiz/mllib-python-2.6 and squashes the following commits: f26c59f [Matei Zaharia] Update docs to no longer say we need Python 2.7 a84d6af [Matei Zaharia] SPARK-1421. Make MLlib work on Python 2.6 --- docs/mllib-guide.md | 3 +-- docs/python-programming-guide.md | 2 +- python/pyspark/mllib/__init__.py | 6 +----- python/pyspark/serializers.py | 11 ++++++++++- 4 files changed, 13 insertions(+), 9 deletions(-) diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 203d235bf9663..a5e0cc50809cf 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -38,6 +38,5 @@ depends on native Fortran routines. You may need to install the if it is not already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries automatically. -To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.7 or newer -and Python 2.7. +To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.7 or newer. diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index cbe7d820b455e..c2e5327324898 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -152,7 +152,7 @@ Many of the methods also contain [doctests](http://docs.python.org/2/library/doc # Libraries [MLlib](mllib-guide.html) is also available in PySpark. To use it, you'll need -[NumPy](http://www.numpy.org) version 1.7 or newer, and Python 2.7. The [MLlib guide](mllib-guide.html) contains +[NumPy](http://www.numpy.org) version 1.7 or newer. The [MLlib guide](mllib-guide.html) contains some example applications. # Where to Go from Here diff --git a/python/pyspark/mllib/__init__.py b/python/pyspark/mllib/__init__.py index b420d7a7f23ba..538ff26ce7c33 100644 --- a/python/pyspark/mllib/__init__.py +++ b/python/pyspark/mllib/__init__.py @@ -19,11 +19,7 @@ Python bindings for MLlib. """ -# MLlib currently needs Python 2.7+ and NumPy 1.7+, so complain if lower - -import sys -if sys.version_info[0:2] < (2, 7): - raise Exception("MLlib requires Python 2.7+") +# MLlib currently needs and NumPy 1.7+, so complain if lower import numpy if numpy.version.version < '1.7': diff --git a/python/pyspark/serializers.py b/python/pyspark/serializers.py index 4d802924df4a1..b253807974a2e 100644 --- a/python/pyspark/serializers.py +++ b/python/pyspark/serializers.py @@ -64,6 +64,7 @@ from itertools import chain, izip, product import marshal import struct +import sys from pyspark import cloudpickle @@ -113,6 +114,11 @@ class FramedSerializer(Serializer): where C{length} is a 32-bit integer and data is C{length} bytes. """ + def __init__(self): + # On Python 2.6, we can't write bytearrays to streams, so we need to convert them + # to strings first. Check if the version number is that old. + self._only_write_strings = sys.version_info[0:2] <= (2, 6) + def dump_stream(self, iterator, stream): for obj in iterator: self._write_with_length(obj, stream) @@ -127,7 +133,10 @@ def load_stream(self, stream): def _write_with_length(self, obj, stream): serialized = self.dumps(obj) write_int(len(serialized), stream) - stream.write(serialized) + if self._only_write_strings: + stream.write(str(serialized)) + else: + stream.write(serialized) def _read_with_length(self, stream): length = read_int(stream) From 7012ffafad8fa876aa8bcb0b848445eec6734ef1 Mon Sep 17 00:00:00 2001 From: witgo Date: Sun, 6 Apr 2014 16:03:06 -0700 Subject: [PATCH 211/397] Fix SPARK-1420 The maven build error for Spark Catalyst Author: witgo Closes #333 from witgo/SPARK-1420 and squashes the following commits: 902519e [witgo] add dependency scala-reflect to catalyst --- sql/catalyst/pom.xml | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/sql/catalyst/pom.xml b/sql/catalyst/pom.xml index 0edce55a93338..9d5c6a857bb00 100644 --- a/sql/catalyst/pom.xml +++ b/sql/catalyst/pom.xml @@ -44,6 +44,10 @@ + + org.scala-lang + scala-reflect + org.apache.spark spark-core_${scala.binary.version} From e258e5040fa1905a04efcb7b3ca4a6d33e18fa61 Mon Sep 17 00:00:00 2001 From: Egor Pakhomov Date: Sun, 6 Apr 2014 16:41:23 -0700 Subject: [PATCH 212/397] [SPARK-1259] Make RDD locally iterable Author: Egor Pakhomov Closes #156 from epahomov/SPARK-1259 and squashes the following commits: 8ec8f24 [Egor Pakhomov] Make to local iterator shorter 34aa300 [Egor Pakhomov] Fix toLocalIterator docs 08363ef [Egor Pakhomov] SPARK-1259 from toLocallyIterable to toLocalIterator 6a994eb [Egor Pakhomov] SPARK-1259 Make RDD locally iterable 8be3dcf [Egor Pakhomov] SPARK-1259 Make RDD locally iterable 33ecb17 [Egor Pakhomov] SPARK-1259 Make RDD locally iterable --- .../org/apache/spark/api/java/JavaRDDLike.scala | 14 +++++++++++++- core/src/main/scala/org/apache/spark/rdd/RDD.scala | 12 ++++++++++++ .../test/java/org/apache/spark/JavaAPISuite.java | 9 +++++++++ .../test/scala/org/apache/spark/rdd/RDDSuite.scala | 1 + 4 files changed, 35 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index e03b8e78d5f52..6e8ec8e0c7629 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -17,7 +17,8 @@ package org.apache.spark.api.java -import java.util.{Comparator, List => JList} +import java.util.{Comparator, Iterator => JIterator, List => JList} +import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -280,6 +281,17 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } + /** + * Return an iterator that contains all of the elements in this RDD. + * + * The iterator will consume as much memory as the largest partition in this RDD. + */ + def toLocalIterator(): JIterator[T] = { + import scala.collection.JavaConversions._ + rdd.toLocalIterator + } + + /** * Return an array that contains all of the elements in this RDD. * @deprecated As of Spark 1.0.0, toArray() is deprecated, use {@link #collect()} instead diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 08c42c5ee87b6..c43823bd769b7 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -661,6 +661,18 @@ abstract class RDD[T: ClassTag]( Array.concat(results: _*) } + /** + * Return an iterator that contains all of the elements in this RDD. + * + * The iterator will consume as much memory as the largest partition in this RDD. + */ + def toLocalIterator: Iterator[T] = { + def collectPartition(p: Int): Array[T] = { + sc.runJob(this, (iter: Iterator[T]) => iter.toArray, Seq(p), allowLocal = false).head + } + (0 until partitions.length).iterator.flatMap(i => collectPartition(i)) + } + /** * Return an array that contains all of the elements in this RDD. */ diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 2372f2d9924a1..762405be2a8f9 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -22,6 +22,7 @@ import scala.Tuple2; +import com.google.common.collect.Lists; import com.google.common.base.Optional; import com.google.common.base.Charsets; import com.google.common.io.Files; @@ -179,6 +180,14 @@ public void call(String s) { Assert.assertEquals(2, foreachCalls); } + @Test + public void toLocalIterator() { + List correct = Arrays.asList(1, 2, 3, 4); + JavaRDD rdd = sc.parallelize(correct); + List result = Lists.newArrayList(rdd.toLocalIterator()); + Assert.assertTrue(correct.equals(result)); + } + @SuppressWarnings("unchecked") @Test public void lookup() { diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index d6b5fdc7984b4..25973348a7837 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -33,6 +33,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("basic operations") { val nums = sc.makeRDD(Array(1, 2, 3, 4), 2) assert(nums.collect().toList === List(1, 2, 3, 4)) + assert(nums.toLocalIterator.toList === List(1, 2, 3, 4)) val dups = sc.makeRDD(Array(1, 1, 2, 2, 3, 3, 4, 4), 2) assert(dups.distinct().count() === 4) assert(dups.distinct.count === 4) // Can distinct and count be called without parentheses? From 856c50f59bffbf76ad495eaab837febaf65cf02d Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Sun, 6 Apr 2014 17:40:37 -0700 Subject: [PATCH 213/397] SPARK-1387. Update build plugins, avoid plugin version warning, centralize versions Another handful of small build changes to organize and standardize a bit, and avoid warnings: - Update Maven plugin versions for good measure - Since plugins need maven 3.0.4 already, require it explicitly (<3.0.4 had some bugs anyway) - Use variables to define versions across dependencies where they should move in lock step - ... and make this consistent between Maven/SBT OK, I also updated the JIRA URL while I was at it here. Author: Sean Owen Closes #291 from srowen/SPARK-1387 and squashes the following commits: 461eca1 [Sean Owen] Couldn't resist also updating JIRA location to new one c2d5cc5 [Sean Owen] Update plugins and Maven version; use variables consistently across Maven/SBT to define dependency versions that should stay in step. --- assembly/pom.xml | 2 +- core/pom.xml | 2 - dev/audit-release/maven_app_core/pom.xml | 2 +- docs/building-with-maven.md | 2 +- examples/pom.xml | 2 +- graphx/pom.xml | 2 +- mllib/pom.xml | 2 +- pom.xml | 43 +++++++++++---------- project/SparkBuild.scala | 49 ++++++++++++++---------- streaming/pom.xml | 1 - 10 files changed, 57 insertions(+), 50 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index b5e752c6cd1f6..255107a2c47cb 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -208,7 +208,7 @@ org.codehaus.mojo buildnumber-maven-plugin - 1.1 + 1.2 validate diff --git a/core/pom.xml b/core/pom.xml index 66f9fc4961b03..1f808380817c9 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -117,12 +117,10 @@ com.twitter chill_${scala.binary.version} - 0.3.1 com.twitter chill-java - 0.3.1 commons-net diff --git a/dev/audit-release/maven_app_core/pom.xml b/dev/audit-release/maven_app_core/pom.xml index 0b837c01751fe..76a381f8e17e0 100644 --- a/dev/audit-release/maven_app_core/pom.xml +++ b/dev/audit-release/maven_app_core/pom.xml @@ -49,7 +49,7 @@ maven-compiler-plugin - 2.3.2 + 3.1 diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 730a6e7932564..9cebaf12283fc 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -6,7 +6,7 @@ title: Building Spark with Maven * This will become a table of contents (this text will be scraped). {:toc} -Building Spark using Maven Requires Maven 3 (the build process is tested with Maven 3.0.4) and Java 1.6 or newer. +Building Spark using Maven requires Maven 3.0.4 or newer and Java 1.6 or newer. ## Setting up Maven's Memory Usage ## diff --git a/examples/pom.xml b/examples/pom.xml index a5569ff5e71f3..0b6212b5d1549 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -110,7 +110,7 @@ org.apache.hbase hbase - 0.94.6 + ${hbase.version} asm diff --git a/graphx/pom.xml b/graphx/pom.xml index 5a5022916d234..b4c67ddcd8ca9 100644 --- a/graphx/pom.xml +++ b/graphx/pom.xml @@ -54,7 +54,7 @@ org.jblas jblas - 1.2.3 + ${jblas.version} org.eclipse.jetty diff --git a/mllib/pom.xml b/mllib/pom.xml index fec1cc94b2642..e7ce00efc4af6 100644 --- a/mllib/pom.xml +++ b/mllib/pom.xml @@ -58,7 +58,7 @@ org.jblas jblas - 1.2.3 + ${jblas.version} org.scalanlp diff --git a/pom.xml b/pom.xml index 01341d21b7f23..1426e0e00214c 100644 --- a/pom.xml +++ b/pom.xml @@ -54,11 +54,11 @@ JIRA - https://spark-project.atlassian.net/browse/SPARK + https://issues.apache.org/jira/browse/SPARK - 3.0.0 + 3.0.4 @@ -123,6 +123,10 @@ 0.94.6 0.12.0 1.3.2 + 1.2.3 + 8.1.14.v20131031 + 0.3.1 + 3.0.0 64m 512m @@ -192,22 +196,22 @@ org.eclipse.jetty jetty-util - 8.1.14.v20131031 + ${jetty.version} org.eclipse.jetty jetty-security - 8.1.14.v20131031 + ${jetty.version} org.eclipse.jetty jetty-plus - 8.1.14.v20131031 + ${jetty.version} org.eclipse.jetty jetty-server - 8.1.14.v20131031 + ${jetty.version} com.google.guava @@ -273,7 +277,7 @@ com.twitter chill_${scala.binary.version} - 0.3.1 + ${chill.version} org.ow2.asm @@ -288,7 +292,7 @@ com.twitter chill-java - 0.3.1 + ${chill.version} org.ow2.asm @@ -392,27 +396,27 @@ com.codahale.metrics metrics-core - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-jvm - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-json - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-ganglia - 3.0.0 + ${codahale.metrics.version} com.codahale.metrics metrics-graphite - 3.0.0 + ${codahale.metrics.version} org.scala-lang @@ -585,7 +589,7 @@ org.apache.maven.plugins maven-enforcer-plugin - 1.1.1 + 1.3.1 enforce-versions @@ -595,7 +599,7 @@ - 3.0.0 + 3.0.4 ${java.version} @@ -608,12 +612,12 @@ org.codehaus.mojo build-helper-maven-plugin - 1.7 + 1.8 net.alchim31.maven scala-maven-plugin - 3.1.5 + 3.1.6 scala-compile-first @@ -674,7 +678,7 @@ org.apache.maven.plugins maven-surefire-plugin - 2.12.4 + 2.17 true @@ -713,7 +717,7 @@ org.apache.maven.plugins maven-shade-plugin - 2.0 + 2.2 org.apache.maven.plugins @@ -810,7 +814,6 @@ org.apache.maven.plugins maven-jar-plugin - 2.4 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 843a874fbfdb0..3489b43d43f0d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -248,10 +248,10 @@ object SparkBuild extends Build { libraryDependencies ++= Seq( "io.netty" % "netty-all" % "4.0.17.Final", - "org.eclipse.jetty" % "jetty-server" % "8.1.14.v20131031", - "org.eclipse.jetty" % "jetty-util" % "8.1.14.v20131031", - "org.eclipse.jetty" % "jetty-plus" % "8.1.14.v20131031", - "org.eclipse.jetty" % "jetty-security" % "8.1.14.v20131031", + "org.eclipse.jetty" % "jetty-server" % jettyVersion, + "org.eclipse.jetty" % "jetty-util" % jettyVersion, + "org.eclipse.jetty" % "jetty-plus" % jettyVersion, + "org.eclipse.jetty" % "jetty-security" % jettyVersion, /** Workaround for SPARK-959. Dependency used by org.eclipse.jetty. Fixed in ivy 2.3.0. */ "org.eclipse.jetty.orbit" % "javax.servlet" % "3.0.0.v201112011016" artifacts Artifact("javax.servlet", "jar", "jar"), "org.scalatest" %% "scalatest" % "1.9.1" % "test", @@ -276,6 +276,13 @@ object SparkBuild extends Build { publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings + val akkaVersion = "2.2.3-shaded-protobuf" + val chillVersion = "0.3.1" + val codahaleMetricsVersion = "3.0.0" + val jblasVersion = "1.2.3" + val jettyVersion = "8.1.14.v20131031" + val hiveVersion = "0.12.0" + val parquetVersion = "1.3.2" val slf4jVersion = "1.7.5" val excludeNetty = ExclusionRule(organization = "org.jboss.netty") @@ -309,9 +316,9 @@ object SparkBuild extends Build { "commons-daemon" % "commons-daemon" % "1.0.10", // workaround for bug HADOOP-9407 "com.ning" % "compress-lzf" % "1.0.0", "org.xerial.snappy" % "snappy-java" % "1.0.5", - "org.spark-project.akka" %% "akka-remote" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-slf4j" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty), - "org.spark-project.akka" %% "akka-testkit" % "2.2.3-shaded-protobuf" % "test", + "org.spark-project.akka" %% "akka-remote" % akkaVersion excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-slf4j" % akkaVersion excludeAll(excludeNetty), + "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", @@ -321,12 +328,12 @@ object SparkBuild extends Build { "org.apache.derby" % "derby" % "10.4.2.0" % "test", "org.apache.hadoop" % hadoopClient % hadoopVersion excludeAll(excludeNetty, excludeAsm, excludeCommonsLogging, excludeSLF4J, excludeOldAsm), "org.apache.curator" % "curator-recipes" % "2.4.0" excludeAll(excludeNetty), - "com.codahale.metrics" % "metrics-core" % "3.0.0", - "com.codahale.metrics" % "metrics-jvm" % "3.0.0", - "com.codahale.metrics" % "metrics-json" % "3.0.0", - "com.codahale.metrics" % "metrics-graphite" % "3.0.0", - "com.twitter" %% "chill" % "0.3.1" excludeAll(excludeAsm), - "com.twitter" % "chill-java" % "0.3.1" excludeAll(excludeAsm), + "com.codahale.metrics" % "metrics-core" % codahaleMetricsVersion, + "com.codahale.metrics" % "metrics-jvm" % codahaleMetricsVersion, + "com.codahale.metrics" % "metrics-json" % codahaleMetricsVersion, + "com.codahale.metrics" % "metrics-graphite" % codahaleMetricsVersion, + "com.twitter" %% "chill" % chillVersion excludeAll(excludeAsm), + "com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm), "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" ), @@ -370,7 +377,7 @@ object SparkBuild extends Build { name := "spark-graphx", previousArtifact := sparkPreviousArtifact("spark-graphx"), libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3" + "org.jblas" % "jblas" % jblasVersion ) ) @@ -383,7 +390,7 @@ object SparkBuild extends Build { name := "spark-mllib", previousArtifact := sparkPreviousArtifact("spark-mllib"), libraryDependencies ++= Seq( - "org.jblas" % "jblas" % "1.2.3", + "org.jblas" % "jblas" % jblasVersion, "org.scalanlp" %% "breeze" % "0.7" ) ) @@ -403,8 +410,8 @@ object SparkBuild extends Build { 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" % parquetVersion, + "com.twitter" % "parquet-hadoop" % parquetVersion ) ) @@ -416,9 +423,9 @@ object SparkBuild extends Build { 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", - "org.apache.hive" % "hive-exec" % "0.12.0", - "org.apache.hive" % "hive-serde" % "0.12.0" + "org.apache.hive" % "hive-metastore" % hiveVersion, + "org.apache.hive" % "hive-exec" % hiveVersion, + "org.apache.hive" % "hive-serde" % hiveVersion ), // Multiple queries rely on the TestHive singleton. See comments there for more details. parallelExecution in Test := false, @@ -549,7 +556,7 @@ object SparkBuild extends Build { name := "spark-streaming-zeromq", previousArtifact := sparkPreviousArtifact("spark-streaming-zeromq"), libraryDependencies ++= Seq( - "org.spark-project.akka" %% "akka-zeromq" % "2.2.3-shaded-protobuf" excludeAll(excludeNetty) + "org.spark-project.akka" %% "akka-zeromq" % akkaVersion excludeAll(excludeNetty) ) ) diff --git a/streaming/pom.xml b/streaming/pom.xml index 1953cc6883378..93b1c5a37aff9 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -96,7 +96,6 @@ org.apache.maven.plugins maven-jar-plugin - 2.2 From 7ce52c4a7a07b0db5e7c1312b1920efb1165ce6a Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 6 Apr 2014 17:43:44 -0700 Subject: [PATCH 214/397] SPARK-1349: spark-shell gets its own command history Currently, spark-shell shares its command history with scala repl. This fix is simply a modification of the default FileBackedHistory file setting: https://github.com/scala/scala/blob/master/src/repl/scala/tools/nsc/interpreter/session/FileBackedHistory.scala#L77 Author: Aaron Davidson Closes #267 from aarondav/repl and squashes the following commits: f9c62d2 [Aaron Davidson] SPARK-1349: spark-shell gets its own command history separate from scala repl --- .../org/apache/spark/repl/SparkJLineReader.scala | 12 +++++++++++- 1 file changed, 11 insertions(+), 1 deletion(-) diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala index 946e71039088d..0db26c3407dff 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkJLineReader.scala @@ -7,8 +7,10 @@ package org.apache.spark.repl +import scala.reflect.io.{Path, File} import scala.tools.nsc._ import scala.tools.nsc.interpreter._ +import scala.tools.nsc.interpreter.session.JLineHistory.JLineFileHistory import scala.tools.jline.console.ConsoleReader import scala.tools.jline.console.completer._ @@ -25,7 +27,7 @@ class SparkJLineReader(_completion: => Completion) extends InteractiveReader { val consoleReader = new JLineConsoleReader() lazy val completion = _completion - lazy val history: JLineHistory = JLineHistory() + lazy val history: JLineHistory = new SparkJLineHistory private def term = consoleReader.getTerminal() def reset() = term.reset() @@ -78,3 +80,11 @@ class SparkJLineReader(_completion: => Completion) extends InteractiveReader { def readOneLine(prompt: String) = consoleReader readLine prompt def readOneKey(prompt: String) = consoleReader readOneKey prompt } + +/** Changes the default history file to not collide with the scala repl's. */ +class SparkJLineHistory extends JLineFileHistory { + import Properties.userHome + + def defaultFileName = ".spark_history" + override protected lazy val historyFile = File(Path(userHome) / defaultFileName) +} From 4106558435889261243d186f5f0b51c5f9e98d56 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Sun, 6 Apr 2014 17:48:41 -0700 Subject: [PATCH 215/397] SPARK-1314: Use SPARK_HIVE to determine if we include Hive in packaging Previously, we based our decision regarding including datanucleus jars based on the existence of a spark-hive-assembly jar, which was incidentally built whenever "sbt assembly" is run. This means that a typical and previously supported pathway would start using hive jars. This patch has the following features/bug fixes: - Use of SPARK_HIVE (default false) to determine if we should include Hive in the assembly jar. - Analagous feature in Maven with -Phive (previously, there was no support for adding Hive to any of our jars produced by Maven) - assemble-deps fixed since we no longer use a different ASSEMBLY_DIR - avoid adding log message in compute-classpath.sh to the classpath :) Still TODO before mergeable: - We need to download the datanucleus jars outside of sbt. Perhaps we can have spark-class download them if SPARK_HIVE is set similar to how sbt downloads itself. - Spark SQL documentation updates. Author: Aaron Davidson Closes #237 from aarondav/master and squashes the following commits: 5dc4329 [Aaron Davidson] Typo fixes dd4f298 [Aaron Davidson] Doc update dd1a365 [Aaron Davidson] Eliminate need for SPARK_HIVE at runtime by d/ling datanucleus from Maven a9269b5 [Aaron Davidson] [WIP] Use SPARK_HIVE to determine if we include Hive in packaging --- assembly/pom.xml | 10 ++++++++ bin/compute-classpath.sh | 35 +++++++++++++++------------- bin/spark-class | 2 -- dev/create-release/create-release.sh | 4 ++-- docs/sql-programming-guide.md | 4 ++-- pom.xml | 7 +++++- project/SparkBuild.scala | 25 +++++++++++++------- sql/hive/pom.xml | 28 ++++++++++++++++++++++ 8 files changed, 83 insertions(+), 32 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 255107a2c47cb..923bf47f7076a 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -163,6 +163,16 @@ + + hive + + + org.apache.spark + spark-hive_${scala.binary.version} + ${project.version} + + + spark-ganglia-lgpl diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index bef42df71ce01..be37102dc069a 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -30,21 +30,7 @@ FWDIR="$(cd `dirname $0`/..; pwd)" # Build up classpath CLASSPATH="$SPARK_CLASSPATH:$FWDIR/conf" -# 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 entirely and deploy only the needed packages in -# the future. -if [ -f "$FWDIR"/sql/hive/target/scala-$SCALA_VERSION/spark-hive-assembly-*.jar ]; then - - # 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 +ASSEMBLY_DIR="$FWDIR/assembly/target/scala-$SCALA_VERSION" # First check if we have a dependencies jar. If so, include binary classes with the deps jar if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then @@ -59,7 +45,7 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then CLASSPATH="$CLASSPATH:$FWDIR/sql/core/target/scala-$SCALA_VERSION/classes" CLASSPATH="$CLASSPATH:$FWDIR/sql/hive/target/scala-$SCALA_VERSION/classes" - DEPS_ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/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 @@ -71,6 +57,23 @@ else CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi +# When Hive support is needed, Datanucleus jars must be included on the classpath. +# Datanucleus jars do not work if only included in the uber jar as plugin.xml metadata is lost. +# Both sbt and maven will populate "lib_managed/jars/" with the datanucleus jars when Spark is +# built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark +# assembly is built for Hive, before actually populating the CLASSPATH with the jars. +# Note that this check order is faster (by up to half a second) in the case where Hive is not used. +num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ | grep "datanucleus-.*\\.jar" | wc -l) +if [ $num_datanucleus_jars -gt 0 ]; then + AN_ASSEMBLY_JAR=${ASSEMBLY_JAR:-$DEPS_ASSEMBLY_JAR} + num_hive_files=$(jar tvf "$AN_ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null | wc -l) + if [ $num_hive_files -gt 0 ]; then + echo "Spark assembly has been built with Hive, including Datanucleus jars on classpath" 1>&2 + DATANUCLEUSJARS=$(echo "$FWDIR/lib_managed/jars"/datanucleus-*.jar | tr " " :) + CLASSPATH=$CLASSPATH:$DATANUCLEUSJARS + fi +fi + # Add test classes if we're running from SBT or Maven with SPARK_TESTING set to 1 if [[ $SPARK_TESTING == 1 ]]; then CLASSPATH="$CLASSPATH:$FWDIR/core/target/scala-$SCALA_VERSION/test-classes" diff --git a/bin/spark-class b/bin/spark-class index 0dcf0e156cb52..76fde3e448891 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -154,5 +154,3 @@ if [ "$SPARK_PRINT_LAUNCH_COMMAND" == "1" ]; then fi exec "$RUNNER" -cp "$CLASSPATH" $JAVA_OPTS "$@" - - diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index 995106f111443..bf1c5d7953bd2 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -49,14 +49,14 @@ mvn -DskipTests \ -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dusername=$GIT_USERNAME -Dpassword=$GIT_PASSWORD \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Pspark-ganglia-lgpl \ + -Pyarn -Phive -Pspark-ganglia-lgpl\ -Dtag=$GIT_TAG -DautoVersionSubmodules=true \ --batch-mode release:prepare mvn -DskipTests \ -Darguments="-DskipTests=true -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -Dgpg.passphrase=${GPG_PASSPHRASE}" \ -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 \ - -Pyarn -Pspark-ganglia-lgpl\ + -Pyarn -Phive -Pspark-ganglia-lgpl\ release:perform rm -rf spark diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index f849716f7a48f..a59393e1424de 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -264,8 +264,8 @@ evaluated by the SQL execution engine. A full list of the functions supported c Spark SQL also supports reading and writing data stored in [Apache Hive](http://hive.apache.org/). However, since Hive has a large number of dependencies, it is not included in the default Spark assembly. -In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`'. This command builds a new assembly -jar that includes Hive. Note that this Hive assembly jar must also be present +In order to use Hive you must first run '`SPARK_HIVE=true sbt/sbt assembly/assembly`' (or use `-Phive` for maven). +This command builds a new assembly jar that includes Hive. Note that this Hive assembly jar must also be present on all of the worker nodes, as they will need access to the Hive serialization and deserialization libraries (SerDes) in order to acccess data stored in Hive. diff --git a/pom.xml b/pom.xml index 1426e0e00214c..c03bb35c99442 100644 --- a/pom.xml +++ b/pom.xml @@ -377,7 +377,6 @@ org.apache.derby derby 10.4.2.0 - test net.liftweb @@ -580,6 +579,12 @@ + + + org.codehaus.jackson + jackson-mapper-asl + 1.8.8 + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 3489b43d43f0d..d1e4b8b964b88 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -43,6 +43,8 @@ object SparkBuild extends Build { val DEFAULT_YARN = false + val DEFAULT_HIVE = false + // HBase version; set as appropriate. val HBASE_VERSION = "0.94.6" @@ -67,15 +69,17 @@ object SparkBuild extends Build { lazy val sql = Project("sql", file("sql/core"), settings = sqlCoreSettings) dependsOn(core, catalyst) - // 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 hive = Project("hive", file("sql/hive"), settings = hiveSettings) dependsOn(sql) + + lazy val maybeHive: Seq[ClasspathDependency] = if (isHiveEnabled) Seq(hive) else Seq() + lazy val maybeHiveRef: Seq[ProjectReference] = if (isHiveEnabled) Seq(hive) else Seq() 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, streaming, repl, sql) dependsOn(maybeYarn: _*) dependsOn(maybeGanglia: _*) + .dependsOn(core, graphx, bagel, mllib, streaming, repl, sql) dependsOn(maybeYarn: _*) dependsOn(maybeHive: _*) dependsOn(maybeGanglia: _*) lazy val assembleDeps = TaskKey[Unit]("assemble-deps", "Build assembly of dependencies and packages Spark projects") @@ -101,6 +105,11 @@ object SparkBuild extends Build { lazy val hadoopClient = if (hadoopVersion.startsWith("0.20.") || hadoopVersion == "1.0.0") "hadoop-core" else "hadoop-client" val maybeAvro = if (hadoopVersion.startsWith("0.23.") && isYarnEnabled) Seq("org.apache.avro" % "avro" % "1.7.4") else Seq() + lazy val isHiveEnabled = Properties.envOrNone("SPARK_HIVE") match { + case None => DEFAULT_HIVE + case Some(v) => v.toBoolean + } + // Include Ganglia integration if the user has enabled Ganglia // This is isolated from the normal build due to LGPL-licensed code in the library lazy val isGangliaEnabled = Properties.envOrNone("SPARK_GANGLIA_LGPL").isDefined @@ -141,13 +150,13 @@ 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, hive) dependsOn(allExternal: _*) + .dependsOn(core, mllib, graphx, bagel, streaming, hive) dependsOn(allExternal: _*) // 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 packageProjects = Seq[ProjectReference](core, repl, bagel, streaming, mllib, graphx, catalyst, sql) ++ maybeYarnRef ++ maybeHiveRef ++ maybeGangliaRef lazy val allProjects = packageProjects ++ allExternalRefs ++ - Seq[ProjectReference](examples, tools, assemblyProj, hive) ++ maybeJava8Tests + Seq[ProjectReference](examples, tools, assemblyProj) ++ maybeJava8Tests def sharedSettings = Defaults.defaultSettings ++ MimaBuild.mimaSettings(file(sparkHome)) ++ Seq( organization := "org.apache.spark", @@ -417,10 +426,8 @@ object SparkBuild extends Build { // Since we don't include hive in the main assembly this project also acts as an alternative // assembly jar. - def hiveSettings = sharedSettings ++ assemblyProjSettings ++ Seq( + def hiveSettings = sharedSettings ++ 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" % hiveVersion, diff --git a/sql/hive/pom.xml b/sql/hive/pom.xml index 63f592cb4b441..a662da76ce25a 100644 --- a/sql/hive/pom.xml +++ b/sql/hive/pom.xml @@ -63,6 +63,10 @@ hive-exec ${hive.version} + + org.codehaus.jackson + jackson-mapper-asl + org.apache.hive hive-serde @@ -87,6 +91,30 @@ org.scalatest scalatest-maven-plugin + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.4 + + + copy-dependencies + package + + copy-dependencies + + + + ${basedir}/../../lib_managed/jars + false + false + true + org.datanucleus + + + + From 1440154c27ca48b5a75103eccc9057286d3f6ca8 Mon Sep 17 00:00:00 2001 From: Evan Chan Date: Sun, 6 Apr 2014 19:17:33 -0700 Subject: [PATCH 216/397] SPARK-1154: Clean up app folders in worker nodes This is a fix for [SPARK-1154](https://issues.apache.org/jira/browse/SPARK-1154). The issue is that worker nodes fill up with a huge number of app-* folders after some time. This change adds a periodic cleanup task which asynchronously deletes app directories older than a configurable TTL. Two new configuration parameters have been introduced: spark.worker.cleanup_interval spark.worker.app_data_ttl This change does not include moving the downloads of application jars to a location outside of the work directory. We will address that if we have time, but that potentially involves caching so it will come either as part of this PR or a separate PR. Author: Evan Chan Author: Kelvin Chu Closes #288 from velvia/SPARK-1154-cleanup-app-folders and squashes the following commits: 0689995 [Evan Chan] CR from @aarondav - move config, clarify for standalone mode 9f10d96 [Evan Chan] CR from @pwendell - rename configs and add cleanup.enabled f2f6027 [Evan Chan] CR from @andrewor14 553d8c2 [Kelvin Chu] change the variable name to currentTimeMillis since it actually tracks in seconds 8dc9cb5 [Kelvin Chu] Fixed a bug in Utils.findOldFiles() after merge. cb52f2b [Kelvin Chu] Change the name of findOldestFiles() to findOldFiles() 72f7d2d [Kelvin Chu] Fix a bug of Utils.findOldestFiles(). file.lastModified is returned in milliseconds. ad99955 [Kelvin Chu] Add unit test for Utils.findOldestFiles() dc1a311 [Evan Chan] Don't recompute current time with every new file e3c408e [Evan Chan] Document the two new settings b92752b [Evan Chan] SPARK-1154: Add a periodic task to clean up app directories --- .../apache/spark/deploy/DeployMessage.scala | 4 +++ .../apache/spark/deploy/worker/Worker.scala | 23 +++++++++++++++- .../scala/org/apache/spark/util/Utils.scala | 19 ++++++++++++-- .../org/apache/spark/util/UtilsSuite.scala | 15 ++++++++++- docs/configuration.md | 26 +++++++++++++++++++ 5 files changed, 83 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala index 83ce14a0a806a..a7368f9f3dfbe 100644 --- a/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/DeployMessage.scala @@ -86,6 +86,10 @@ private[deploy] object DeployMessages { case class KillDriver(driverId: String) extends DeployMessage + // Worker internal + + case object WorkDirCleanup // Sent to Worker actor periodically for cleaning up app folders + // AppClient to Master case class RegisterApplication(appDescription: ApplicationDescription) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 8a71ddda4cb5e..bf5a8d09dd2df 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -64,6 +64,12 @@ private[spark] class Worker( val REGISTRATION_TIMEOUT = 20.seconds val REGISTRATION_RETRIES = 3 + val CLEANUP_ENABLED = conf.getBoolean("spark.worker.cleanup.enabled", true) + // How often worker will clean up old app folders + val CLEANUP_INTERVAL_MILLIS = conf.getLong("spark.worker.cleanup.interval", 60 * 30) * 1000 + // TTL for app folders/data; after TTL expires it will be cleaned up + val APP_DATA_RETENTION_SECS = conf.getLong("spark.worker.cleanup.appDataTtl", 7 * 24 * 3600) + // Index into masterUrls that we're currently trying to register with. var masterIndex = 0 @@ -179,12 +185,28 @@ private[spark] class Worker( registered = true changeMaster(masterUrl, masterWebUiUrl) context.system.scheduler.schedule(0 millis, HEARTBEAT_MILLIS millis, self, SendHeartbeat) + if (CLEANUP_ENABLED) { + context.system.scheduler.schedule(CLEANUP_INTERVAL_MILLIS millis, + CLEANUP_INTERVAL_MILLIS millis, self, WorkDirCleanup) + } case SendHeartbeat => masterLock.synchronized { if (connected) { master ! Heartbeat(workerId) } } + case WorkDirCleanup => + // Spin up a separate thread (in a future) to do the dir cleanup; don't tie up worker actor + val cleanupFuture = concurrent.future { + logInfo("Cleaning up oldest application directories in " + workDir + " ...") + Utils.findOldFiles(workDir, APP_DATA_RETENTION_SECS) + .foreach(Utils.deleteRecursively) + } + cleanupFuture onFailure { + case e: Throwable => + logError("App dir cleanup failed: " + e.getMessage, e) + } + case MasterChanged(masterUrl, masterWebUiUrl) => logInfo("Master has changed, new master is at " + masterUrl) changeMaster(masterUrl, masterWebUiUrl) @@ -331,7 +353,6 @@ private[spark] class Worker( } private[spark] object Worker { - def main(argStrings: Array[String]) { val args = new WorkerArguments(argStrings) val (actorSystem, _) = startSystemAndActor(args.host, args.port, args.webUiPort, args.cores, diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d3c39dee330b2..4435b21a7505e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -597,9 +597,24 @@ private[spark] object Utils extends Logging { } if (fileInCanonicalDir.getCanonicalFile().equals(fileInCanonicalDir.getAbsoluteFile())) { - return false; + return false } else { - return true; + return true + } + } + + /** + * Finds all the files in a directory whose last modified time is older than cutoff seconds. + * @param dir must be the path to a directory, or IllegalArgumentException is thrown + * @param cutoff measured in seconds. Files older than this are returned. + */ + def findOldFiles(dir: File, cutoff: Long): Seq[File] = { + val currentTimeMillis = System.currentTimeMillis + if (dir.isDirectory) { + val files = listFilesSafely(dir) + files.filter { file => file.lastModified < (currentTimeMillis - cutoff * 1000) } + } else { + throw new IllegalArgumentException(dir + " is not a directory!") } } diff --git a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala index 616214fb5e3a6..eb7fb6318262b 100644 --- a/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/UtilsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.util import scala.util.Random -import java.io.{ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} +import java.io.{File, ByteArrayOutputStream, ByteArrayInputStream, FileOutputStream} import java.nio.{ByteBuffer, ByteOrder} import com.google.common.base.Charsets @@ -154,5 +154,18 @@ class UtilsSuite extends FunSuite { val iterator = Iterator.range(0, 5) assert(Utils.getIteratorSize(iterator) === 5L) } + + test("findOldFiles") { + // create some temporary directories and files + val parent: File = Utils.createTempDir() + val child1: File = Utils.createTempDir(parent.getCanonicalPath) // The parent directory has two child directories + val child2: File = Utils.createTempDir(parent.getCanonicalPath) + // set the last modified time of child1 to 10 secs old + child1.setLastModified(System.currentTimeMillis() - (1000 * 10)) + + val result = Utils.findOldFiles(parent, 5) // find files older than 5 secs + assert(result.size.equals(1)) + assert(result(0).getCanonicalPath.equals(child1.getCanonicalPath)) + } } diff --git a/docs/configuration.md b/docs/configuration.md index b6005acac8b93..57bda20edcdf1 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -348,6 +348,32 @@ Apart from these, the following properties are also available, and may be useful receives no heartbeats. + + + + + + + + + + + + + + + From 87d0928a3301835705652c24a26096546597e156 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 6 Apr 2014 21:04:45 -0700 Subject: [PATCH 217/397] SPARK-1431: Allow merging conflicting pull requests Sometimes if there is a small conflict it's nice to be able to just manually fix it up rather than have another RTT with the contributor. Author: Patrick Wendell Closes #342 from pwendell/merge-conflicts and squashes the following commits: cdce61a [Patrick Wendell] SPARK-1431: Allow merging conflicting pull requests --- dev/merge_spark_pr.py | 26 ++++++++++++++++++++++---- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index e8f78fc5f231a..7a61943e94814 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -87,11 +87,20 @@ def merge_pr(pr_num, target_ref): run_cmd("git fetch %s %s:%s" % (PUSH_REMOTE_NAME, target_ref, target_branch_name)) run_cmd("git checkout %s" % target_branch_name) - run_cmd(['git', 'merge', pr_branch_name, '--squash']) + had_conflicts = False + try: + run_cmd(['git', 'merge', pr_branch_name, '--squash']) + except Exception as e: + msg = "Error merging: %s\nWould you like to manually fix-up this merge?" % e + continue_maybe(msg) + msg = "Okay, please fix any conflicts and 'git add' conflicting files... Finished?" + continue_maybe(msg) + had_conflicts = True commit_authors = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, '--pretty=format:%an <%ae>']).split("\n") - distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), reverse=True) + distinct_authors = sorted(set(commit_authors), key=lambda x: commit_authors.count(x), + reverse=True) primary_author = distinct_authors[0] commits = run_cmd(['git', 'log', 'HEAD..%s' % pr_branch_name, '--pretty=format:%h [%an] %s']).split("\n\n") @@ -105,6 +114,13 @@ def merge_pr(pr_num, target_ref): merge_message_flags += ["-m", authors] + if had_conflicts: + committer_name = run_cmd("git config --get user.name").strip() + committer_email = run_cmd("git config --get user.email").strip() + message = "This patch had conflicts when merged, resolved by\nCommitter: %s <%s>" % ( + committer_name, committer_email) + merge_message_flags += ["-m", message] + # The string "Closes #%s" string is required for GitHub to correctly close the PR merge_message_flags += ["-m", "Closes #%s from %s and squashes the following commits:" % (pr_num, pr_repo_desc)] @@ -186,8 +202,10 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): maybe_cherry_pick(pr_num, merge_hash, latest_branch) sys.exit(0) -if bool(pr["mergeable"]) == False: - fail("Pull request %s is not mergeable in its current form" % pr_num) +if not bool(pr["mergeable"]): + msg = "Pull request %s is not mergeable in its current form.\n" % pr_num + \ + "Continue? (experts only!)" + continue_maybe(msg) print ("\n=== Pull Request #%s ===" % pr_num) print("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( From accd0999f9cb6a449434d3fc5274dd469eeecab2 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 7 Apr 2014 00:14:00 -0700 Subject: [PATCH 218/397] [SQL] SPARK-1371 Hash Aggregation Improvements Given: ```scala case class Data(a: Int, b: Int) val rdd = sparkContext .parallelize(1 to 200) .flatMap(_ => (1 to 50000).map(i => Data(i % 100, i))) rdd.registerAsTable("data") cacheTable("data") ``` Before: ``` SELECT COUNT(*) FROM data:[10000000] 16795.567ms SELECT a, SUM(b) FROM data GROUP BY a 7536.436ms SELECT SUM(b) FROM data 10954.1ms ``` After: ``` SELECT COUNT(*) FROM data:[10000000] 1372.175ms SELECT a, SUM(b) FROM data GROUP BY a 2070.446ms SELECT SUM(b) FROM data 958.969ms ``` Author: Michael Armbrust Closes #295 from marmbrus/hashAgg and squashes the following commits: ec63575 [Michael Armbrust] Add comment. d0495a9 [Michael Armbrust] Use scaladoc instead. b4a6887 [Michael Armbrust] Address review comments. a2d90ba [Michael Armbrust] Capture child output statically to avoid issues with generators and serialization. 7c13112 [Michael Armbrust] Rewrite Aggregate operator to stream input and use projections. Remove unused local RDD functions implicits. 5096f99 [Michael Armbrust] Make HiveUDAF fields transient since object inspectors are not serializable. 6a4b671 [Michael Armbrust] Add option to avoid binding operators expressions automatically. 92cca08 [Michael Armbrust] Always include serialization debug info when running tests. 1279df2 [Michael Armbrust] Increase default number of partitions. --- project/SparkBuild.scala | 1 + .../catalyst/expressions/BoundAttribute.scala | 6 + .../sql/catalyst/expressions/Projection.scala | 6 +- .../sql/catalyst/expressions/aggregates.scala | 16 +- .../rdd/PartitionLocalRDDFunctions.scala | 100 ---------- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../spark/sql/execution/aggregates.scala | 183 +++++++++++++----- .../org/apache/spark/sql/hive/hiveUdfs.scala | 3 + 8 files changed, 157 insertions(+), 160 deletions(-) delete mode 100644 sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index d1e4b8b964b88..6b8740d9f21a1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -178,6 +178,7 @@ object SparkBuild extends Build { fork := true, javaOptions in Test += "-Dspark.home=" + sparkHome, javaOptions in Test += "-Dspark.testing=1", + javaOptions in Test += "-Dsun.io.serialization.extendedDebugInfo=true", 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. 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 f70e80b7f27f2..37b9035df9d8c 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 @@ -48,11 +48,17 @@ case class BoundReference(ordinal: Int, baseReference: Attribute) override def apply(input: Row): Any = input(ordinal) } +/** + * Used to denote operators that do their own binding of attributes internally. + */ +trait NoBind { self: trees.TreeNode[_] => } + class BindReferences[TreeNode <: QueryPlan[TreeNode]] extends Rule[TreeNode] { import BindReferences._ def apply(plan: TreeNode): TreeNode = { plan.transform { + case n: NoBind => n.asInstanceOf[TreeNode] case leafNode if leafNode.children.isEmpty => leafNode case unaryNode if unaryNode.children.size == 1 => unaryNode.transformExpressions { case e => bindReference(e, unaryNode.children.head.output) 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 38542d3fc7290..5576ecbb65ef5 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,9 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { protected val exprArray = expressions.toArray def apply(input: Row): Row = { - val outputArray = new Array[Any](exprArray.size) + val outputArray = new Array[Any](exprArray.length) var i = 0 - while (i < exprArray.size) { + while (i < exprArray.length) { outputArray(i) = exprArray(i).apply(input) i += 1 } @@ -57,7 +57,7 @@ case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) def apply(input: Row): Row = { var i = 0 - while (i < exprArray.size) { + while (i < exprArray.length) { mutableRow(i) = exprArray(i).apply(input) i += 1 } 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 7303b155cae3d..53b884a41e16b 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 @@ -27,7 +27,7 @@ abstract class AggregateExpression extends Expression { * Creates a new instance that can be used to compute this aggregate expression for a group * of input rows/ */ - def newInstance: AggregateFunction + def newInstance(): AggregateFunction } /** @@ -75,7 +75,7 @@ abstract class AggregateFunction override def apply(input: Row): Any // Do we really need this? - def newInstance = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) + def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -89,7 +89,7 @@ case class Count(child: Expression) extends PartialAggregate with trees.UnaryNod SplitEvaluation(Sum(partialCount.toAttribute), partialCount :: Nil) } - override def newInstance = new CountFunction(child, this) + override def newInstance()= new CountFunction(child, this) } case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { @@ -98,7 +98,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) + override def newInstance()= new CountDistinctFunction(expressions, this) } case class Average(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -118,7 +118,7 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN partialCount :: partialSum :: Nil) } - override def newInstance = new AverageFunction(child, this) + override def newInstance()= new AverageFunction(child, this) } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -134,7 +134,7 @@ case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[ partialSum :: Nil) } - override def newInstance = new SumFunction(child, this) + override def newInstance()= new SumFunction(child, this) } case class SumDistinct(child: Expression) @@ -145,7 +145,7 @@ case class SumDistinct(child: Expression) def dataType = child.dataType override def toString = s"SUM(DISTINCT $child)" - override def newInstance = new SumDistinctFunction(child, this) + override def newInstance()= new SumDistinctFunction(child, this) } case class First(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { @@ -160,7 +160,7 @@ case class First(child: Expression) extends PartialAggregate with trees.UnaryNod First(partialFirst.toAttribute), partialFirst :: Nil) } - override def newInstance = new FirstFunction(child, this) + override def newInstance()= new FirstFunction(child, this) } case class AverageFunction(expr: Expression, base: AggregateExpression) diff --git a/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala b/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala deleted file mode 100644 index f1230e7526ab1..0000000000000 --- a/sql/core/src/main/scala/org/apache/spark/rdd/PartitionLocalRDDFunctions.scala +++ /dev/null @@ -1,100 +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.rdd - -import scala.language.implicitConversions - -import scala.reflect._ -import scala.collection.mutable.ArrayBuffer - -import org.apache.spark.{Aggregator, InterruptibleIterator, Logging} -import org.apache.spark.util.collection.AppendOnlyMap - -/* Implicit conversions */ -import org.apache.spark.SparkContext._ - -/** - * Extra functions on RDDs that perform only local operations. These can be used when data has - * already been partitioned correctly. - */ -private[spark] class PartitionLocalRDDFunctions[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, _ ++ _) - val bufs = self.mapPartitionsWithContext((context, iter) => { - new InterruptibleIterator(context, aggregator.combineValuesByKey(iter, context)) - }, 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) - } - } -} - -private[spark] object PartitionLocalRDDFunctions { - implicit def rddToPartitionLocalRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = - new PartitionLocalRDDFunctions(rdd) -} - - 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 869673b1fe978..450c142c0baa4 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 @@ -76,7 +76,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una */ object AddExchange extends Rule[SparkPlan] { // TODO: Determine the number of partitions. - val numPartitions = 8 + val numPartitions = 150 def apply(plan: SparkPlan): SparkPlan = plan.transformUp { case operator: SparkPlan => 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 8515a18f18c55..2a4f7b5670457 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 @@ -17,14 +17,13 @@ package org.apache.spark.sql.execution +import java.util.HashMap + import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ -/* Implicit conversions */ -import org.apache.spark.rdd.PartitionLocalRDDFunctions._ - /** * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each * group. @@ -40,7 +39,7 @@ case class Aggregate( groupingExpressions: Seq[Expression], aggregateExpressions: Seq[NamedExpression], child: SparkPlan)(@transient sc: SparkContext) - extends UnaryNode { + extends UnaryNode with NoBind { override def requiredChildDistribution = if (partial) { @@ -55,61 +54,149 @@ case class Aggregate( override def otherCopyArgs = sc :: Nil + // HACK: Generators don't correctly preserve their output through serializations so we grab + // out child's output attributes statically here. + val childOutput = child.output + 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 + /** + * An aggregate that needs to be computed for each row in a group. + * + * @param unbound Unbound version of this aggregate, used for result substitution. + * @param aggregate A bound copy of this aggregate used to create a new aggregation buffer. + * @param resultAttribute An attribute used to refer to the result of this aggregate in the final + * output. + */ + case class ComputedAggregate( + unbound: AggregateExpression, + aggregate: AggregateExpression, + resultAttribute: AttributeReference) + + /** A list of aggregates that need to be computed for each group. */ + @transient + lazy val computedAggregates = aggregateExpressions.flatMap { agg => + agg.collect { + case a: AggregateExpression => + ComputedAggregate( + a, + BindReferences.bindReference(a, childOutput).asInstanceOf[AggregateExpression], + AttributeReference(s"aggResult:$a", a.dataType, nullable = true)()) } + }.toArray + + /** The schema of the result of all aggregate evaluations */ + @transient + lazy val computedSchema = computedAggregates.map(_.resultAttribute) + + /** Creates a new aggregate buffer for a group. */ + def newAggregateBuffer(): Array[AggregateFunction] = { + val buffer = new Array[AggregateFunction](computedAggregates.length) + var i = 0 + while (i < computedAggregates.length) { + buffer(i) = computedAggregates(i).aggregate.newInstance() + i += 1 + } + buffer + } - 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(ordinal, Alias(impl, "AGGEXPR")().toAttribute)) - } + /** Named attributes used to substitute grouping attributes into the final result. */ + @transient + lazy val namedGroups = groupingExpressions.map { + case ne: NamedExpression => ne -> ne.toAttribute + case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute + } - exactGroupingExpr.getOrElse( - sys.error(s"$agg is not in grouping expressions: $groupingExpressions")) - } else { - impl + /** + * A map of substitutions that are used to insert the aggregate expressions and grouping + * expression into the final result expression. + */ + @transient + lazy val resultMap = + (computedAggregates.map { agg => agg.unbound -> agg.resultAttribute} ++ namedGroups).toMap + + /** + * Substituted version of aggregateExpressions expressions which are used to compute final + * output rows given a group and the result of all aggregate computations. + */ + @transient + lazy val resultExpressions = aggregateExpressions.map { agg => + agg.transform { + case e: Expression if resultMap.contains(e) => resultMap(e) } } 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().mapPartitions { iter => - val buildGrouping = new Projection(groupingExpressions) - iter.map(row => (buildGrouping(row), row.copy())) - }.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 => - aggFunctions.foreach(_.update(row)) + if (groupingExpressions.isEmpty) { + child.execute().mapPartitions { iter => + val buffer = newAggregateBuffer() + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + var i = 0 + while (i < buffer.length) { + buffer(i).update(currentRow) + i += 1 + } + } + val resultProjection = new Projection(resultExpressions, computedSchema) + val aggregateResults = new GenericMutableRow(computedAggregates.length) + + var i = 0 + while (i < buffer.length) { + aggregateResults(i) = buffer(i).apply(EmptyRow) + i += 1 + } + + Iterator(resultProjection(aggregateResults)) } - buildRow(aggImplementations.map(_.apply(group))) - } - - // 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(_.apply(null))) :: Nil) } else { - result + child.execute().mapPartitions { iter => + val hashTable = new HashMap[Row, Array[AggregateFunction]] + val groupingProjection = new MutableProjection(groupingExpressions, childOutput) + + var currentRow: Row = null + while (iter.hasNext) { + currentRow = iter.next() + val currentGroup = groupingProjection(currentRow) + var currentBuffer = hashTable.get(currentGroup) + if (currentBuffer == null) { + currentBuffer = newAggregateBuffer() + hashTable.put(currentGroup.copy(), currentBuffer) + } + + var i = 0 + while (i < currentBuffer.length) { + currentBuffer(i).update(currentRow) + i += 1 + } + } + + new Iterator[Row] { + private[this] val hashTableIter = hashTable.entrySet().iterator() + private[this] val aggregateResults = new GenericMutableRow(computedAggregates.length) + private[this] val resultProjection = + new MutableProjection(resultExpressions, computedSchema ++ namedGroups.map(_._2)) + private[this] val joinedRow = new JoinedRow + + override final def hasNext: Boolean = hashTableIter.hasNext + + override final def next(): Row = { + val currentEntry = hashTableIter.next() + val currentGroup = currentEntry.getKey + val currentBuffer = currentEntry.getValue + + var i = 0 + while (i < currentBuffer.length) { + // Evaluating an aggregate buffer returns the result. No row is required since we + // already added all rows in the group using update. + aggregateResults(i) = currentBuffer(i).apply(EmptyRow) + i += 1 + } + resultProjection(joinedRow(aggregateResults, currentGroup)) + } + } + } } } } 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 44901db3f963b..2c607455c8e4b 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 @@ -337,13 +337,16 @@ case class HiveGenericUdaf( type UDFType = AbstractGenericUDAFResolver + @transient protected lazy val resolver: AbstractGenericUDAFResolver = createFunction(name) + @transient protected lazy val objectInspector = { resolver.getEvaluator(children.map(_.dataType.toTypeInfo).toArray) .init(GenericUDAFEvaluator.Mode.COMPLETE, inspectors.toArray) } + @transient protected lazy val inspectors = children.map(_.dataType).map(toInspector) def dataType: DataType = inspectorToDataType(objectInspector) From b5bae849dbdf833a80746b705732b8c86971fddc Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Mon, 7 Apr 2014 01:46:50 -0700 Subject: [PATCH 219/397] [SQL] SPARK-1427 Fix toString for SchemaRDD NativeCommands. Author: Michael Armbrust Closes #343 from marmbrus/toStringFix and squashes the following commits: 37198fe [Michael Armbrust] Fix toString for SchemaRDD NativeCommands. --- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 ++ .../src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala | 3 +-- .../main/scala/org/apache/spark/sql/hive/HiveContext.scala | 6 ++++++ .../apache/spark/sql/hive/execution/HiveQuerySuite.scala | 4 ++++ 4 files changed, 13 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 36059c6630aa4..3193787680d16 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -224,6 +224,8 @@ class SQLContext(@transient val sparkContext: SparkContext) protected def stringOrError[A](f: => A): String = try f.toString catch { case e: Throwable => e.toString } + def simpleString: String = stringOrError(executedPlan) + override def toString: String = s"""== Logical Plan == |${stringOrError(analyzed)} diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index 840803a52c1cf..3dd9897c0d3b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -41,8 +41,7 @@ trait SchemaRDDLike { override def toString = s"""${super.toString} |== Query Plan == - |${queryExecution.executedPlan}""".stripMargin.trim - + |${queryExecution.simpleString}""".stripMargin.trim /** * Saves the contents of this `SchemaRDD` as a parquet file, preserving the schema. Files that 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 f66a667c0a942..353458432b210 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 @@ -297,5 +297,11 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { val asString = result.map(_.zip(types).map(toHiveString)).map(_.mkString("\t")).toSeq asString } + + override def simpleString: String = + logical match { + case _: NativeCommand => "" + case _ => executedPlan.toString + } } } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index 0c27498a93507..a09667ac84b01 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -146,4 +146,8 @@ class HiveQuerySuite extends HiveComparisonTest { hql("SELECT * FROM src TABLESAMPLE(0.1 PERCENT) s") } + test("SchemaRDD toString") { + hql("SHOW TABLES").toString + hql("SELECT * FROM src").toString + } } From a3c51c6ea2320efdeb2a6a5c1cd11d714f8994aa Mon Sep 17 00:00:00 2001 From: Davis Shepherd Date: Mon, 7 Apr 2014 10:02:00 -0700 Subject: [PATCH 220/397] SPARK-1432: Make sure that all metadata fields are properly cleaned While working on spark-1337 with @pwendell, we noticed that not all of the metadata maps in JobProgessListener were being properly cleaned. This could lead to a (hypothetical) memory leak issue should a job run long enough. This patch aims to address the issue. Author: Davis Shepherd Closes #338 from dgshep/master and squashes the following commits: a77b65c [Davis Shepherd] In the contex of SPARK-1337: Make sure that all metadata fields are properly cleaned --- .../scala/org/apache/spark/ui/jobs/JobProgressListener.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index cd4be57227a16..048f671c8788f 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -83,7 +83,6 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { if (stages.size > retainedStages) { val toRemove = math.max(retainedStages / 10, 1) stages.take(toRemove).foreach { s => - stageIdToTaskData.remove(s.stageId) stageIdToTime.remove(s.stageId) stageIdToShuffleRead.remove(s.stageId) stageIdToShuffleWrite.remove(s.stageId) @@ -92,8 +91,10 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { stageIdToTasksActive.remove(s.stageId) stageIdToTasksComplete.remove(s.stageId) stageIdToTasksFailed.remove(s.stageId) + stageIdToTaskData.remove(s.stageId) + stageIdToExecutorSummaries.remove(s.stageId) stageIdToPool.remove(s.stageId) - if (stageIdToDescription.contains(s.stageId)) {stageIdToDescription.remove(s.stageId)} + stageIdToDescription.remove(s.stageId) } stages.trimStart(toRemove) } From 83f2a2f14e4145a04672e42216d43100a66b1fc2 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 10:45:31 -0700 Subject: [PATCH 221/397] [sql] Rename Expression.apply to eval for better readability. Also used this opportunity to add a bunch of override's and made some members private. Author: Reynold Xin Closes #340 from rxin/eval and squashes the following commits: a7c7ca7 [Reynold Xin] Fixed conflicts in merge. 9069de6 [Reynold Xin] Merge branch 'master' into eval 3ccc313 [Reynold Xin] Merge branch 'master' into eval 1a47e10 [Reynold Xin] Renamed apply to eval for generators and added a bunch of override's. ea061de [Reynold Xin] Rename Expression.apply to eval for better readability. --- .../catalyst/expressions/BoundAttribute.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 4 +- .../sql/catalyst/expressions/Expression.scala | 26 ++--- .../sql/catalyst/expressions/Projection.scala | 5 +- .../spark/sql/catalyst/expressions/Row.scala | 4 +- .../sql/catalyst/expressions/ScalaUdf.scala | 8 +- .../catalyst/expressions/WrapDynamic.scala | 2 +- .../sql/catalyst/expressions/aggregates.scala | 96 +++++++++---------- .../sql/catalyst/expressions/arithmetic.scala | 12 +-- .../catalyst/expressions/complexTypes.scala | 14 +-- .../sql/catalyst/expressions/generators.scala | 20 ++-- .../sql/catalyst/expressions/literals.scala | 6 +- .../expressions/namedExpressions.scala | 2 +- .../catalyst/expressions/nullFunctions.scala | 12 +-- .../sql/catalyst/expressions/predicates.scala | 46 ++++----- .../expressions/stringOperations.scala | 10 +- .../sql/catalyst/optimizer/Optimizer.scala | 2 +- .../ExpressionEvaluationSuite.scala | 8 +- .../apache/spark/sql/execution/Generate.scala | 8 +- .../spark/sql/execution/aggregates.scala | 4 +- .../spark/sql/execution/basicOperators.scala | 2 +- .../apache/spark/sql/execution/TgfSuite.scala | 6 +- .../apache/spark/sql/hive/hiveOperators.scala | 4 +- .../org/apache/spark/sql/hive/hiveUdfs.scala | 12 +-- 24 files changed, 156 insertions(+), 159 deletions(-) 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 37b9035df9d8c..4ebf6c4584b94 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 @@ -45,7 +45,7 @@ case class BoundReference(ordinal: Int, baseReference: Attribute) override def toString = s"$baseReference:$ordinal" - override def apply(input: Row): Any = input(ordinal) + override def eval(input: Row): Any = input(ordinal) } /** 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 941b53fe70d23..89226999ca005 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 @@ -185,8 +185,8 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case DoubleType => castToDouble } - override def apply(input: Row): Any = { - val evaluated = child.apply(input) + override def eval(input: Row): Any = { + val evaluated = child.eval(input) if (evaluated == null) { null } else { 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 a3d19525503ba..f190bd0cca375 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 @@ -17,8 +17,8 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.trees.TreeNode import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType, NativeType} @@ -50,7 +50,7 @@ abstract class Expression extends TreeNode[Expression] { def references: Set[Attribute] /** Returns the result of evaluating this expression on a given input Row */ - def apply(input: Row = null): EvaluatedType = + def eval(input: Row = null): EvaluatedType = throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") /** @@ -73,7 +73,7 @@ abstract class Expression extends TreeNode[Expression] { */ @inline def n1(e: Expression, i: Row, f: ((Numeric[Any], Any) => Any)): Any = { - val evalE = e.apply(i) + val evalE = e.eval(i) if (evalE == null) { null } else { @@ -102,11 +102,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i) + val evalE1 = e1.eval(i) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i) + val evalE2 = e2.eval(i) if (evalE2 == null) { null } else { @@ -135,11 +135,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i: Row) + val evalE1 = e1.eval(i: Row) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i: Row) + val evalE2 = e2.eval(i: Row) if (evalE2 == null) { null } else { @@ -168,11 +168,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i) + val evalE1 = e1.eval(i) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i) + val evalE2 = e2.eval(i) if (evalE2 == null) { null } else { @@ -205,11 +205,11 @@ abstract class Expression extends TreeNode[Expression] { throw new TreeNodeException(this, s"Types do not match ${e1.dataType} != ${e2.dataType}") } - val evalE1 = e1.apply(i) + val evalE1 = e1.eval(i) if(evalE1 == null) { null } else { - val evalE2 = e2.apply(i) + val evalE2 = e2.eval(i) if (evalE2 == null) { null } else { @@ -231,7 +231,7 @@ abstract class BinaryExpression extends Expression with trees.BinaryNode[Express override def foldable = left.foldable && right.foldable - def references = left.references ++ right.references + override def references = left.references ++ right.references override def toString = s"($left $symbol $right)" } @@ -243,5 +243,5 @@ abstract class LeafExpression extends Expression with trees.LeafNode[Expression] abstract class UnaryExpression extends Expression with trees.UnaryNode[Expression] { self: Product => - def references = child.references + override def references = child.references } 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 5576ecbb65ef5..c9b7cea6a3e5f 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 @@ -27,11 +27,12 @@ class Projection(expressions: Seq[Expression]) extends (Row => Row) { this(expressions.map(BindReferences.bindReference(_, inputSchema))) protected val exprArray = expressions.toArray + def apply(input: Row): Row = { val outputArray = new Array[Any](exprArray.length) var i = 0 while (i < exprArray.length) { - outputArray(i) = exprArray(i).apply(input) + outputArray(i) = exprArray(i).eval(input) i += 1 } new GenericRow(outputArray) @@ -58,7 +59,7 @@ case class MutableProjection(expressions: Seq[Expression]) extends (Row => Row) def apply(input: Row): Row = { var i = 0 while (i < exprArray.length) { - mutableRow(i) = exprArray(i).apply(input) + mutableRow(i) = exprArray(i).eval(input) i += 1 } mutableRow 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 9f4d84466e6ac..0f06ea088e1a1 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 @@ -212,8 +212,8 @@ class RowOrdering(ordering: Seq[SortOrder]) extends Ordering[Row] { var i = 0 while (i < ordering.size) { val order = ordering(i) - val left = order.child.apply(a) - val right = order.child.apply(b) + val left = order.child.eval(a) + val right = order.child.eval(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 f53d8504b083f..5e089f7618e0a 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 @@ -27,13 +27,13 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi def references = children.flatMap(_.references).toSet def nullable = true - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { children.size match { - case 1 => function.asInstanceOf[(Any) => Any](children(0).apply(input)) + case 1 => function.asInstanceOf[(Any) => Any](children(0).eval(input)) case 2 => function.asInstanceOf[(Any, Any) => Any]( - children(0).apply(input), - children(1).apply(input)) + children(0).eval(input), + children(1).eval(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 9828d0b9bd8b2..e787c59e75723 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 @@ -30,7 +30,7 @@ case class WrapDynamic(children: Seq[Attribute]) extends Expression { def references = children.toSet def dataType = DynamicType - override def apply(input: Row): DynamicRow = input match { + override def eval(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) 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 53b884a41e16b..5edcea14278c7 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 @@ -43,7 +43,7 @@ case class SplitEvaluation( partialEvaluations: Seq[NamedExpression]) /** - * An [[AggregateExpression]] that can be partially computed without seeing all relevent tuples. + * An [[AggregateExpression]] that can be partially computed without seeing all relevant tuples. * These partial evaluations can then be combined to compute the actual answer. */ abstract class PartialAggregate extends AggregateExpression { @@ -63,28 +63,28 @@ abstract class AggregateFunction extends AggregateExpression with Serializable with trees.LeafNode[Expression] { self: Product => - type EvaluatedType = Any + override 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 + override def references = base.references + override def nullable = base.nullable + override def dataType = base.dataType def update(input: Row): Unit - override def apply(input: Row): Any + override def eval(input: Row): Any // Do we really need this? - def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) + override def newInstance() = makeCopy(productIterator.map { case a: AnyRef => a }.toArray) } case class Count(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = IntegerType + override def references = child.references + override def nullable = false + override def dataType = IntegerType override def toString = s"COUNT($child)" - def asPartial: SplitEvaluation = { + override def asPartial: SplitEvaluation = { val partialCount = Alias(Count(child), "PartialCount")() SplitEvaluation(Sum(partialCount.toAttribute), partialCount :: Nil) } @@ -93,18 +93,18 @@ case class Count(child: Expression) extends PartialAggregate with trees.UnaryNod } case class CountDistinct(expressions: Seq[Expression]) extends AggregateExpression { - def children = expressions - def references = expressions.flatMap(_.references).toSet - def nullable = false - def dataType = IntegerType + override def children = expressions + override def references = expressions.flatMap(_.references).toSet + override def nullable = false + override 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] { - def references = child.references - def nullable = false - def dataType = DoubleType + override def references = child.references + override def nullable = false + override def dataType = DoubleType override def toString = s"AVG($child)" override def asPartial: SplitEvaluation = { @@ -122,9 +122,9 @@ case class Average(child: Expression) extends PartialAggregate with trees.UnaryN } case class Sum(child: Expression) extends PartialAggregate with trees.UnaryNode[Expression] { - def references = child.references - def nullable = false - def dataType = child.dataType + override def references = child.references + override def nullable = false + override def dataType = child.dataType override def toString = s"SUM($child)" override def asPartial: SplitEvaluation = { @@ -140,18 +140,18 @@ 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 references = child.references + override def nullable = false + override 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] { - def references = child.references - def nullable = child.nullable - def dataType = child.dataType + override def references = child.references + override def nullable = child.nullable + override def dataType = child.dataType override def toString = s"FIRST($child)" override def asPartial: SplitEvaluation = { @@ -169,17 +169,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(EmptyRow)) + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).eval(EmptyRow)) private val sumAsDouble = Cast(sum, DoubleType) - - private val addFunction = Add(sum, expr) - override def apply(input: Row): Any = - sumAsDouble.apply(EmptyRow).asInstanceOf[Double] / count.toDouble + override def eval(input: Row): Any = + sumAsDouble.eval(EmptyRow).asInstanceOf[Double] / count.toDouble - def update(input: Row): Unit = { + override def update(input: Row): Unit = { count += 1 sum.update(addFunction, input) } @@ -190,28 +188,28 @@ case class CountFunction(expr: Expression, base: AggregateExpression) extends Ag var count: Int = _ - def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.apply(input)) + override def update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.eval(input)) if (evaluatedExpr.map(_ != null).reduceLeft(_ || _)) { count += 1 } } - override def apply(input: Row): Any = count + override def eval(input: Row): Any = count } case class SumFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { def this() = this(null, null) // Required for serialization. - private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).apply(null)) + private val sum = MutableLiteral(Cast(Literal(0), expr.dataType).eval(null)) private val addFunction = Add(sum, expr) - def update(input: Row): Unit = { + override def update(input: Row): Unit = { sum.update(addFunction, input) } - override def apply(input: Row): Any = sum.apply(null) + override def eval(input: Row): Any = sum.eval(null) } case class SumDistinctFunction(expr: Expression, base: AggregateExpression) @@ -219,16 +217,16 @@ case class SumDistinctFunction(expr: Expression, base: AggregateExpression) def this() = this(null, null) // Required for serialization. - val seen = new scala.collection.mutable.HashSet[Any]() + private val seen = new scala.collection.mutable.HashSet[Any]() - def update(input: Row): Unit = { - val evaluatedExpr = expr.apply(input) + override def update(input: Row): Unit = { + val evaluatedExpr = expr.eval(input) if (evaluatedExpr != null) { seen += evaluatedExpr } } - override def apply(input: Row): Any = + override def eval(input: Row): Any = seen.reduceLeft(base.dataType.asInstanceOf[NumericType].numeric.asInstanceOf[Numeric[Any]].plus) } @@ -239,14 +237,14 @@ case class CountDistinctFunction(expr: Seq[Expression], base: AggregateExpressio val seen = new scala.collection.mutable.HashSet[Any]() - def update(input: Row): Unit = { - val evaluatedExpr = expr.map(_.apply(input)) + override def update(input: Row): Unit = { + val evaluatedExpr = expr.map(_.eval(input)) if (evaluatedExpr.map(_ != null).reduceLeft(_ && _)) { seen += evaluatedExpr } } - override def apply(input: Row): Any = seen.size + override def eval(input: Row): Any = seen.size } case class FirstFunction(expr: Expression, base: AggregateExpression) extends AggregateFunction { @@ -254,11 +252,11 @@ case class FirstFunction(expr: Expression, base: AggregateExpression) extends Ag var result: Any = null - def update(input: Row): Unit = { + override def update(input: Row): Unit = { if (result == null) { - result = expr.apply(input) + result = expr.eval(input) } } - override def apply(input: Row): Any = result + override def eval(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 fba056e7c07e3..c79c1847cedf5 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 @@ -28,7 +28,7 @@ case class UnaryMinus(child: Expression) extends UnaryExpression { def nullable = child.nullable override def toString = s"-$child" - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { n1(child, input, _.negate(_)) } } @@ -55,25 +55,25 @@ 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(_, _)) + override def eval(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(_, _)) + override def eval(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(_, _)) + override def eval(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 { + override def eval(input: Row): Any = dataType match { case _: FractionalType => f2(input, left, right, _.div(_, _)) case _: IntegralType => i2(input, left , right, _.quot(_, _)) } @@ -83,5 +83,5 @@ case class Divide(left: Expression, right: Expression) extends BinaryArithmetic case class Remainder(left: Expression, right: Expression) extends BinaryArithmetic { def symbol = "%" - override def apply(input: Row): Any = i2(input, left, right, _.rem(_, _)) + override def eval(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 ab96618d73df7..c947155cb701c 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 @@ -39,10 +39,10 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { override def toString = s"$child[$ordinal]" - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { if (child.dataType.isInstanceOf[ArrayType]) { - val baseValue = child.apply(input).asInstanceOf[Seq[_]] - val o = ordinal.apply(input).asInstanceOf[Int] + val baseValue = child.eval(input).asInstanceOf[Seq[_]] + val o = ordinal.eval(input).asInstanceOf[Int] if (baseValue == null) { null } else if (o >= baseValue.size || o < 0) { @@ -51,8 +51,8 @@ case class GetItem(child: Expression, ordinal: Expression) extends Expression { baseValue(o) } } else { - val baseValue = child.apply(input).asInstanceOf[Map[Any, _]] - val key = ordinal.apply(input) + val baseValue = child.eval(input).asInstanceOf[Map[Any, _]] + val key = ordinal.eval(input) if (baseValue == null) { null } else { @@ -85,8 +85,8 @@ case class GetField(child: Expression, fieldName: String) extends UnaryExpressio override lazy val resolved = childrenResolved && child.dataType.isInstanceOf[StructType] - override def apply(input: Row): Any = { - val baseValue = child.apply(input).asInstanceOf[Row] + override def eval(input: Row): Any = { + val baseValue = child.eval(input).asInstanceOf[Row] if (baseValue == null) null else baseValue(ordinal) } 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 e9b491b10a5f2..dd78614754e12 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 @@ -35,17 +35,17 @@ import org.apache.spark.sql.catalyst.types._ * 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]) { +abstract class Generator extends Expression { self: Product => - type EvaluatedType = TraversableOnce[Row] + override type EvaluatedType = TraversableOnce[Row] - lazy val dataType = + override lazy val dataType = ArrayType(StructType(output.map(a => StructField(a.name, a.dataType, a.nullable)))) - def nullable = false + override def nullable = false - def references = children.flatMap(_.references).toSet + override def references = children.flatMap(_.references).toSet /** * Should be overridden by specific generators. Called only once for each instance to ensure @@ -63,7 +63,7 @@ abstract class Generator extends Expression with (Row => TraversableOnce[Row]) { } /** Should be implemented by child classes to perform specific Generators. */ - def apply(input: Row): TraversableOnce[Row] + override def eval(input: Row): TraversableOnce[Row] /** Overridden `makeCopy` also copies the attributes that are produced by this generator. */ override def makeCopy(newArgs: Array[AnyRef]): this.type = { @@ -83,7 +83,7 @@ case class Explode(attributeNames: Seq[String], child: Expression) child.resolved && (child.dataType.isInstanceOf[ArrayType] || child.dataType.isInstanceOf[MapType]) - lazy val elementTypes = child.dataType match { + private lazy val elementTypes = child.dataType match { case ArrayType(et) => et :: Nil case MapType(kt,vt) => kt :: vt :: Nil } @@ -100,13 +100,13 @@ case class Explode(attributeNames: Seq[String], child: Expression) } } - override def apply(input: Row): TraversableOnce[Row] = { + override def eval(input: Row): TraversableOnce[Row] = { child.dataType match { case ArrayType(_) => - val inputArray = child.apply(input).asInstanceOf[Seq[Any]] + val inputArray = child.eval(input).asInstanceOf[Seq[Any]] if (inputArray == null) Nil else inputArray.map(v => new GenericRow(Array(v))) case MapType(_, _) => - val inputMap = child.apply(input).asInstanceOf[Map[Any,Any]] + val inputMap = child.eval(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 d879b2b5e8ba1..e15e16d633365 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 @@ -57,7 +57,7 @@ case class Literal(value: Any, dataType: DataType) extends LeafExpression { override def toString = if (value != null) value.toString else "null" type EvaluatedType = Any - override def apply(input: Row):Any = value + override def eval(input: Row):Any = value } // TODO: Specialize @@ -69,8 +69,8 @@ case class MutableLiteral(var value: Any, nullable: Boolean = true) extends Leaf def references = Set.empty def update(expression: Expression, input: Row) = { - value = expression.apply(input) + value = expression.eval(input) } - override def apply(input: Row) = value + override def eval(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 69c8bed309c18..eb4bc8e755284 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 @@ -79,7 +79,7 @@ case class Alias(child: Expression, name: String) type EvaluatedType = Any - override def apply(input: Row) = child.apply(input) + override def eval(input: Row) = child.eval(input) def dataType = child.dataType def nullable = child.nullable 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 5a47768dcb4a1..ce6d99c911ab3 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 @@ -41,11 +41,11 @@ case class Coalesce(children: Seq[Expression]) extends Expression { throw new UnresolvedException(this, "Coalesce cannot have children of different types.") } - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { var i = 0 var result: Any = null while(i < children.size && result == null) { - result = children(i).apply(input) + result = children(i).eval(input) i += 1 } result @@ -57,8 +57,8 @@ case class IsNull(child: Expression) extends Predicate with trees.UnaryNode[Expr override def foldable = child.foldable def nullable = false - override def apply(input: Row): Any = { - child.apply(input) == null + override def eval(input: Row): Any = { + child.eval(input) == null } } @@ -68,7 +68,7 @@ case class IsNotNull(child: Expression) extends Predicate with trees.UnaryNode[E def nullable = false override def toString = s"IS NOT NULL $child" - override def apply(input: Row): Any = { - child.apply(input) != null + override def eval(input: Row): Any = { + child.eval(input) != null } } 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 b74809e5ca67d..da5b2cf5b0362 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.types.{BooleanType, StringType, TimestampTy object InterpretedPredicate { def apply(expression: Expression): (Row => Boolean) = { - (r: Row) => expression.apply(r).asInstanceOf[Boolean] + (r: Row) => expression.eval(r).asInstanceOf[Boolean] } } @@ -54,8 +54,8 @@ case class Not(child: Expression) extends Predicate with trees.UnaryNode[Express def nullable = child.nullable override def toString = s"NOT $child" - override def apply(input: Row): Any = { - child.apply(input) match { + override def eval(input: Row): Any = { + child.eval(input) match { case null => null case b: Boolean => !b } @@ -71,18 +71,18 @@ case class In(value: Expression, list: Seq[Expression]) extends Predicate { 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) + override def eval(input: Row): Any = { + val evaluatedValue = value.eval(input) + list.exists(e => e.eval(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) + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) if (l == false || r == false) { false } else if (l == null || r == null ) { @@ -96,9 +96,9 @@ case class And(left: Expression, right: Expression) extends BinaryPredicate { 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) + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(input) if (l == true || r == true) { true } else if (l == null || r == null) { @@ -115,31 +115,31 @@ 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) + override def eval(input: Row): Any = { + val l = left.eval(input) + val r = right.eval(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 = c2(input, left, right, _.lt(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.lt(_, _)) } case class LessThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = "<=" - override def apply(input: Row): Any = c2(input, left, right, _.lteq(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.lteq(_, _)) } case class GreaterThan(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">" - override def apply(input: Row): Any = c2(input, left, right, _.gt(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.gt(_, _)) } case class GreaterThanOrEqual(left: Expression, right: Expression) extends BinaryComparison { def symbol = ">=" - override def apply(input: Row): Any = c2(input, left, right, _.gteq(_, _)) + override def eval(input: Row): Any = c2(input, left, right, _.gteq(_, _)) } case class If(predicate: Expression, trueValue: Expression, falseValue: Expression) @@ -159,11 +159,11 @@ case class If(predicate: Expression, trueValue: Expression, falseValue: Expressi } type EvaluatedType = Any - override def apply(input: Row): Any = { - if (predicate(input).asInstanceOf[Boolean]) { - trueValue.apply(input) + override def eval(input: Row): Any = { + if (predicate.eval(input).asInstanceOf[Boolean]) { + trueValue.eval(input) } else { - falseValue.apply(input) + falseValue.eval(input) } } 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 index 42b7a9b125b7a..a27c71db1b999 100644 --- 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 @@ -22,8 +22,6 @@ import java.util.regex.Pattern import org.apache.spark.sql.catalyst.types.DataType import org.apache.spark.sql.catalyst.types.StringType import org.apache.spark.sql.catalyst.types.BooleanType -import org.apache.spark.sql.catalyst.trees.TreeNode -import org.apache.spark.sql.catalyst.errors.`package`.TreeNodeException trait StringRegexExpression { @@ -52,12 +50,12 @@ trait StringRegexExpression { protected def pattern(str: String) = if(cache == null) compile(str) else cache - override def apply(input: Row): Any = { - val l = left.apply(input) - if(l == null) { + override def eval(input: Row): Any = { + val l = left.eval(input) + if (l == null) { null } else { - val r = right.apply(input) + val r = right.eval(input) if(r == null) { null } else { 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 3dd6818029bcf..37b23ba58289c 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 @@ -45,7 +45,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(e.apply(null), e.dataType) + case e if e.foldable => Literal(e.eval(null), e.dataType) } } } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 43876033d327b..92987405aa313 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -29,7 +29,7 @@ import org.apache.spark.sql.catalyst.dsl.expressions._ class ExpressionEvaluationSuite extends FunSuite { test("literals") { - assert((Literal(1) + Literal(1)).apply(null) === 2) + assert((Literal(1) + Literal(1)).eval(null) === 2) } /** @@ -62,7 +62,7 @@ class ExpressionEvaluationSuite extends FunSuite { notTrueTable.foreach { case (v, answer) => val expr = Not(Literal(v, BooleanType)) - val result = expr.apply(null) + val result = expr.eval(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 = expr.apply(null) + val result = expr.eval(null) if (result != answer) fail(s"$expr should not evaluate to $result, expected: $answer") } @@ -113,7 +113,7 @@ class ExpressionEvaluationSuite extends FunSuite { } def evaluate(expression: Expression, inputRow: Row = EmptyRow): Any = { - expression.apply(inputRow) + expression.eval(inputRow) } def checkEvaluation(expression: Expression, expected: Any, inputRow: Row = EmptyRow): Unit = { 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 e902e6ced521d..cff4887936ae1 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 @@ -36,10 +36,10 @@ case class Generate( child: SparkPlan) extends UnaryNode { - def output = + override def output = if (join) child.output ++ generator.output else generator.output - def execute() = { + override def execute() = { if (join) { child.execute().mapPartitions { iter => val nullValues = Seq.fill(generator.output.size)(Literal(null)) @@ -52,7 +52,7 @@ case class Generate( val joinedRow = new JoinedRow iter.flatMap {row => - val outputRows = generator(row) + val outputRows = generator.eval(row) if (outer && outputRows.isEmpty) { outerProjection(row) :: Nil } else { @@ -61,7 +61,7 @@ case class Generate( } } } else { - child.execute().mapPartitions(iter => iter.flatMap(generator)) + child.execute().mapPartitions(iter => iter.flatMap(row => generator.eval(row))) } } } 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 2a4f7b5670457..0890faa33b507 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 @@ -144,7 +144,7 @@ case class Aggregate( var i = 0 while (i < buffer.length) { - aggregateResults(i) = buffer(i).apply(EmptyRow) + aggregateResults(i) = buffer(i).eval(EmptyRow) i += 1 } @@ -190,7 +190,7 @@ case class Aggregate( while (i < currentBuffer.length) { // Evaluating an aggregate buffer returns the result. No row is required since we // already added all rows in the group using update. - aggregateResults(i) = currentBuffer(i).apply(EmptyRow) + aggregateResults(i) = currentBuffer(i).eval(EmptyRow) i += 1 } resultProjection(joinedRow(aggregateResults, currentGroup)) 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 524e5022ee14b..ab2e62463764a 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 @@ -41,7 +41,7 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def output = child.output override def execute() = child.execute().mapPartitions { iter => - iter.filter(condition.apply(_).asInstanceOf[Boolean]) + iter.filter(condition.eval(_).asInstanceOf[Boolean]) } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala index ca5c8b8eb63dc..e55648b8ed15a 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala @@ -39,9 +39,9 @@ case class ExampleTGF(input: Seq[Attribute] = Seq('name, 'age)) extends Generato val Seq(nameAttr, ageAttr) = input - override def apply(input: Row): TraversableOnce[Row] = { - val name = nameAttr.apply(input) - val age = ageAttr.apply(input).asInstanceOf[Int] + override def eval(input: Row): TraversableOnce[Row] = { + val name = nameAttr.eval(input) + val age = ageAttr.eval(input).asInstanceOf[Int] Iterator( new GenericRow(Array[Any](s"$name is $age years old")), 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 e2d9d8de2572a..821fb22112f87 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 @@ -106,7 +106,7 @@ case class HiveTableScan( } private def castFromString(value: String, dataType: DataType) = { - Cast(Literal(value), dataType).apply(null) + Cast(Literal(value), dataType).eval(null) } @transient @@ -134,7 +134,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.apply(row).asInstanceOf[Boolean] + shouldKeep.eval(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 2c607455c8e4b..f9b437d435eba 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 @@ -190,8 +190,8 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } // TODO: Finish input output types. - override def apply(input: Row): Any = { - val evaluatedChildren = children.map(_.apply(input)) + override def eval(input: Row): Any = { + val evaluatedChildren = children.map(_.eval(input)) // Wrap the function arguments in the expected types. val args = evaluatedChildren.zip(wrappers).map { case (arg, wrapper) => wrapper(arg) @@ -216,12 +216,12 @@ case class HiveGenericUdf( val dataType: DataType = inspectorToDataType(returnInspector) - override def apply(input: Row): Any = { + override def eval(input: Row): Any = { returnInspector // Make sure initialized. val args = children.map { v => new DeferredObject { override def prepare(i: Int) = {} - override def get(): AnyRef = wrap(v.apply(input)) + override def get(): AnyRef = wrap(v.eval(input)) } }.toArray unwrap(function.evaluate(args)) @@ -406,7 +406,7 @@ case class HiveGenericUdtf( } } - override def apply(input: Row): TraversableOnce[Row] = { + override def eval(input: Row): TraversableOnce[Row] = { outputInspectors // Make sure initialized. val inputProjection = new Projection(children) @@ -460,7 +460,7 @@ case class HiveUdafFunction( private val buffer = function.getNewAggregationBuffer.asInstanceOf[GenericUDAFEvaluator.AbstractAggregationBuffer] - override def apply(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) + override def eval(input: Row): Any = unwrapData(function.evaluate(buffer), returnInspector) @transient val inputProjection = new Projection(exprs) From 9dd8b9166225cbcfcab1f39268244c2feb42a658 Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Mon, 7 Apr 2014 13:28:14 -0500 Subject: [PATCH 222/397] SPARK-1252. On YARN, use container-log4j.properties for executors container-log4j.properties is a file that YARN provides so that containers can have log4j.properties distinct from that of the NodeManagers. Logs now go to syslog, and stderr and stdout just have the process's standard err and standard out. I tested this on pseudo-distributed clusters for both yarn (Hadoop 2.2) and yarn-alpha (Hadoop 0.23.7)/ Author: Sandy Ryza Closes #148 from sryza/sandy-spark-1252 and squashes the following commits: c0043b8 [Sandy Ryza] Put log4j.properties file under common 55823da [Sandy Ryza] Add license headers to new files 10934b8 [Sandy Ryza] Add log4j-spark-container.properties and support SPARK_LOG4J_CONF e74450b [Sandy Ryza] SPARK-1252. On YARN, use container-log4j.properties for executors --- .../spark/deploy/yarn/ExecutorRunnable.scala | 3 ++- .../log4j-spark-container.properties | 24 +++++++++++++++++++ .../apache/spark/deploy/yarn/ClientBase.scala | 8 +++++-- .../deploy/yarn/ExecutorRunnableUtil.scala | 7 +++++- .../deploy/yarn/YarnSparkHadoopUtil.scala | 7 ++++++ yarn/pom.xml | 6 +++++ .../spark/deploy/yarn/ExecutorRunnable.scala | 3 ++- 7 files changed, 53 insertions(+), 5 deletions(-) create mode 100644 yarn/common/src/main/resources/log4j-spark-container.properties diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 981e8b05f602d..3469b7decedf6 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -81,7 +81,8 @@ class ExecutorRunnable( credentials.writeTokenStorageToStream(dob) ctx.setContainerTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, + localResources.contains(ClientBase.LOG4J_PROP)) logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) diff --git a/yarn/common/src/main/resources/log4j-spark-container.properties b/yarn/common/src/main/resources/log4j-spark-container.properties new file mode 100644 index 0000000000000..a1e37a0be27dd --- /dev/null +++ b/yarn/common/src/main/resources/log4j-spark-container.properties @@ -0,0 +1,24 @@ +# +# 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. See accompanying LICENSE file. + +# Set everything to be logged to the console +log4j.rootCategory=INFO, console +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{1}: %m%n + +# Settings to quiet third party logs that are too verbose +log4j.logger.org.eclipse.jetty=WARN +log4j.logger.org.apache.spark.repl.SparkIMain$exprTyper=INFO +log4j.logger.org.apache.spark.repl.SparkILoop$SparkILoopInterpreter=INFO diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index bc267900fcf1d..eb42922aea228 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -266,11 +266,11 @@ trait ClientBase extends Logging { localResources: HashMap[String, LocalResource], stagingDir: String): HashMap[String, String] = { logInfo("Setting up the launch environment") - val log4jConfLocalRes = localResources.getOrElse(ClientBase.LOG4J_PROP, null) val env = new HashMap[String, String]() - ClientBase.populateClasspath(yarnConf, sparkConf, log4jConfLocalRes != null, env) + ClientBase.populateClasspath(yarnConf, sparkConf, localResources.contains(ClientBase.LOG4J_PROP), + env) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -344,6 +344,10 @@ trait ClientBase extends Logging { JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") } + if (!localResources.contains(ClientBase.LOG4J_PROP)) { + JAVA_OPTS += " " + YarnSparkHadoopUtil.getLoggingArgsForContainerCommandLine() + } + // Command for the ApplicationMaster val commands = List[String]( Environment.JAVA_HOME.$() + "/bin/java" + diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 2079697d8160e..b3696c5fe7183 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -50,7 +50,8 @@ trait ExecutorRunnableUtil extends Logging { slaveId: String, hostname: String, executorMemory: Int, - executorCores: Int) = { + executorCores: Int, + userSpecifiedLogFile: Boolean) = { // Extra options for the JVM var JAVA_OPTS = "" // Set the JVM memory @@ -63,6 +64,10 @@ trait ExecutorRunnableUtil extends Logging { JAVA_OPTS += " -Djava.io.tmpdir=" + new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) + " " + if (!userSpecifiedLogFile) { + JAVA_OPTS += " " + YarnSparkHadoopUtil.getLoggingArgsForContainerCommandLine() + } + // Commenting it out for now - so that people can refer to the properties if required. Remove // it once cpuset version is pushed out. // The context is, default gc for server class machines end up using all cores to do gc - hence diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 4c6e1dcd6dac3..314a7550ada71 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -22,6 +22,7 @@ import org.apache.hadoop.mapred.JobConf import org.apache.hadoop.security.Credentials import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.api.ApplicationConstants import org.apache.hadoop.conf.Configuration import org.apache.spark.deploy.SparkHadoopUtil @@ -67,3 +68,9 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { } } + +object YarnSparkHadoopUtil { + def getLoggingArgsForContainerCommandLine(): String = { + "-Dlog4j.configuration=log4j-spark-container.properties" + } +} diff --git a/yarn/pom.xml b/yarn/pom.xml index 35e31760c1f02..3342cb65edcd1 100644 --- a/yarn/pom.xml +++ b/yarn/pom.xml @@ -167,6 +167,12 @@ target/scala-${scala.binary.version}/classes target/scala-${scala.binary.version}/test-classes + + + + ../common/src/main/resources + + diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala index 53c403f7d0913..81d9d1b5c9280 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnable.scala @@ -78,7 +78,8 @@ class ExecutorRunnable( credentials.writeTokenStorageToStream(dob) ctx.setTokens(ByteBuffer.wrap(dob.getData())) - val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores) + val commands = prepareCommand(masterAddress, slaveId, hostname, executorMemory, executorCores, + localResources.contains(ClientBase.LOG4J_PROP)) logInfo("Setting up executor with commands: " + commands) ctx.setCommands(commands) From 2a2ca48be61ed0d72c4347e1c042a264b94db3e8 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 7 Apr 2014 12:47:27 -0700 Subject: [PATCH 223/397] HOTFIX: Disable actor input stream test. This test makes incorrect assumptions about the behavior of Thread.sleep(). Author: Patrick Wendell Closes #347 from pwendell/stream-tests and squashes the following commits: 10e09e0 [Patrick Wendell] HOTFIX: Disable actor input stream. --- .../scala/org/apache/spark/streaming/InputStreamsSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 7df206241beb6..389b23d4d5e4b 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -144,8 +144,8 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } - - test("actor input stream") { + // TODO: This test makes assumptions about Thread.sleep() and is flaky + ignore("actor input stream") { // Start the server val testServer = new TestServer() val port = testServer.port From 0307db0f55b714930c7ea118d5451190ea8c1a94 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Mon, 7 Apr 2014 13:06:30 -0700 Subject: [PATCH 224/397] SPARK-1099: Introduce local[*] mode to infer number of cores This is the default mode for running spark-shell and pyspark, intended to allow users running spark for the first time to see the performance benefits of using multiple cores, while not breaking backwards compatibility for users who use "local" mode and expect exactly 1 core. Author: Aaron Davidson Closes #182 from aarondav/110 and squashes the following commits: a88294c [Aaron Davidson] Rebased changes for new spark-shell a9f393e [Aaron Davidson] SPARK-1099: Introduce local[*] mode to infer number of cores --- bin/spark-shell | 4 ++-- core/src/main/scala/org/apache/spark/SparkContext.scala | 9 ++++++--- .../spark/SparkContextSchedulerCreationSuite.scala | 8 ++++++++ docs/python-programming-guide.md | 7 ++++--- docs/scala-programming-guide.md | 5 +++-- python/pyspark/shell.py | 2 +- .../main/scala/org/apache/spark/repl/SparkILoop.scala | 2 +- 7 files changed, 25 insertions(+), 12 deletions(-) diff --git a/bin/spark-shell b/bin/spark-shell index 535ee3ccd8269..ea12d256b23a1 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -34,7 +34,7 @@ set -o posix FWDIR="$(cd `dirname $0`/..; pwd)" SPARK_REPL_OPTS="${SPARK_REPL_OPTS:-""}" -DEFAULT_MASTER="local" +DEFAULT_MASTER="local[*]" MASTER=${MASTER:-""} info_log=0 @@ -64,7 +64,7 @@ ${txtbld}OPTIONS${txtrst}: is followed by m for megabytes or g for gigabytes, e.g. "1g". -dm --driver-memory : The memory used by the Spark Shell, the number is followed by m for megabytes or g for gigabytes, e.g. "1g". - -m --master : A full string that describes the Spark Master, defaults to "local" + -m --master : A full string that describes the Spark Master, defaults to "local[*]" e.g. "spark://localhost:7077". --log-conf : Enables logging of the supplied SparkConf as INFO at start of the Spark Context. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 8382dd44f3484..e5ebd350eeced 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1285,8 +1285,8 @@ object SparkContext extends Logging { /** Creates a task scheduler based on a given master URL. Extracted for testing. */ private def createTaskScheduler(sc: SparkContext, master: String): TaskScheduler = { - // Regular expression used for local[N] master format - val LOCAL_N_REGEX = """local\[([0-9]+)\]""".r + // Regular expression used for local[N] and local[*] master formats + val LOCAL_N_REGEX = """local\[([0-9\*]+)\]""".r // Regular expression for local[N, maxRetries], used in tests with failing tasks val LOCAL_N_FAILURES_REGEX = """local\[([0-9]+)\s*,\s*([0-9]+)\]""".r // Regular expression for simulating a Spark cluster of [N, cores, memory] locally @@ -1309,8 +1309,11 @@ object SparkContext extends Logging { scheduler case LOCAL_N_REGEX(threads) => + def localCpuCount = Runtime.getRuntime.availableProcessors() + // local[*] estimates the number of cores on the machine; local[N] uses exactly N threads. + val threadCount = if (threads == "*") localCpuCount else threads.toInt val scheduler = new TaskSchedulerImpl(sc, MAX_LOCAL_TASK_FAILURES, isLocal = true) - val backend = new LocalBackend(scheduler, threads.toInt) + val backend = new LocalBackend(scheduler, threadCount) scheduler.initialize(backend) scheduler diff --git a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala index b543471a5d35b..94fba102865b3 100644 --- a/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala +++ b/core/src/test/scala/org/apache/spark/SparkContextSchedulerCreationSuite.scala @@ -51,6 +51,14 @@ class SparkContextSchedulerCreationSuite } } + test("local-*") { + val sched = createTaskScheduler("local[*]") + sched.backend match { + case s: LocalBackend => assert(s.totalCores === Runtime.getRuntime.availableProcessors()) + case _ => fail() + } + } + test("local-n") { val sched = createTaskScheduler("local[5]") assert(sched.maxTaskFailures === 1) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index c2e5327324898..888631e7025b0 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -82,15 +82,16 @@ The Python shell can be used explore data interactively and is a simple way to l >>> help(pyspark) # Show all pyspark functions {% endhighlight %} -By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on a single core. -To connect to a non-local cluster, or use multiple cores, set the `MASTER` environment variable. +By default, the `bin/pyspark` shell creates SparkContext that runs applications locally on all of +your machine's logical cores. +To connect to a non-local cluster, or to specify a number of cores, set the `MASTER` environment variable. For example, to use the `bin/pyspark` shell with a [standalone Spark cluster](spark-standalone.html): {% highlight bash %} $ MASTER=spark://IP:PORT ./bin/pyspark {% endhighlight %} -Or, to use four cores on the local machine: +Or, to use exactly four cores on the local machine: {% highlight bash %} $ MASTER=local[4] ./bin/pyspark diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 77373890eead7..a07cd2e0a32a2 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -54,7 +54,7 @@ object for more advanced configuration. The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use {% highlight bash %} $ MASTER=local[4] ./bin/spark-shell @@ -74,6 +74,7 @@ The master URL passed to Spark can be in one of the following formats:
    Storage LevelMeaning
    MEMORY_AND_DISK_SER Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of recomputing them - on the fly each time they're needed. Similar to MEMORY_ONLY_SER, but spill partitions that don't fit in memory to disk instead of + recomputing them on the fly each time they're needed.
    OFF_HEAP Store RDD in a serialized format in Tachyon. + This is generally more space-efficient than deserialized objects, especially when using a + fast serializer, but more CPU-intensive to read. + This also significantly reduces the overheads of GC. +
    DISK_ONLY
    spark.worker.cleanup.enabledtrue + Enable periodic cleanup of worker / application directories. Note that this only affects standalone + mode, as YARN works differently. +
    spark.worker.cleanup.interval1800 (30 minutes) + Controls the interval, in seconds, at which the worker cleans up old application work dirs + on the local machine. +
    spark.worker.cleanup.appDataTtl7 * 24 * 3600 (7 days) + The number of seconds to retain application work directories on each worker. This is a Time To Live + and should depend on the amount of available disk space you have. Application logs and jars are + downloaded to each application work dir. Over time, the work dirs can quickly fill up disk space, + especially if you run jobs very frequently. +
    spark.akka.frameSize 10
    Master URLMeaning
    local Run Spark locally with one worker thread (i.e. no parallelism at all).
    local[K] Run Spark locally with K worker threads (ideally, set this to the number of cores on your machine). +
    local[*] Run Spark locally with as many worker threads as logical cores on your machine.
    spark://HOST:PORT Connect to the given Spark standalone cluster master. The port must be whichever one your master is configured to use, which is 7077 by default. @@ -84,7 +85,7 @@ The master URL passed to Spark can be in one of the following formats:
    -If no master URL is specified, the spark shell defaults to "local". +If no master URL is specified, the spark shell defaults to "local[*]". For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details. diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 3d779faf1fa44..35e48276e3cb9 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -29,7 +29,7 @@ # this is the equivalent of ADD_JARS add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None -sc = SparkContext(os.environ.get("MASTER", "local"), "PySparkShell", pyFiles=add_files) +sc = SparkContext(os.environ.get("MASTER", "local[*]"), "PySparkShell", pyFiles=add_files) print """Welcome to ____ __ diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 9b1da195002c2..5a367b6bb79de 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -963,7 +963,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, case Some(m) => m case None => { val prop = System.getenv("MASTER") - if (prop != null) prop else "local" + if (prop != null) prop else "local[*]" } } master From 14c9238aa7173ba663a999ef320d8cffb73306c4 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 18:38:44 -0700 Subject: [PATCH 225/397] [sql] Rename execution/aggregates.scala Aggregate.scala, and added a bunch of private[this] to variables. Author: Reynold Xin Closes #348 from rxin/aggregate and squashes the following commits: f4bc36f [Reynold Xin] Rename execution/aggregates.scala Aggregate.scala, and added a bunch of private[this] to variables. --- .../{aggregates.scala => Aggregate.scala} | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) rename sql/core/src/main/scala/org/apache/spark/sql/execution/{aggregates.scala => Aggregate.scala} (92%) 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/Aggregate.scala similarity index 92% rename from sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala rename to sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 0890faa33b507..3a4f071eebedf 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/Aggregate.scala @@ -56,9 +56,9 @@ case class Aggregate( // HACK: Generators don't correctly preserve their output through serializations so we grab // out child's output attributes statically here. - val childOutput = child.output + private[this] val childOutput = child.output - def output = aggregateExpressions.map(_.toAttribute) + override def output = aggregateExpressions.map(_.toAttribute) /** * An aggregate that needs to be computed for each row in a group. @@ -75,7 +75,7 @@ case class Aggregate( /** A list of aggregates that need to be computed for each group. */ @transient - lazy val computedAggregates = aggregateExpressions.flatMap { agg => + private[this] lazy val computedAggregates = aggregateExpressions.flatMap { agg => agg.collect { case a: AggregateExpression => ComputedAggregate( @@ -87,10 +87,10 @@ case class Aggregate( /** The schema of the result of all aggregate evaluations */ @transient - lazy val computedSchema = computedAggregates.map(_.resultAttribute) + private[this] lazy val computedSchema = computedAggregates.map(_.resultAttribute) /** Creates a new aggregate buffer for a group. */ - def newAggregateBuffer(): Array[AggregateFunction] = { + private[this] def newAggregateBuffer(): Array[AggregateFunction] = { val buffer = new Array[AggregateFunction](computedAggregates.length) var i = 0 while (i < computedAggregates.length) { @@ -102,7 +102,7 @@ case class Aggregate( /** Named attributes used to substitute grouping attributes into the final result. */ @transient - lazy val namedGroups = groupingExpressions.map { + private[this] lazy val namedGroups = groupingExpressions.map { case ne: NamedExpression => ne -> ne.toAttribute case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute } @@ -112,7 +112,7 @@ case class Aggregate( * expression into the final result expression. */ @transient - lazy val resultMap = + private[this] lazy val resultMap = (computedAggregates.map { agg => agg.unbound -> agg.resultAttribute} ++ namedGroups).toMap /** @@ -120,13 +120,13 @@ case class Aggregate( * output rows given a group and the result of all aggregate computations. */ @transient - lazy val resultExpressions = aggregateExpressions.map { agg => + private[this] lazy val resultExpressions = aggregateExpressions.map { agg => agg.transform { case e: Expression if resultMap.contains(e) => resultMap(e) } } - def execute() = attachTree(this, "execute") { + override def execute() = attachTree(this, "execute") { if (groupingExpressions.isEmpty) { child.execute().mapPartitions { iter => val buffer = newAggregateBuffer() From 55dfd5dcdbf3a9bfddb2108c8325bda3100eb33d Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 18:39:18 -0700 Subject: [PATCH 226/397] Removed the default eval implementation from Expression, and added a bunch of override's in classes I touched. It is more robust to not provide a default implementation for Expression's. Author: Reynold Xin Closes #350 from rxin/eval-default and squashes the following commits: 0a83b8f [Reynold Xin] Removed the default eval implementation from Expression, and added a bunch of override's in classes I touched. --- .../sql/catalyst/analysis/unresolved.scala | 52 ++++++++++++------- .../sql/catalyst/expressions/Expression.scala | 3 +- .../sql/catalyst/expressions/SortOrder.scala | 11 +++- .../sql/catalyst/expressions/aggregates.scala | 8 +++ .../expressions/namedExpressions.scala | 21 +++++--- .../plans/physical/partitioning.scala | 32 ++++++++---- .../ExpressionEvaluationSuite.scala | 5 +- .../optimizer/ConstantFoldingSuite.scala | 2 +- 8 files changed, 89 insertions(+), 45 deletions(-) diff --git a/sql/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 index 41e9bcef3cd7f..d629172a7426e 100644 --- a/sql/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 @@ -18,7 +18,8 @@ package org.apache.spark.sql.catalyst.analysis import org.apache.spark.sql.catalyst.{errors, trees} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression} +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.BaseRelation import org.apache.spark.sql.catalyst.trees.TreeNode @@ -36,7 +37,7 @@ case class UnresolvedRelation( databaseName: Option[String], tableName: String, alias: Option[String] = None) extends BaseRelation { - def output = Nil + override def output = Nil override lazy val resolved = false } @@ -44,26 +45,33 @@ case class UnresolvedRelation( * Holds the name of an attribute that has yet to be resolved. */ 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") - def qualifiers = throw new UnresolvedException(this, "qualifiers") + override def exprId = throw new UnresolvedException(this, "exprId") + override def dataType = throw new UnresolvedException(this, "dataType") + override def nullable = throw new UnresolvedException(this, "nullable") + override def qualifiers = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - def newInstance = this - def withQualifiers(newQualifiers: Seq[String]) = this + override def newInstance = this + override def withQualifiers(newQualifiers: Seq[String]) = this + + // Unresolved attributes are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") 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") + override def dataType = throw new UnresolvedException(this, "dataType") 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 + override def nullable = throw new UnresolvedException(this, "nullable") + override def references = children.flatMap(_.references).toSet override lazy val resolved = false + + // Unresolved functions are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + override def toString = s"'$name(${children.mkString(",")})" } @@ -79,15 +87,15 @@ case class Star( 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") - def nullable = throw new UnresolvedException(this, "nullable") - def qualifiers = throw new UnresolvedException(this, "qualifiers") + override def name = throw new UnresolvedException(this, "exprId") + override def exprId = throw new UnresolvedException(this, "exprId") + override def dataType = throw new UnresolvedException(this, "dataType") + override def nullable = throw new UnresolvedException(this, "nullable") + override def qualifiers = throw new UnresolvedException(this, "qualifiers") override lazy val resolved = false - def newInstance = this - def withQualifiers(newQualifiers: Seq[String]) = this + override def newInstance = this + override def withQualifiers(newQualifiers: Seq[String]) = this def expand(input: Seq[Attribute]): Seq[NamedExpression] = { val expandedAttributes: Seq[Attribute] = table match { @@ -104,5 +112,9 @@ case class Star( mappedAttributes } + // Star gets expanded at runtime so we never evaluate a Star. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + override def toString = table.map(_ + ".").getOrElse("") + "*" } 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 f190bd0cca375..8a1db8e796816 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 @@ -50,8 +50,7 @@ abstract class Expression extends TreeNode[Expression] { def references: Set[Attribute] /** Returns the result of evaluating this expression on a given input Row */ - def eval(input: Row = null): EvaluatedType = - throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + def eval(input: Row = null): EvaluatedType /** * Returns `true` if this expression and all its children have been resolved to a specific schema 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 index d5d93778f4b8d..08b2f11d20f5e 100644 --- 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 @@ -17,6 +17,8 @@ package org.apache.spark.sql.catalyst.expressions +import org.apache.spark.sql.catalyst.errors.TreeNodeException + abstract sealed class SortDirection case object Ascending extends SortDirection case object Descending extends SortDirection @@ -26,7 +28,12 @@ case object Descending extends SortDirection * 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 dataType = child.dataType + override def nullable = child.nullable + + // SortOrder itself is never evaluated. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + 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/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala index 5edcea14278c7..b152f95f96c70 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.errors.TreeNodeException abstract class AggregateExpression extends Expression { self: Product => @@ -28,6 +29,13 @@ abstract class AggregateExpression extends Expression { * of input rows/ */ def newInstance(): AggregateFunction + + /** + * [[AggregateExpression.eval]] should never be invoked because [[AggregateExpression]]'s are + * replaced with a physical aggregate operator at runtime. + */ + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } /** 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 eb4bc8e755284..a8145c37c20fa 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.catalyst.expressions import org.apache.spark.sql.catalyst.trees import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute +import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.types._ object NamedExpression { @@ -58,9 +59,9 @@ abstract class Attribute extends NamedExpression { def withQualifiers(newQualifiers: Seq[String]): Attribute - def references = Set(this) def toAttribute = this def newInstance: Attribute + override def references = Set(this) } /** @@ -77,15 +78,15 @@ 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 type EvaluatedType = Any override def eval(input: Row) = child.eval(input) - def dataType = child.dataType - def nullable = child.nullable - def references = child.references + override def dataType = child.dataType + override def nullable = child.nullable + override def references = child.references - def toAttribute = { + override def toAttribute = { if (resolved) { AttributeReference(name, child.dataType, child.nullable)(exprId, qualifiers) } else { @@ -127,7 +128,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea h } - def newInstance = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) + override def newInstance = AttributeReference(name, dataType, nullable)(qualifiers = qualifiers) /** * Returns a copy of this [[AttributeReference]] with changed nullability. @@ -143,7 +144,7 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea /** * Returns a copy of this [[AttributeReference]] with new qualifiers. */ - def withQualifiers(newQualifiers: Seq[String]) = { + override def withQualifiers(newQualifiers: Seq[String]) = { if (newQualifiers == qualifiers) { this } else { @@ -151,5 +152,9 @@ case class AttributeReference(name: String, dataType: DataType, nullable: Boolea } } + // Unresolved attributes are transient at compile time and don't get evaluated during execution. + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") + override def toString: String = s"$name#${exprId.id}$typeSuffix" } diff --git a/sql/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 index 8893744eb2e7a..ffb3a92f8f340 100644 --- a/sql/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 @@ -17,7 +17,8 @@ package org.apache.spark.sql.catalyst.plans.physical -import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder} +import org.apache.spark.sql.catalyst.errors.TreeNodeException +import org.apache.spark.sql.catalyst.expressions.{Expression, Row, SortOrder} import org.apache.spark.sql.catalyst.types.IntegerType /** @@ -139,12 +140,12 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) extends Expression with Partitioning { - def children = expressions - def references = expressions.flatMap(_.references).toSet - def nullable = false - def dataType = IntegerType + override def children = expressions + override def references = expressions.flatMap(_.references).toSet + override def nullable = false + override def dataType = IntegerType - lazy val clusteringSet = expressions.toSet + private[this] lazy val clusteringSet = expressions.toSet override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true @@ -158,6 +159,9 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) case h: HashPartitioning if h == this => true case _ => false } + + override def eval(input: Row = null): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } /** @@ -168,17 +172,20 @@ case class HashPartitioning(expressions: Seq[Expression], numPartitions: Int) * 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. + * + * This class extends expression primarily so that transformations over expression will descend + * into its child. */ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) extends Expression with Partitioning { - def children = ordering - def references = ordering.flatMap(_.references).toSet - def nullable = false - def dataType = IntegerType + override def children = ordering + override def references = ordering.flatMap(_.references).toSet + override def nullable = false + override def dataType = IntegerType - lazy val clusteringSet = ordering.map(_.child).toSet + private[this] lazy val clusteringSet = ordering.map(_.child).toSet override def satisfies(required: Distribution): Boolean = required match { case UnspecifiedDistribution => true @@ -195,4 +202,7 @@ case class RangePartitioning(ordering: Seq[SortOrder], numPartitions: Int) case r: RangePartitioning if r == this => true case _ => false } + + override def eval(input: Row): EvaluatedType = + throw new TreeNodeException(this, s"No function to evaluate expression. type: ${this.nodeName}") } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 92987405aa313..31be6c4ef1b0b 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -100,7 +100,10 @@ class ExpressionEvaluationSuite extends FunSuite { (null, false, null) :: (null, null, null) :: Nil) - def booleanLogicTest(name: String, op: (Expression, Expression) => Expression, truthTable: Seq[(Any, Any, Any)]) { + def booleanLogicTest( + name: String, + op: (Expression, Expression) => Expression, + truthTable: Seq[(Any, Any, Any)]) { test(s"3VL $name") { truthTable.foreach { case (l,r,answer) => 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 2ab14f48ccc8a..20dfba847790c 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 @@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan} import org.apache.spark.sql.catalyst.rules.RuleExecutor -import org.apache.spark.sql.catalyst.types.IntegerType +import org.apache.spark.sql.catalyst.types.{DoubleType, IntegerType} // For implicit conversions import org.apache.spark.sql.catalyst.dsl.plans._ From 31e6fff03730bb915a836d77dcd43d098afd1dbd Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 18:40:08 -0700 Subject: [PATCH 227/397] Added eval for Rand (without any support for user-defined seed). Author: Reynold Xin Closes #349 from rxin/rand and squashes the following commits: fd11322 [Reynold Xin] Added eval for Rand (without any support for user-defined seed). --- .../spark/sql/catalyst/expressions/Rand.scala | 13 ++++++++++--- 1 file changed, 10 insertions(+), 3 deletions(-) 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 index 0bde621602944..38f836f0a1a0e 100644 --- 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 @@ -17,11 +17,18 @@ package org.apache.spark.sql.catalyst.expressions +import java.util.Random import org.apache.spark.sql.catalyst.types.DoubleType + case object Rand extends LeafExpression { - def dataType = DoubleType - def nullable = false - def references = Set.empty + override def dataType = DoubleType + override def nullable = false + override def references = Set.empty + + private[this] lazy val rand = new Random + + override def eval(input: Row = null) = rand.nextDouble().asInstanceOf[EvaluatedType] + override def toString = "RAND()" } From f27e56aa612538188a8550fe72ee20b8b13304d7 Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Mon, 7 Apr 2014 19:28:24 -0700 Subject: [PATCH 228/397] Change timestamp cast semantics. When cast to numeric types, return the unix time in seconds (instead of millis). @marmbrus @chenghao-intel Author: Reynold Xin Closes #352 from rxin/timestamp-cast and squashes the following commits: 18aacd3 [Reynold Xin] Fixed precision for double. 2adb235 [Reynold Xin] Change timestamp cast semantics. When cast to numeric types, return the unix time in seconds (instead of millis). --- .../spark/sql/catalyst/dsl/package.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 23 ++++++++++------ .../ExpressionEvaluationSuite.scala | 27 ++++++++++++++++--- 3 files changed, 40 insertions(+), 12 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 2d62e4cbbce01..987befe8e22ee 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 @@ -104,7 +104,7 @@ package object dsl { implicit class DslSymbol(sym: Symbol) extends ImplicitAttribute { def s = sym.name } // TODO more implicit class for literal? implicit class DslString(val s: String) extends ImplicitOperators { - def expr: Expression = Literal(s) + override def expr: Expression = Literal(s) def attr = analysis.UnresolvedAttribute(s) } 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 89226999ca005..17118499d0c87 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 @@ -87,7 +87,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { private def decimalToTimestamp(d: BigDecimal) = { val seconds = d.longValue() - val bd = (d - seconds) * (1000000000) + val bd = (d - seconds) * 1000000000 val nanos = bd.intValue() // Convert to millis @@ -96,18 +96,23 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { // remaining fractional portion as nanos t.setNanos(nanos) - t } - private def timestampToDouble(t: Timestamp) = (t.getSeconds() + t.getNanos().toDouble / 1000) + // Timestamp to long, converting milliseconds to seconds + private def timestampToLong(ts: Timestamp) = ts.getTime / 1000 + + private def timestampToDouble(ts: Timestamp) = { + // First part is the seconds since the beginning of time, followed by nanosecs. + ts.getTime / 1000 + ts.getNanos.toDouble / 1000000000 + } def castToLong: Any => Any = child.dataType match { case StringType => nullOrCast[String](_, s => try s.toLong catch { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) - case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toLong) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t)) case DecimalType => nullOrCast[BigDecimal](_, _.toLong) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) } @@ -117,7 +122,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) - case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toInt) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toInt) case DecimalType => nullOrCast[BigDecimal](_, _.toInt) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b) } @@ -127,7 +132,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) - case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toShort) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toShort) case DecimalType => nullOrCast[BigDecimal](_, _.toShort) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort } @@ -137,7 +142,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) - case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toByte) + case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toByte) case DecimalType => nullOrCast[BigDecimal](_, _.toByte) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte } @@ -147,7 +152,9 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case _: NumberFormatException => null }) case BooleanType => nullOrCast[Boolean](_, b => if(b) BigDecimal(1) else BigDecimal(0)) - case TimestampType => nullOrCast[Timestamp](_, t => BigDecimal(timestampToDouble(t))) + case TimestampType => + // Note that we lose precision here. + nullOrCast[Timestamp](_, t => BigDecimal(timestampToDouble(t))) case x: NumericType => b => BigDecimal(x.numeric.asInstanceOf[Numeric[Any]].toDouble(b)) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 31be6c4ef1b0b..888a19d79f7e4 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -201,7 +201,7 @@ class ExpressionEvaluationSuite extends FunSuite { val sts = "1970-01-01 00:00:01.0" val ts = Timestamp.valueOf(sts) - + checkEvaluation("abdef" cast StringType, "abdef") checkEvaluation("abdef" cast DecimalType, null) checkEvaluation("abdef" cast TimestampType, null) @@ -209,7 +209,6 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(1) cast LongType, 1) checkEvaluation(Cast(Literal(1) cast TimestampType, LongType), 1) - checkEvaluation(Cast(Literal(BigDecimal(1)) cast TimestampType, DecimalType), 1) checkEvaluation(Cast(Literal(1.toDouble) cast TimestampType, DoubleType), 1.toDouble) checkEvaluation(Cast(Literal(sts) cast TimestampType, StringType), sts) @@ -240,12 +239,34 @@ class ExpressionEvaluationSuite extends FunSuite { intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} } - + test("timestamp") { val ts1 = new Timestamp(12) val ts2 = new Timestamp(123) checkEvaluation(Literal("ab") < Literal("abc"), true) checkEvaluation(Literal(ts1) < Literal(ts2), true) } + + test("timestamp casting") { + val millis = 15 * 1000 + 2 + val ts = new Timestamp(millis) + val ts1 = new Timestamp(15 * 1000) // a timestamp without the milliseconds part + checkEvaluation(Cast(ts, ShortType), 15) + checkEvaluation(Cast(ts, IntegerType), 15) + checkEvaluation(Cast(ts, LongType), 15) + checkEvaluation(Cast(ts, FloatType), 15.002f) + checkEvaluation(Cast(ts, DoubleType), 15.002) + checkEvaluation(Cast(Cast(ts, ShortType), TimestampType), ts1) + checkEvaluation(Cast(Cast(ts, IntegerType), TimestampType), ts1) + checkEvaluation(Cast(Cast(ts, LongType), TimestampType), ts1) + checkEvaluation(Cast(Cast(millis.toFloat / 1000, TimestampType), FloatType), + millis.toFloat / 1000) + checkEvaluation(Cast(Cast(millis.toDouble / 1000, TimestampType), DoubleType), + millis.toDouble / 1000) + checkEvaluation(Cast(Literal(BigDecimal(1)) cast TimestampType, DecimalType), 1) + + // A test for higher precision than millis + checkEvaluation(Cast(Cast(0.00000001, TimestampType), DoubleType), 0.00000001) + } } From 0d0493fcf7fc86d30b0ddd4e2c5a293c5c88eb9d Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 7 Apr 2014 22:24:12 -0700 Subject: [PATCH 229/397] [SPARK-1402] Added 3 more compression schemes JIRA issue: [SPARK-1402](https://issues.apache.org/jira/browse/SPARK-1402) This PR provides 3 more compression schemes for Spark SQL in-memory columnar storage: * `BooleanBitSet` * `IntDelta` * `LongDelta` Now there are 6 compression schemes in total, including the no-op `PassThrough` scheme. Also fixed a bug in PR #286: not all compression schemes are added as available schemes when accessing an in-memory column, and when a column is compressed with an unrecognised scheme, `ColumnAccessor` throws exception. Author: Cheng Lian Closes #330 from liancheng/moreCompressionSchemes and squashes the following commits: 1d037b8 [Cheng Lian] Fixed SPARK-1436: in-memory column byte buffer must be able to be accessed multiple times d7c0e8f [Cheng Lian] Added test suite for IntegralDelta (IntDelta & LongDelta) 3c1ad7a [Cheng Lian] Added test suite for BooleanBitSet, refactored other test suites 44fe4b2 [Cheng Lian] Refactored CompressionScheme, added 3 more compression schemes. --- .../spark/sql/columnar/ColumnAccessor.scala | 23 +- .../spark/sql/columnar/ColumnStats.scala | 6 + .../CompressibleColumnBuilder.scala | 6 +- .../compression/CompressionScheme.scala | 28 +- .../compression/compressionSchemes.scala | 266 +++++++++++++++--- .../sql/columnar/ColumnarQuerySuite.scala | 8 + .../compression/BooleanBitSetSuite.scala | 98 +++++++ .../compression/DictionaryEncodingSuite.scala | 122 ++++---- .../compression/IntegralDeltaSuite.scala | 115 ++++++++ .../compression/RunLengthEncodingSuite.scala | 87 +++--- .../TestCompressibleColumnBuilder.scala | 6 +- 11 files changed, 586 insertions(+), 179 deletions(-) create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala create mode 100644 sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala index ffd4894b5213d..3c39e1d350fa8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala @@ -100,20 +100,21 @@ private[sql] class GenericColumnAccessor(buffer: ByteBuffer) private[sql] object ColumnAccessor { def apply(buffer: ByteBuffer): ColumnAccessor = { + val dup = buffer.duplicate().order(ByteOrder.nativeOrder) // The first 4 bytes in the buffer indicate the column type. - val columnTypeId = buffer.getInt() + val columnTypeId = dup.getInt() columnTypeId match { - case INT.typeId => new IntColumnAccessor(buffer) - case LONG.typeId => new LongColumnAccessor(buffer) - case FLOAT.typeId => new FloatColumnAccessor(buffer) - case DOUBLE.typeId => new DoubleColumnAccessor(buffer) - case BOOLEAN.typeId => new BooleanColumnAccessor(buffer) - case BYTE.typeId => new ByteColumnAccessor(buffer) - case SHORT.typeId => new ShortColumnAccessor(buffer) - case STRING.typeId => new StringColumnAccessor(buffer) - case BINARY.typeId => new BinaryColumnAccessor(buffer) - case GENERIC.typeId => new GenericColumnAccessor(buffer) + case INT.typeId => new IntColumnAccessor(dup) + case LONG.typeId => new LongColumnAccessor(dup) + case FLOAT.typeId => new FloatColumnAccessor(dup) + case DOUBLE.typeId => new DoubleColumnAccessor(dup) + case BOOLEAN.typeId => new BooleanColumnAccessor(dup) + case BYTE.typeId => new ByteColumnAccessor(dup) + case SHORT.typeId => new ShortColumnAccessor(dup) + case STRING.typeId => new StringColumnAccessor(dup) + case BINARY.typeId => new BinaryColumnAccessor(dup) + case GENERIC.typeId => new GenericColumnAccessor(dup) } } } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala index 30c6bdc7912fc..95602d321dc6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnStats.scala @@ -20,6 +20,12 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.Row import org.apache.spark.sql.catalyst.types._ +/** + * Used to collect statistical information when building in-memory columns. + * + * NOTE: we intentionally avoid using `Ordering[T]` to compare values here because `Ordering[T]` + * brings significant performance penalty. + */ private[sql] sealed abstract class ColumnStats[T <: DataType, JvmType] extends Serializable { /** * Closed lower bound of this column. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index 3ac4b358ddf83..fd3b1adf9687a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -47,9 +47,9 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] import CompressionScheme._ - val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder) + val compressionEncoders = schemes.filter(_.supports(columnType)).map(_.encoder[T]) - protected def isWorthCompressing(encoder: Encoder) = { + protected def isWorthCompressing(encoder: Encoder[T]) = { encoder.compressionRatio < 0.8 } @@ -70,7 +70,7 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] abstract override def build() = { val rawBuffer = super.build() - val encoder = { + val encoder: Encoder[T] = { val candidate = compressionEncoders.minBy(_.compressionRatio) if (isWorthCompressing(candidate)) candidate else PassThrough.encoder } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index d3a4ac8df926b..c605a8e4434e3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -22,10 +22,8 @@ import java.nio.ByteBuffer import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} -private[sql] trait Encoder { - def gatherCompressibilityStats[T <: NativeType]( - value: T#JvmType, - columnType: ColumnType[T, T#JvmType]) {} +private[sql] trait Encoder[T <: NativeType] { + def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) {} def compressedSize: Int @@ -35,10 +33,7 @@ private[sql] trait Encoder { if (uncompressedSize > 0) compressedSize.toDouble / uncompressedSize else 1.0 } - def compress[T <: NativeType]( - from: ByteBuffer, - to: ByteBuffer, - columnType: ColumnType[T, T#JvmType]): ByteBuffer + def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]): ByteBuffer } private[sql] trait Decoder[T <: NativeType] extends Iterator[T#JvmType] @@ -48,7 +43,7 @@ private[sql] trait CompressionScheme { def supports(columnType: ColumnType[_, _]): Boolean - def encoder: Encoder + def encoder[T <: NativeType]: Encoder[T] def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]): Decoder[T] } @@ -58,15 +53,18 @@ private[sql] trait WithCompressionSchemes { } private[sql] trait AllCompressionSchemes extends WithCompressionSchemes { - override val schemes: Seq[CompressionScheme] = { - Seq(PassThrough, RunLengthEncoding, DictionaryEncoding) - } + override val schemes: Seq[CompressionScheme] = CompressionScheme.all } private[sql] object CompressionScheme { - def apply(typeId: Int): CompressionScheme = typeId match { - case PassThrough.typeId => PassThrough - case _ => throw new UnsupportedOperationException() + val all: Seq[CompressionScheme] = + Seq(PassThrough, RunLengthEncoding, DictionaryEncoding, BooleanBitSet, IntDelta, LongDelta) + + private val typeIdToScheme = all.map(scheme => scheme.typeId -> scheme).toMap + + def apply(typeId: Int): CompressionScheme = { + typeIdToScheme.getOrElse(typeId, throw new UnsupportedOperationException( + s"Unrecognized compression scheme type ID: $typeId")) } def copyColumnHeader(from: ByteBuffer, to: ByteBuffer) { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index dc2c153faf8ad..df8220b556edd 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -24,7 +24,7 @@ import scala.reflect.ClassTag import scala.reflect.runtime.universe.runtimeMirror import org.apache.spark.sql.catalyst.expressions.GenericMutableRow -import org.apache.spark.sql.catalyst.types.NativeType +import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar._ private[sql] case object PassThrough extends CompressionScheme { @@ -32,22 +32,18 @@ private[sql] case object PassThrough extends CompressionScheme { override def supports(columnType: ColumnType[_, _]) = true - override def encoder = new this.Encoder + override def encoder[T <: NativeType] = new this.Encoder[T] override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { new this.Decoder(buffer, columnType) } - class Encoder extends compression.Encoder { + class Encoder[T <: NativeType] extends compression.Encoder[T] { override def uncompressedSize = 0 override def compressedSize = 0 - override def compress[T <: NativeType]( - from: ByteBuffer, - to: ByteBuffer, - columnType: ColumnType[T, T#JvmType]) = { - + override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { // Writes compression type ID and copies raw contents to.putInt(PassThrough.typeId).put(from).rewind() to @@ -64,9 +60,9 @@ private[sql] case object PassThrough extends CompressionScheme { } private[sql] case object RunLengthEncoding extends CompressionScheme { - override def typeId = 1 + override val typeId = 1 - override def encoder = new this.Encoder + override def encoder[T <: NativeType] = new this.Encoder[T] override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { new this.Decoder(buffer, columnType) @@ -77,7 +73,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { case _ => false } - class Encoder extends compression.Encoder { + class Encoder[T <: NativeType] extends compression.Encoder[T] { private var _uncompressedSize = 0 private var _compressedSize = 0 @@ -89,10 +85,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { override def compressedSize = _compressedSize - override def gatherCompressibilityStats[T <: NativeType]( - value: T#JvmType, - columnType: ColumnType[T, T#JvmType]) { - + override def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) { val actualSize = columnType.actualSize(value) _uncompressedSize += actualSize @@ -111,11 +104,7 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { } } - override def compress[T <: NativeType]( - from: ByteBuffer, - to: ByteBuffer, - columnType: ColumnType[T, T#JvmType]) = { - + override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { to.putInt(RunLengthEncoding.typeId) if (from.hasRemaining) { @@ -172,23 +161,23 @@ private[sql] case object RunLengthEncoding extends CompressionScheme { } private[sql] case object DictionaryEncoding extends CompressionScheme { - override def typeId: Int = 2 + override val typeId = 2 // 32K unique values allowed - private val MAX_DICT_SIZE = Short.MaxValue - 1 + val MAX_DICT_SIZE = Short.MaxValue override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { - new this.Decoder[T](buffer, columnType) + new this.Decoder(buffer, columnType) } - override def encoder = new this.Encoder + override def encoder[T <: NativeType] = new this.Encoder[T] override def supports(columnType: ColumnType[_, _]) = columnType match { case INT | LONG | STRING => true case _ => false } - class Encoder extends compression.Encoder{ + class Encoder[T <: NativeType] extends compression.Encoder[T] { // Size of the input, uncompressed, in bytes. Note that we only count until the dictionary // overflows. private var _uncompressedSize = 0 @@ -201,7 +190,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { private var count = 0 // The reverse mapping of _dictionary, i.e. mapping encoded integer to the value itself. - private var values = new mutable.ArrayBuffer[Any](1024) + private var values = new mutable.ArrayBuffer[T#JvmType](1024) // The dictionary that maps a value to the encoded short integer. private val dictionary = mutable.HashMap.empty[Any, Short] @@ -210,10 +199,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { // to store dictionary element count. private var dictionarySize = 4 - override def gatherCompressibilityStats[T <: NativeType]( - value: T#JvmType, - columnType: ColumnType[T, T#JvmType]) { - + override def gatherCompressibilityStats(value: T#JvmType, columnType: NativeColumnType[T]) { if (!overflow) { val actualSize = columnType.actualSize(value) count += 1 @@ -234,11 +220,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { } } - override def compress[T <: NativeType]( - from: ByteBuffer, - to: ByteBuffer, - columnType: ColumnType[T, T#JvmType]) = { - + override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[T]) = { if (overflow) { throw new IllegalStateException( "Dictionary encoding should not be used because of dictionary overflow.") @@ -249,7 +231,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { var i = 0 while (i < values.length) { - columnType.append(values(i).asInstanceOf[T#JvmType], to) + columnType.append(values(i), to) i += 1 } @@ -286,3 +268,215 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { override def hasNext = buffer.hasRemaining } } + +private[sql] case object BooleanBitSet extends CompressionScheme { + override val typeId = 3 + + val BITS_PER_LONG = 64 + + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder(buffer).asInstanceOf[compression.Decoder[T]] + } + + override def encoder[T <: NativeType] = (new this.Encoder).asInstanceOf[compression.Encoder[T]] + + override def supports(columnType: ColumnType[_, _]) = columnType == BOOLEAN + + class Encoder extends compression.Encoder[BooleanType.type] { + private var _uncompressedSize = 0 + + override def gatherCompressibilityStats( + value: Boolean, + columnType: NativeColumnType[BooleanType.type]) { + + _uncompressedSize += BOOLEAN.defaultSize + } + + override def compress( + from: ByteBuffer, + to: ByteBuffer, + columnType: NativeColumnType[BooleanType.type]) = { + + to.putInt(BooleanBitSet.typeId) + // Total element count (1 byte per Boolean value) + .putInt(from.remaining) + + while (from.remaining >= BITS_PER_LONG) { + var word = 0: Long + var i = 0 + + while (i < BITS_PER_LONG) { + if (BOOLEAN.extract(from)) { + word |= (1: Long) << i + } + i += 1 + } + + to.putLong(word) + } + + if (from.hasRemaining) { + var word = 0: Long + var i = 0 + + while (from.hasRemaining) { + if (BOOLEAN.extract(from)) { + word |= (1: Long) << i + } + i += 1 + } + + to.putLong(word) + } + + to.rewind() + to + } + + override def uncompressedSize = _uncompressedSize + + override def compressedSize = { + val extra = if (_uncompressedSize % BITS_PER_LONG == 0) 0 else 1 + (_uncompressedSize / BITS_PER_LONG + extra) * 8 + 4 + } + } + + class Decoder(buffer: ByteBuffer) extends compression.Decoder[BooleanType.type] { + private val count = buffer.getInt() + + private var currentWord = 0: Long + + private var visited: Int = 0 + + override def next(): Boolean = { + val bit = visited % BITS_PER_LONG + + visited += 1 + if (bit == 0) { + currentWord = buffer.getLong() + } + + ((currentWord >> bit) & 1) != 0 + } + + override def hasNext: Boolean = visited < count + } +} + +private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends CompressionScheme { + override def decoder[T <: NativeType](buffer: ByteBuffer, columnType: NativeColumnType[T]) = { + new this.Decoder(buffer, columnType.asInstanceOf[NativeColumnType[I]]) + .asInstanceOf[compression.Decoder[T]] + } + + override def encoder[T <: NativeType] = (new this.Encoder).asInstanceOf[compression.Encoder[T]] + + /** + * Computes `delta = x - y`, returns `(true, delta)` if `delta` can fit into a single byte, or + * `(false, 0: Byte)` otherwise. + */ + protected def byteSizedDelta(x: I#JvmType, y: I#JvmType): (Boolean, Byte) + + /** + * Simply computes `x + delta` + */ + protected def addDelta(x: I#JvmType, delta: Byte): I#JvmType + + class Encoder extends compression.Encoder[I] { + private var _compressedSize: Int = 0 + + private var _uncompressedSize: Int = 0 + + private var prev: I#JvmType = _ + + private var initial = true + + override def gatherCompressibilityStats(value: I#JvmType, columnType: NativeColumnType[I]) { + _uncompressedSize += columnType.defaultSize + + if (initial) { + initial = false + prev = value + _compressedSize += 1 + columnType.defaultSize + } else { + val (smallEnough, _) = byteSizedDelta(value, prev) + _compressedSize += (if (smallEnough) 1 else 1 + columnType.defaultSize) + } + } + + override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[I]) = { + to.putInt(typeId) + + if (from.hasRemaining) { + val prev = columnType.extract(from) + + to.put(Byte.MinValue) + columnType.append(prev, to) + + while (from.hasRemaining) { + val current = columnType.extract(from) + val (smallEnough, delta) = byteSizedDelta(current, prev) + + if (smallEnough) { + to.put(delta) + } else { + to.put(Byte.MinValue) + columnType.append(current, to) + } + } + } + + to.rewind() + to + } + + override def uncompressedSize = _uncompressedSize + + override def compressedSize = _compressedSize + } + + class Decoder(buffer: ByteBuffer, columnType: NativeColumnType[I]) + extends compression.Decoder[I] { + + private var prev: I#JvmType = _ + + override def next() = { + val delta = buffer.get() + + if (delta > Byte.MinValue) { + addDelta(prev, delta) + } else { + prev = columnType.extract(buffer) + prev + } + } + + override def hasNext = buffer.hasRemaining + } +} + +private[sql] case object IntDelta extends IntegralDelta[IntegerType.type] { + override val typeId = 4 + + override def supports(columnType: ColumnType[_, _]) = columnType == INT + + override protected def addDelta(x: Int, delta: Byte) = x + delta + + override protected def byteSizedDelta(x: Int, y: Int): (Boolean, Byte) = { + val delta = x - y + if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + } +} + +private[sql] case object LongDelta extends IntegralDelta[LongType.type] { + override val typeId = 5 + + override def supports(columnType: ColumnType[_, _]) = columnType == LONG + + override protected def addDelta(x: Long, delta: Byte) = x + delta + + override protected def byteSizedDelta(x: Long, y: Long): (Boolean, Byte) = { + val delta = x - y + if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala index 70b2e851737f8..2ed4cf2170f9d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala @@ -31,4 +31,12 @@ class ColumnarQuerySuite extends QueryTest { checkAnswer(scan, testData.collect().toSeq) } + + test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { + val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + + checkAnswer(scan, testData.collect().toSeq) + checkAnswer(scan, testData.collect().toSeq) + } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala new file mode 100644 index 0000000000000..a754f98f7fbf1 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/BooleanBitSetSuite.scala @@ -0,0 +1,98 @@ +/* + * 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.columnar.compression + +import org.scalatest.FunSuite + +import org.apache.spark.sql.Row +import org.apache.spark.sql.columnar.{BOOLEAN, BooleanColumnStats} +import org.apache.spark.sql.columnar.ColumnarTestUtils._ + +class BooleanBitSetSuite extends FunSuite { + import BooleanBitSet._ + + def skeleton(count: Int) { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(new BooleanColumnStats, BOOLEAN, BooleanBitSet) + val rows = Seq.fill[Row](count)(makeRandomRow(BOOLEAN)) + val values = rows.map(_.head) + + rows.foreach(builder.appendFrom(_, 0)) + val buffer = builder.build() + + // Column type ID + null count + null positions + val headerSize = CompressionScheme.columnHeaderSize(buffer) + + // Compression scheme ID + element count + bitset words + val compressedSize = 4 + 4 + { + val extra = if (count % BITS_PER_LONG == 0) 0 else 1 + (count / BITS_PER_LONG + extra) * 8 + } + + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + // Skips column header + buffer.position(headerSize) + expectResult(BooleanBitSet.typeId, "Wrong compression scheme ID")(buffer.getInt()) + expectResult(count, "Wrong element count")(buffer.getInt()) + + var word = 0: Long + for (i <- 0 until count) { + val bit = i % BITS_PER_LONG + word = if (bit == 0) buffer.getLong() else word + expectResult(values(i), s"Wrong value in compressed buffer, index=$i") { + (word & ((1: Long) << bit)) != 0 + } + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = BooleanBitSet.decoder(buffer, BOOLEAN) + values.foreach(expectResult(_, "Wrong decoded value")(decoder.next())) + assert(!decoder.hasNext) + } + + test(s"$BooleanBitSet: empty") { + skeleton(0) + } + + test(s"$BooleanBitSet: less than 1 word") { + skeleton(BITS_PER_LONG - 1) + } + + test(s"$BooleanBitSet: exactly 1 word") { + skeleton(BITS_PER_LONG) + } + + test(s"$BooleanBitSet: multiple whole words") { + skeleton(BITS_PER_LONG * 2) + } + + test(s"$BooleanBitSet: multiple words and 1 more bit") { + skeleton(BITS_PER_LONG * 2 + 1) + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala index 184691ab5b46a..eab27987e08ea 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/DictionaryEncodingSuite.scala @@ -24,7 +24,6 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar._ import org.apache.spark.sql.columnar.ColumnarTestUtils._ -import org.apache.spark.sql.catalyst.expressions.GenericMutableRow class DictionaryEncodingSuite extends FunSuite { testDictionaryEncoding(new IntColumnStats, INT) @@ -41,73 +40,82 @@ class DictionaryEncodingSuite extends FunSuite { (0 until buffer.getInt()).map(columnType.extract(buffer) -> _.toShort).toMap } - test(s"$DictionaryEncoding with $typeName: simple case") { + def stableDistinct(seq: Seq[Int]): Seq[Int] = if (seq.isEmpty) { + Seq.empty + } else { + seq.head +: seq.tail.filterNot(_ == seq.head) + } + + def skeleton(uniqueValueCount: Int, inputSeq: Seq[Int]) { // ------------- // Tests encoder // ------------- val builder = TestCompressibleColumnBuilder(columnStats, columnType, DictionaryEncoding) - val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) - - builder.initialize(0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(1), 0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(1), 0) - - val buffer = builder.build() - val headerSize = CompressionScheme.columnHeaderSize(buffer) - // 4 extra bytes for dictionary size - val dictionarySize = 4 + values.map(columnType.actualSize).sum - // 4 `Short`s, 2 bytes each - val compressedSize = dictionarySize + 2 * 4 - // 4 extra bytes for compression scheme type ID - expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) - - // Skips column header - buffer.position(headerSize) - expectResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) - - val dictionary = buildDictionary(buffer) - Array[Short](0, 1).foreach { i => - expectResult(i, "Wrong dictionary entry")(dictionary(values(i))) - } - - Array[Short](0, 1, 0, 1).foreach { - expectResult(_, "Wrong column element value")(buffer.getShort()) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, uniqueValueCount) + val dictValues = stableDistinct(inputSeq) + + inputSeq.foreach(i => builder.appendFrom(rows(i), 0)) + + if (dictValues.length > DictionaryEncoding.MAX_DICT_SIZE) { + withClue("Dictionary overflowed, compression should fail") { + intercept[Throwable] { + builder.build() + } + } + } else { + val buffer = builder.build() + val headerSize = CompressionScheme.columnHeaderSize(buffer) + // 4 extra bytes for dictionary size + val dictionarySize = 4 + values.map(columnType.actualSize).sum + // 2 bytes for each `Short` + val compressedSize = 4 + dictionarySize + 2 * inputSeq.length + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + // Skips column header + buffer.position(headerSize) + expectResult(DictionaryEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + val dictionary = buildDictionary(buffer).toMap + + dictValues.foreach { i => + expectResult(i, "Wrong dictionary entry") { + dictionary(values(i)) + } + } + + inputSeq.foreach { i => + expectResult(i.toShort, "Wrong column element value")(buffer.getShort()) + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = DictionaryEncoding.decoder(buffer, columnType) + + inputSeq.foreach { i => + expectResult(values(i), "Wrong decoded value")(decoder.next()) + } + + assert(!decoder.hasNext) } - - // ------------- - // Tests decoder - // ------------- - - // Rewinds, skips column header and 4 more bytes for compression scheme ID - buffer.rewind().position(headerSize + 4) - - val decoder = new DictionaryEncoding.Decoder[T](buffer, columnType) - - Array[Short](0, 1, 0, 1).foreach { i => - expectResult(values(i), "Wrong decoded value")(decoder.next()) - } - - assert(!decoder.hasNext) } - } - test(s"$DictionaryEncoding: overflow") { - val builder = TestCompressibleColumnBuilder(new IntColumnStats, INT, DictionaryEncoding) - builder.initialize(0) + test(s"$DictionaryEncoding with $typeName: empty") { + skeleton(0, Seq.empty) + } - (0 to Short.MaxValue).foreach { n => - val row = new GenericMutableRow(1) - row.setInt(0, n) - builder.appendFrom(row, 0) + test(s"$DictionaryEncoding with $typeName: simple case") { + skeleton(2, Seq(0, 1, 0, 1)) } - withClue("Dictionary overflowed, encoding should fail") { - intercept[Throwable] { - builder.build() - } + test(s"$DictionaryEncoding with $typeName: dictionary overflow") { + skeleton(DictionaryEncoding.MAX_DICT_SIZE + 1, 0 to DictionaryEncoding.MAX_DICT_SIZE) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala new file mode 100644 index 0000000000000..1390e5eef6106 --- /dev/null +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -0,0 +1,115 @@ +/* + * 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.columnar.compression + +import org.scalatest.FunSuite + +import org.apache.spark.sql.catalyst.expressions.GenericMutableRow +import org.apache.spark.sql.catalyst.types.IntegralType +import org.apache.spark.sql.columnar._ + +class IntegralDeltaSuite extends FunSuite { + testIntegralDelta(new IntColumnStats, INT, IntDelta) + testIntegralDelta(new LongColumnStats, LONG, LongDelta) + + def testIntegralDelta[I <: IntegralType]( + columnStats: NativeColumnStats[I], + columnType: NativeColumnType[I], + scheme: IntegralDelta[I]) { + + def skeleton(input: Seq[I#JvmType]) { + // ------------- + // Tests encoder + // ------------- + + val builder = TestCompressibleColumnBuilder(columnStats, columnType, scheme) + val deltas = if (input.isEmpty) { + Seq.empty[Long] + } else { + (input.tail, input.init).zipped.map { + case (x: Int, y: Int) => (x - y).toLong + case (x: Long, y: Long) => x - y + } + } + + input.map { value => + val row = new GenericMutableRow(1) + columnType.setField(row, 0, value) + builder.appendFrom(row, 0) + } + + val buffer = builder.build() + // Column type ID + null count + null positions + val headerSize = CompressionScheme.columnHeaderSize(buffer) + + // Compression scheme ID + compressed contents + val compressedSize = 4 + (if (deltas.isEmpty) { + 0 + } else { + val oneBoolean = columnType.defaultSize + 1 + oneBoolean + deltas.map { + d => if (math.abs(d) < Byte.MaxValue) 1 else 1 + oneBoolean + }.sum + }) + + // 4 extra bytes for compression scheme type ID + expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) + + buffer.position(headerSize) + expectResult(scheme.typeId, "Wrong compression scheme ID")(buffer.getInt()) + + if (input.nonEmpty) { + expectResult(Byte.MinValue, "The first byte should be an escaping mark")(buffer.get()) + expectResult(input.head, "The first value is wrong")(columnType.extract(buffer)) + + (input.tail, deltas).zipped.foreach { (value, delta) => + if (delta < Byte.MaxValue) { + expectResult(delta, "Wrong delta")(buffer.get()) + } else { + expectResult(Byte.MinValue, "Expecting escaping mark here")(buffer.get()) + expectResult(value, "Wrong value")(columnType.extract(buffer)) + } + } + } + + // ------------- + // Tests decoder + // ------------- + + // Rewinds, skips column header and 4 more bytes for compression scheme ID + buffer.rewind().position(headerSize + 4) + + val decoder = scheme.decoder(buffer, columnType) + input.foreach(expectResult(_, "Wrong decoded value")(decoder.next())) + assert(!decoder.hasNext) + } + + test(s"$scheme: empty column") { + skeleton(Seq.empty) + } + + test(s"$scheme: simple case") { + val input = columnType match { + case INT => Seq(1: Int, 2: Int, 130: Int) + case LONG => Seq(1: Long, 2: Long, 130: Long) + } + + skeleton(input.map(_.asInstanceOf[I#JvmType])) + } + } +} diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala index 2089ad120d4f2..89f9b60a4397b 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/RunLengthEncodingSuite.scala @@ -37,34 +37,39 @@ class RunLengthEncodingSuite extends FunSuite { val typeName = columnType.getClass.getSimpleName.stripSuffix("$") - test(s"$RunLengthEncoding with $typeName: simple case") { + def skeleton(uniqueValueCount: Int, inputRuns: Seq[(Int, Int)]) { // ------------- // Tests encoder // ------------- val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) - val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) - - builder.initialize(0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(1), 0) - builder.appendFrom(rows(1), 0) + val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, uniqueValueCount) + val inputSeq = inputRuns.flatMap { case (index, run) => + Seq.fill(run)(index) + } + inputSeq.foreach(i => builder.appendFrom(rows(i), 0)) val buffer = builder.build() + + // Column type ID + null count + null positions val headerSize = CompressionScheme.columnHeaderSize(buffer) - // 4 extra bytes each run for run length - val compressedSize = values.map(columnType.actualSize(_) + 4).sum + + // Compression scheme ID + compressed contents + val compressedSize = 4 + inputRuns.map { case (index, _) => + // 4 extra bytes each run for run length + columnType.actualSize(values(index)) + 4 + }.sum + // 4 extra bytes for compression scheme type ID - expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) + expectResult(headerSize + compressedSize, "Wrong buffer capacity")(buffer.capacity) // Skips column header buffer.position(headerSize) expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) - Array(0, 1).foreach { i => - expectResult(values(i), "Wrong column element value")(columnType.extract(buffer)) - expectResult(2, "Wrong run length")(buffer.getInt()) + inputRuns.foreach { case (index, run) => + expectResult(values(index), "Wrong column element value")(columnType.extract(buffer)) + expectResult(run, "Wrong run length")(buffer.getInt()) } // ------------- @@ -74,57 +79,29 @@ class RunLengthEncodingSuite extends FunSuite { // Rewinds, skips column header and 4 more bytes for compression scheme ID buffer.rewind().position(headerSize + 4) - val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) + val decoder = RunLengthEncoding.decoder(buffer, columnType) - Array(0, 0, 1, 1).foreach { i => + inputSeq.foreach { i => expectResult(values(i), "Wrong decoded value")(decoder.next()) } assert(!decoder.hasNext) } - test(s"$RunLengthEncoding with $typeName: run length == 1") { - // ------------- - // Tests encoder - // ------------- - - val builder = TestCompressibleColumnBuilder(columnStats, columnType, RunLengthEncoding) - val (values, rows) = makeUniqueValuesAndSingleValueRows(columnType, 2) - - builder.initialize(0) - builder.appendFrom(rows(0), 0) - builder.appendFrom(rows(1), 0) - - val buffer = builder.build() - val headerSize = CompressionScheme.columnHeaderSize(buffer) - // 4 bytes each run for run length - val compressedSize = values.map(columnType.actualSize(_) + 4).sum - // 4 bytes for compression scheme type ID - expectResult(headerSize + 4 + compressedSize, "Wrong buffer capacity")(buffer.capacity) - - // Skips column header - buffer.position(headerSize) - expectResult(RunLengthEncoding.typeId, "Wrong compression scheme ID")(buffer.getInt()) - - Array(0, 1).foreach { i => - expectResult(values(i), "Wrong column element value")(columnType.extract(buffer)) - expectResult(1, "Wrong run length")(buffer.getInt()) - } - - // ------------- - // Tests decoder - // ------------- - - // Rewinds, skips column header and 4 more bytes for compression scheme ID - buffer.rewind().position(headerSize + 4) + test(s"$RunLengthEncoding with $typeName: empty column") { + skeleton(0, Seq.empty) + } - val decoder = new RunLengthEncoding.Decoder[T](buffer, columnType) + test(s"$RunLengthEncoding with $typeName: simple case") { + skeleton(2, Seq(0 -> 2, 1 ->2)) + } - Array(0, 1).foreach { i => - expectResult(values(i), "Wrong decoded value")(decoder.next()) - } + test(s"$RunLengthEncoding with $typeName: run length == 1") { + skeleton(2, Seq(0 -> 1, 1 ->1)) + } - assert(!decoder.hasNext) + test(s"$RunLengthEncoding with $typeName: single long run") { + skeleton(1, Seq(0 -> 1000)) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala index e0ec812863dcf..81bf5e99d19b9 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/TestCompressibleColumnBuilder.scala @@ -28,7 +28,7 @@ class TestCompressibleColumnBuilder[T <: NativeType]( with NullableColumnBuilder with CompressibleColumnBuilder[T] { - override protected def isWorthCompressing(encoder: Encoder) = true + override protected def isWorthCompressing(encoder: Encoder[T]) = true } object TestCompressibleColumnBuilder { @@ -37,7 +37,9 @@ object TestCompressibleColumnBuilder { columnType: NativeColumnType[T], scheme: CompressionScheme) = { - new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) + val builder = new TestCompressibleColumnBuilder(columnStats, columnType, Seq(scheme)) + builder.initialize(0) + builder } } From 11eabbe125b2ee572fad359c33c93f5e6fdf0b2d Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 7 Apr 2014 23:40:21 -0700 Subject: [PATCH 230/397] [SPARK-1103] Automatic garbage collection of RDD, shuffle and broadcast data MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This PR allows Spark to automatically cleanup metadata and data related to persisted RDDs, shuffles and broadcast variables when the corresponding RDDs, shuffles and broadcast variables fall out of scope from the driver program. This is still a work in progress as broadcast cleanup has not been implemented. **Implementation Details** A new class `ContextCleaner` is responsible cleaning all the state. It is instantiated as part of a `SparkContext`. RDD and ShuffleDependency classes have overridden `finalize()` function that gets called whenever their instances go out of scope. The `finalize()` function enqueues the object’s identifier (i.e. RDD ID, shuffle ID, etc.) with the `ContextCleaner`, which is a very short and cheap operation and should not significantly affect the garbage collection mechanism. The `ContextCleaner`, on a different thread, performs the cleanup, whose details are given below. *RDD cleanup:* `ContextCleaner` calls `RDD.unpersist()` is used to cleanup persisted RDDs. Regarding metadata, the DAGScheduler automatically cleans up all metadata related to a RDD after all jobs have completed. Only the `SparkContext.persistentRDDs` keeps strong references to persisted RDDs. The `TimeStampedHashMap` used for that has been replaced by `TimeStampedWeakValueHashMap` that keeps only weak references to the RDDs, allowing them to be garbage collected. *Shuffle cleanup:* New BlockManager message `RemoveShuffle()` asks the `BlockManagerMaster` and currently active `BlockManager`s to delete all the disk blocks related to the shuffle ID. `ContextCleaner` cleans up shuffle data using this message and also cleans up the metadata in the `MapOutputTracker` of the driver. The `MapOutputTracker` at the workers, that caches the shuffle metadata, maintains a `BoundedHashMap` to limit the shuffle information it caches. Refetching the shuffle information from the driver is not too costly. *Broadcast cleanup:* To be done. [This PR](https://github.com/apache/incubator-spark/pull/543/) adds mechanism for explicit cleanup of broadcast variables. `Broadcast.finalize()` will enqueue its own ID with ContextCleaner and the PRs mechanism will be used to unpersist the Broadcast data. *Other cleanup:* `ShuffleMapTask` and `ResultTask` caches tasks and used TTL based cleanup (using `TimeStampedHashMap`), so nothing got cleaned up if TTL was not set. Instead, they now use `BoundedHashMap` to keep a limited number of map output information. Cost of repopulating the cache if necessary is very small. **Current state of implementation** Implemented RDD and shuffle cleanup. Things left to be done are. - Cleaning up for broadcast variable still to be done. - Automatic cleaning up keys with empty weak refs as values in `TimeStampedWeakValueHashMap` Author: Tathagata Das Author: Andrew Or Author: Roman Pastukhov Closes #126 from tdas/state-cleanup and squashes the following commits: 61b8d6e [Tathagata Das] Fixed issue with Tachyon + new BlockManager methods. f489fdc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup d25a86e [Tathagata Das] Fixed stupid typo. cff023c [Tathagata Das] Fixed issues based on Andrew's comments. 4d05314 [Tathagata Das] Scala style fix. 2b95b5e [Tathagata Das] Added more documentation on Broadcast implementations, specially which blocks are told about to the driver. Also, fixed Broadcast API to hide destroy functionality. 41c9ece [Tathagata Das] Added more unit tests for BlockManager, DiskBlockManager, and ContextCleaner. 6222697 [Tathagata Das] Fixed bug and adding unit test for removeBroadcast in BlockManagerSuite. 104a89a [Tathagata Das] Fixed failing BroadcastSuite unit tests by introducing blocking for removeShuffle and removeBroadcast in BlockManager* a430f06 [Tathagata Das] Fixed compilation errors. b27f8e8 [Tathagata Das] Merge pull request #3 from andrewor14/cleanup cd72d19 [Andrew Or] Make automatic cleanup configurable (not documented) ada45f0 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup a2cc8bc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup c5b1d98 [Andrew Or] Address Patrick's comments a6460d4 [Andrew Or] Merge github.com:apache/spark into cleanup 762a4d8 [Tathagata Das] Merge pull request #1 from andrewor14/cleanup f0aabb1 [Andrew Or] Correct semantics for TimeStampedWeakValueHashMap + add tests 5016375 [Andrew Or] Address TD's comments 7ed72fb [Andrew Or] Fix style test fail + remove verbose test message regarding broadcast 634a097 [Andrew Or] Merge branch 'state-cleanup' of github.com:tdas/spark into cleanup 7edbc98 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into state-cleanup 8557c12 [Andrew Or] Merge github.com:apache/spark into cleanup e442246 [Andrew Or] Merge github.com:apache/spark into cleanup 88904a3 [Andrew Or] Make TimeStampedWeakValueHashMap a wrapper of TimeStampedHashMap fbfeec8 [Andrew Or] Add functionality to query executors for their local BlockStatuses 34f436f [Andrew Or] Generalize BroadcastBlockId to remove BroadcastHelperBlockId 0d17060 [Andrew Or] Import, comments, and style fixes (minor) c92e4d9 [Andrew Or] Merge github.com:apache/spark into cleanup f201a8d [Andrew Or] Test broadcast cleanup in ContextCleanerSuite + remove BoundedHashMap e95479c [Andrew Or] Add tests for unpersisting broadcast 544ac86 [Andrew Or] Clean up broadcast blocks through BlockManager* d0edef3 [Andrew Or] Add framework for broadcast cleanup ba52e00 [Andrew Or] Refactor broadcast classes c7ccef1 [Andrew Or] Merge branch 'bc-unpersist-merge' of github.com:ignatich/incubator-spark into cleanup 6c9dcf6 [Tathagata Das] Added missing Apache license d2f8b97 [Tathagata Das] Removed duplicate unpersistRDD. a007307 [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup 620eca3 [Tathagata Das] Changes based on PR comments. f2881fd [Tathagata Das] Changed ContextCleaner to use ReferenceQueue instead of finalizer e1fba5f [Tathagata Das] Style fix 892b952 [Tathagata Das] Removed use of BoundedHashMap, and made BlockManagerSlaveActor cleanup shuffle metadata in MapOutputTrackerWorker. a7260d3 [Tathagata Das] Added try-catch in context cleaner and null value cleaning in TimeStampedWeakValueHashMap. e61daa0 [Tathagata Das] Modifications based on the comments on PR 126. ae9da88 [Tathagata Das] Removed unncessary TimeStampedHashMap from DAGScheduler, added try-catches in finalize() methods, and replaced ArrayBlockingQueue to LinkedBlockingQueue to avoid blocking in Java's finalizing thread. cb0a5a6 [Tathagata Das] Fixed docs and styles. a24fefc [Tathagata Das] Merge remote-tracking branch 'apache/master' into state-cleanup 8512612 [Tathagata Das] Changed TimeStampedHashMap to use WrappedJavaHashMap. e427a9e [Tathagata Das] Added ContextCleaner to automatically clean RDDs and shuffles when they fall out of scope. Also replaced TimeStampedHashMap to BoundedHashMaps and TimeStampedWeakValueHashMap for the necessary hashmap behavior. 80dd977 [Roman Pastukhov] Fix for Broadcast unpersist patch. 1e752f1 [Roman Pastukhov] Added unpersist method to Broadcast. --- .../org/apache/spark/ContextCleaner.scala | 192 ++++++++ .../scala/org/apache/spark/Dependency.scala | 2 + .../org/apache/spark/MapOutputTracker.scala | 148 ++++--- .../scala/org/apache/spark/SparkContext.scala | 23 +- .../scala/org/apache/spark/SparkEnv.scala | 25 +- .../apache/spark/broadcast/Broadcast.scala | 107 +++-- .../spark/broadcast/BroadcastFactory.scala | 3 +- .../spark/broadcast/BroadcastManager.scala | 66 +++ .../spark/broadcast/HttpBroadcast.scala | 128 ++++-- .../broadcast/HttpBroadcastFactory.scala | 45 ++ .../spark/broadcast/TorrentBroadcast.scala | 162 ++++--- .../broadcast/TorrentBroadcastFactory.scala | 46 ++ .../spark/network/ConnectionManager.scala | 1 - .../main/scala/org/apache/spark/rdd/RDD.scala | 5 +- .../apache/spark/scheduler/DAGScheduler.scala | 38 +- .../apache/spark/scheduler/ResultTask.scala | 16 +- .../spark/scheduler/ShuffleMapTask.scala | 14 +- .../spark/scheduler/TaskSchedulerImpl.scala | 2 +- .../org/apache/spark/storage/BlockId.scala | 24 +- .../apache/spark/storage/BlockManager.scala | 67 ++- .../spark/storage/BlockManagerMaster.scala | 84 +++- .../storage/BlockManagerMasterActor.scala | 107 ++++- .../spark/storage/BlockManagerMessages.scala | 20 +- .../storage/BlockManagerSlaveActor.scala | 60 ++- .../spark/storage/DiskBlockManager.scala | 14 + .../spark/storage/ShuffleBlockManager.scala | 44 +- .../apache/spark/storage/ThreadingTest.scala | 6 +- .../apache/spark/util/MetadataCleaner.scala | 19 +- .../spark/util/TimeStampedHashMap.scala | 109 ++--- .../util/TimeStampedWeakValueHashMap.scala | 170 +++++++ .../scala/org/apache/spark/util/Utils.scala | 8 +- .../org/apache/spark/AkkaUtilsSuite.scala | 8 +- .../org/apache/spark/BroadcastSuite.scala | 311 +++++++++++-- .../apache/spark/ContextCleanerSuite.scala | 415 ++++++++++++++++++ .../apache/spark/MapOutputTrackerSuite.scala | 25 +- .../spark/storage/BlockManagerSuite.scala | 243 ++++++++-- .../spark/storage/DiskBlockManagerSuite.scala | 10 +- .../apache/spark/util/JsonProtocolSuite.scala | 5 +- .../spark/util/TimeStampedHashMapSuite.scala | 264 +++++++++++ .../spark/streaming/dstream/DStream.scala | 4 +- 40 files changed, 2571 insertions(+), 469 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/ContextCleaner.scala create mode 100644 core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala create mode 100644 core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala create mode 100644 core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala create mode 100644 core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala create mode 100644 core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala create mode 100644 core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala diff --git a/core/src/main/scala/org/apache/spark/ContextCleaner.scala b/core/src/main/scala/org/apache/spark/ContextCleaner.scala new file mode 100644 index 0000000000000..54e08d7866f75 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ContextCleaner.scala @@ -0,0 +1,192 @@ +/* + * 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 + +import java.lang.ref.{ReferenceQueue, WeakReference} + +import scala.collection.mutable.{ArrayBuffer, SynchronizedBuffer} + +import org.apache.spark.broadcast.Broadcast +import org.apache.spark.rdd.RDD + +/** + * Classes that represent cleaning tasks. + */ +private sealed trait CleanupTask +private case class CleanRDD(rddId: Int) extends CleanupTask +private case class CleanShuffle(shuffleId: Int) extends CleanupTask +private case class CleanBroadcast(broadcastId: Long) extends CleanupTask + +/** + * A WeakReference associated with a CleanupTask. + * + * When the referent object becomes only weakly reachable, the corresponding + * CleanupTaskWeakReference is automatically added to the given reference queue. + */ +private class CleanupTaskWeakReference( + val task: CleanupTask, + referent: AnyRef, + referenceQueue: ReferenceQueue[AnyRef]) + extends WeakReference(referent, referenceQueue) + +/** + * An asynchronous cleaner for RDD, shuffle, and broadcast state. + * + * This maintains a weak reference for each RDD, ShuffleDependency, and Broadcast of interest, + * to be processed when the associated object goes out of scope of the application. Actual + * cleanup is performed in a separate daemon thread. + */ +private[spark] class ContextCleaner(sc: SparkContext) extends Logging { + + private val referenceBuffer = new ArrayBuffer[CleanupTaskWeakReference] + with SynchronizedBuffer[CleanupTaskWeakReference] + + private val referenceQueue = new ReferenceQueue[AnyRef] + + private val listeners = new ArrayBuffer[CleanerListener] + with SynchronizedBuffer[CleanerListener] + + private val cleaningThread = new Thread() { override def run() { keepCleaning() }} + + /** + * Whether the cleaning thread will block on cleanup tasks. + * This is set to true only for tests. + */ + private val blockOnCleanupTasks = sc.conf.getBoolean( + "spark.cleaner.referenceTracking.blocking", false) + + @volatile private var stopped = false + + /** Attach a listener object to get information of when objects are cleaned. */ + def attachListener(listener: CleanerListener) { + listeners += listener + } + + /** Start the cleaner. */ + def start() { + cleaningThread.setDaemon(true) + cleaningThread.setName("Spark Context Cleaner") + cleaningThread.start() + } + + /** Stop the cleaner. */ + def stop() { + stopped = true + } + + /** Register a RDD for cleanup when it is garbage collected. */ + def registerRDDForCleanup(rdd: RDD[_]) { + registerForCleanup(rdd, CleanRDD(rdd.id)) + } + + /** Register a ShuffleDependency for cleanup when it is garbage collected. */ + def registerShuffleForCleanup(shuffleDependency: ShuffleDependency[_, _]) { + registerForCleanup(shuffleDependency, CleanShuffle(shuffleDependency.shuffleId)) + } + + /** Register a Broadcast for cleanup when it is garbage collected. */ + def registerBroadcastForCleanup[T](broadcast: Broadcast[T]) { + registerForCleanup(broadcast, CleanBroadcast(broadcast.id)) + } + + /** Register an object for cleanup. */ + private def registerForCleanup(objectForCleanup: AnyRef, task: CleanupTask) { + referenceBuffer += new CleanupTaskWeakReference(task, objectForCleanup, referenceQueue) + } + + /** Keep cleaning RDD, shuffle, and broadcast state. */ + private def keepCleaning() { + while (!stopped) { + try { + val reference = Option(referenceQueue.remove(ContextCleaner.REF_QUEUE_POLL_TIMEOUT)) + .map(_.asInstanceOf[CleanupTaskWeakReference]) + reference.map(_.task).foreach { task => + logDebug("Got cleaning task " + task) + referenceBuffer -= reference.get + task match { + case CleanRDD(rddId) => + doCleanupRDD(rddId, blocking = blockOnCleanupTasks) + case CleanShuffle(shuffleId) => + doCleanupShuffle(shuffleId, blocking = blockOnCleanupTasks) + case CleanBroadcast(broadcastId) => + doCleanupBroadcast(broadcastId, blocking = blockOnCleanupTasks) + } + } + } catch { + case t: Throwable => logError("Error in cleaning thread", t) + } + } + } + + /** Perform RDD cleanup. */ + def doCleanupRDD(rddId: Int, blocking: Boolean) { + try { + logDebug("Cleaning RDD " + rddId) + sc.unpersistRDD(rddId, blocking) + listeners.foreach(_.rddCleaned(rddId)) + logInfo("Cleaned RDD " + rddId) + } catch { + case t: Throwable => logError("Error cleaning RDD " + rddId, t) + } + } + + /** Perform shuffle cleanup, asynchronously. */ + def doCleanupShuffle(shuffleId: Int, blocking: Boolean) { + try { + logDebug("Cleaning shuffle " + shuffleId) + mapOutputTrackerMaster.unregisterShuffle(shuffleId) + blockManagerMaster.removeShuffle(shuffleId, blocking) + listeners.foreach(_.shuffleCleaned(shuffleId)) + logInfo("Cleaned shuffle " + shuffleId) + } catch { + case t: Throwable => logError("Error cleaning shuffle " + shuffleId, t) + } + } + + /** Perform broadcast cleanup. */ + def doCleanupBroadcast(broadcastId: Long, blocking: Boolean) { + try { + logDebug("Cleaning broadcast " + broadcastId) + broadcastManager.unbroadcast(broadcastId, true, blocking) + listeners.foreach(_.broadcastCleaned(broadcastId)) + logInfo("Cleaned broadcast " + broadcastId) + } catch { + case t: Throwable => logError("Error cleaning broadcast " + broadcastId, t) + } + } + + private def blockManagerMaster = sc.env.blockManager.master + private def broadcastManager = sc.env.broadcastManager + private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] + + // Used for testing. These methods explicitly blocks until cleanup is completed + // to ensure that more reliable testing. +} + +private object ContextCleaner { + private val REF_QUEUE_POLL_TIMEOUT = 100 +} + +/** + * Listener class used for testing when any item has been cleaned by the Cleaner class. + */ +private[spark] trait CleanerListener { + def rddCleaned(rddId: Int) + def shuffleCleaned(shuffleId: Int) + def broadcastCleaned(broadcastId: Long) +} diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 3132dcf745e19..1cd629c15bd46 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -55,6 +55,8 @@ class ShuffleDependency[K, V]( extends Dependency(rdd.asInstanceOf[RDD[Product2[K, V]]]) { val shuffleId: Int = rdd.context.newShuffleId() + + rdd.sparkContext.cleaner.foreach(_.registerShuffleForCleanup(this)) } diff --git a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala index 80cbf951cb70e..ee82d9fa7874b 100644 --- a/core/src/main/scala/org/apache/spark/MapOutputTracker.scala +++ b/core/src/main/scala/org/apache/spark/MapOutputTracker.scala @@ -20,21 +20,21 @@ package org.apache.spark import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} -import scala.collection.mutable.HashSet +import scala.collection.mutable.{HashSet, HashMap, Map} import scala.concurrent.Await import akka.actor._ import akka.pattern.ask - import org.apache.spark.scheduler.MapStatus import org.apache.spark.storage.BlockManagerId -import org.apache.spark.util.{AkkaUtils, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.util._ private[spark] sealed trait MapOutputTrackerMessage private[spark] case class GetMapOutputStatuses(shuffleId: Int) extends MapOutputTrackerMessage private[spark] case object StopMapOutputTracker extends MapOutputTrackerMessage +/** Actor class for MapOutputTrackerMaster */ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster, conf: SparkConf) extends Actor with Logging { val maxAkkaFrameSize = AkkaUtils.maxFrameSizeBytes(conf) @@ -65,26 +65,41 @@ private[spark] class MapOutputTrackerMasterActor(tracker: MapOutputTrackerMaster } } -private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { - +/** + * Class that keeps track of the location of the map output of + * a stage. This is abstract because different versions of MapOutputTracker + * (driver and worker) use different HashMap to store its metadata. + */ +private[spark] abstract class MapOutputTracker(conf: SparkConf) extends Logging { private val timeout = AkkaUtils.askTimeout(conf) - // Set to the MapOutputTrackerActor living on the driver + /** Set to the MapOutputTrackerActor living on the driver. */ var trackerActor: ActorRef = _ - protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]] + /** + * This HashMap has different behavior for the master and the workers. + * + * On the master, it serves as the source of map outputs recorded from ShuffleMapTasks. + * On the workers, it simply serves as a cache, in which a miss triggers a fetch from the + * master's corresponding HashMap. + */ + protected val mapStatuses: Map[Int, Array[MapStatus]] - // Incremented every time a fetch fails so that client nodes know to clear - // their cache of map output locations if this happens. + /** + * Incremented every time a fetch fails so that client nodes know to clear + * their cache of map output locations if this happens. + */ protected var epoch: Long = 0 - protected val epochLock = new java.lang.Object + protected val epochLock = new AnyRef - private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) + /** Remembers which map output locations are currently being fetched on a worker. */ + private val fetching = new HashSet[Int] - // Send a message to the trackerActor and get its result within a default timeout, or - // throw a SparkException if this fails. - private def askTracker(message: Any): Any = { + /** + * Send a message to the trackerActor and get its result within a default timeout, or + * throw a SparkException if this fails. + */ + protected def askTracker(message: Any): Any = { try { val future = trackerActor.ask(message)(timeout) Await.result(future, timeout) @@ -94,17 +109,17 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { } } - // Send a one-way message to the trackerActor, to which we expect it to reply with true. - private def communicate(message: Any) { + /** Send a one-way message to the trackerActor, to which we expect it to reply with true. */ + protected def sendTracker(message: Any) { if (askTracker(message) != true) { throw new SparkException("Error reply received from MapOutputTracker") } } - // Remembers which map output locations are currently being fetched on a worker - private val fetching = new HashSet[Int] - - // Called on possibly remote nodes to get the server URIs and output sizes for a given shuffle + /** + * Called from executors to get the server URIs and output sizes of the map outputs of + * a given shuffle. + */ def getServerStatuses(shuffleId: Int, reduceId: Int): Array[(BlockManagerId, Long)] = { val statuses = mapStatuses.get(shuffleId).orNull if (statuses == null) { @@ -152,8 +167,7 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { fetchedStatuses.synchronized { return MapOutputTracker.convertMapStatuses(shuffleId, reduceId, fetchedStatuses) } - } - else { + } else { throw new FetchFailedException(null, shuffleId, -1, reduceId, new Exception("Missing all output locations for shuffle " + shuffleId)) } @@ -164,27 +178,18 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { } } - protected def cleanup(cleanupTime: Long) { - mapStatuses.clearOldValues(cleanupTime) - } - - def stop() { - communicate(StopMapOutputTracker) - mapStatuses.clear() - metadataCleaner.cancel() - trackerActor = null - } - - // Called to get current epoch number + /** Called to get current epoch number. */ def getEpoch: Long = { epochLock.synchronized { return epoch } } - // Called on workers to update the epoch number, potentially clearing old outputs - // because of a fetch failure. (Each worker task calls this with the latest epoch - // number on the master at the time it was created.) + /** + * Called from executors to update the epoch number, potentially clearing old outputs + * because of a fetch failure. Each worker task calls this with the latest epoch + * number on the master at the time it was created. + */ def updateEpoch(newEpoch: Long) { epochLock.synchronized { if (newEpoch > epoch) { @@ -194,17 +199,40 @@ private[spark] class MapOutputTracker(conf: SparkConf) extends Logging { } } } + + /** Unregister shuffle data. */ + def unregisterShuffle(shuffleId: Int) { + mapStatuses.remove(shuffleId) + } + + /** Stop the tracker. */ + def stop() { } } +/** + * MapOutputTracker for the driver. This uses TimeStampedHashMap to keep track of map + * output information, which allows old output information based on a TTL. + */ private[spark] class MapOutputTrackerMaster(conf: SparkConf) extends MapOutputTracker(conf) { - // Cache a serialized version of the output statuses for each shuffle to send them out faster + /** Cache a serialized version of the output statuses for each shuffle to send them out faster */ private var cacheEpoch = epoch - private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]] + + /** + * Timestamp based HashMap for storing mapStatuses and cached serialized statuses in the master, + * so that statuses are dropped only by explicit de-registering or by TTL-based cleaning (if set). + * Other than these two scenarios, nothing should be dropped from this HashMap. + */ + protected val mapStatuses = new TimeStampedHashMap[Int, Array[MapStatus]]() + private val cachedSerializedStatuses = new TimeStampedHashMap[Int, Array[Byte]]() + + // For cleaning up TimeStampedHashMaps + private val metadataCleaner = + new MetadataCleaner(MetadataCleanerType.MAP_OUTPUT_TRACKER, this.cleanup, conf) def registerShuffle(shuffleId: Int, numMaps: Int) { - if (mapStatuses.putIfAbsent(shuffleId, new Array[MapStatus](numMaps)).isDefined) { + if (mapStatuses.put(shuffleId, new Array[MapStatus](numMaps)).isDefined) { throw new IllegalArgumentException("Shuffle ID " + shuffleId + " registered twice") } } @@ -216,6 +244,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + /** Register multiple map output information for the given shuffle */ def registerMapOutputs(shuffleId: Int, statuses: Array[MapStatus], changeEpoch: Boolean = false) { mapStatuses.put(shuffleId, Array[MapStatus]() ++ statuses) if (changeEpoch) { @@ -223,6 +252,7 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + /** Unregister map output information of the given shuffle, mapper and block manager */ def unregisterMapOutput(shuffleId: Int, mapId: Int, bmAddress: BlockManagerId) { val arrayOpt = mapStatuses.get(shuffleId) if (arrayOpt.isDefined && arrayOpt.get != null) { @@ -238,6 +268,17 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) } } + /** Unregister shuffle data */ + override def unregisterShuffle(shuffleId: Int) { + mapStatuses.remove(shuffleId) + cachedSerializedStatuses.remove(shuffleId) + } + + /** Check if the given shuffle is being tracked */ + def containsShuffle(shuffleId: Int): Boolean = { + cachedSerializedStatuses.contains(shuffleId) || mapStatuses.contains(shuffleId) + } + def incrementEpoch() { epochLock.synchronized { epoch += 1 @@ -274,23 +315,26 @@ private[spark] class MapOutputTrackerMaster(conf: SparkConf) bytes } - protected override def cleanup(cleanupTime: Long) { - super.cleanup(cleanupTime) - cachedSerializedStatuses.clearOldValues(cleanupTime) - } - override def stop() { - super.stop() + sendTracker(StopMapOutputTracker) + mapStatuses.clear() + trackerActor = null + metadataCleaner.cancel() cachedSerializedStatuses.clear() } - override def updateEpoch(newEpoch: Long) { - // This might be called on the MapOutputTrackerMaster if we're running in local mode. + private def cleanup(cleanupTime: Long) { + mapStatuses.clearOldValues(cleanupTime) + cachedSerializedStatuses.clearOldValues(cleanupTime) } +} - def has(shuffleId: Int): Boolean = { - cachedSerializedStatuses.get(shuffleId).isDefined || mapStatuses.contains(shuffleId) - } +/** + * MapOutputTracker for the workers, which fetches map output information from the driver's + * MapOutputTrackerMaster. + */ +private[spark] class MapOutputTrackerWorker(conf: SparkConf) extends MapOutputTracker(conf) { + protected val mapStatuses = new HashMap[Int, Array[MapStatus]] } private[spark] object MapOutputTracker { diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e5ebd350eeced..d7124616d3bfb 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -45,7 +45,7 @@ import org.apache.spark.scheduler.cluster.mesos.{CoarseMesosSchedulerBackend, Me import org.apache.spark.scheduler.local.LocalBackend import org.apache.spark.storage.{BlockManagerSource, RDDInfo, StorageStatus, StorageUtils} import org.apache.spark.ui.SparkUI -import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark @@ -157,7 +157,7 @@ class SparkContext( private[spark] val addedJars = HashMap[String, Long]() // Keeps track of all persisted RDDs - private[spark] val persistentRdds = new TimeStampedHashMap[Int, RDD[_]] + private[spark] val persistentRdds = new TimeStampedWeakValueHashMap[Int, RDD[_]] private[spark] val metadataCleaner = new MetadataCleaner(MetadataCleanerType.SPARK_CONTEXT, this.cleanup, conf) @@ -233,6 +233,15 @@ class SparkContext( @volatile private[spark] var dagScheduler = new DAGScheduler(this) dagScheduler.start() + private[spark] val cleaner: Option[ContextCleaner] = { + if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { + Some(new ContextCleaner(this)) + } else { + None + } + } + cleaner.foreach(_.start()) + postEnvironmentUpdate() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ @@ -679,7 +688,11 @@ class SparkContext( * [[org.apache.spark.broadcast.Broadcast]] object for reading it in distributed functions. * The variable will be sent to each cluster only once. */ - def broadcast[T](value: T): Broadcast[T] = env.broadcastManager.newBroadcast[T](value, isLocal) + def broadcast[T](value: T): Broadcast[T] = { + val bc = env.broadcastManager.newBroadcast[T](value, isLocal) + cleaner.foreach(_.registerBroadcastForCleanup(bc)) + bc + } /** * Add a file to be downloaded with this Spark job on every node. @@ -789,8 +802,7 @@ class SparkContext( /** * Unpersist an RDD from memory and/or disk storage */ - private[spark] def unpersistRDD(rdd: RDD[_], blocking: Boolean = true) { - val rddId = rdd.id + private[spark] def unpersistRDD(rddId: Int, blocking: Boolean = true) { env.blockManager.master.removeRdd(rddId, blocking) persistentRdds.remove(rddId) listenerBus.post(SparkListenerUnpersistRDD(rddId)) @@ -869,6 +881,7 @@ class SparkContext( dagScheduler = null if (dagSchedulerCopy != null) { metadataCleaner.cancel() + cleaner.foreach(_.stop()) dagSchedulerCopy.stop() listenerBus.stop() taskScheduler = null diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 5ceac28fe7afb..9ea123f174b95 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -180,12 +180,24 @@ object SparkEnv extends Logging { } } + val mapOutputTracker = if (isDriver) { + new MapOutputTrackerMaster(conf) + } else { + new MapOutputTrackerWorker(conf) + } + + // Have to assign trackerActor after initialization as MapOutputTrackerActor + // requires the MapOutputTracker itself + mapOutputTracker.trackerActor = registerOrLookup( + "MapOutputTracker", + new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) + val blockManagerMaster = new BlockManagerMaster(registerOrLookup( "BlockManagerMaster", new BlockManagerMasterActor(isLocal, conf, listenerBus)), conf) val blockManager = new BlockManager(executorId, actorSystem, blockManagerMaster, - serializer, conf, securityManager) + serializer, conf, securityManager, mapOutputTracker) val connectionManager = blockManager.connectionManager @@ -193,17 +205,6 @@ object SparkEnv extends Logging { val cacheManager = new CacheManager(blockManager) - // Have to assign trackerActor after initialization as MapOutputTrackerActor - // requires the MapOutputTracker itself - val mapOutputTracker = if (isDriver) { - new MapOutputTrackerMaster(conf) - } else { - new MapOutputTracker(conf) - } - mapOutputTracker.trackerActor = registerOrLookup( - "MapOutputTracker", - new MapOutputTrackerMasterActor(mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], conf)) - val shuffleFetcher = instantiateClass[ShuffleFetcher]( "spark.shuffle.fetcher", "org.apache.spark.BlockStoreShuffleFetcher") diff --git a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala index e3c3a12d16f2a..738a3b1bed7f3 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/Broadcast.scala @@ -18,9 +18,8 @@ package org.apache.spark.broadcast import java.io.Serializable -import java.util.concurrent.atomic.AtomicLong -import org.apache.spark._ +import org.apache.spark.SparkException /** * A broadcast variable. Broadcast variables allow the programmer to keep a read-only variable @@ -29,7 +28,8 @@ import org.apache.spark._ * attempts to distribute broadcast variables using efficient broadcast algorithms to reduce * communication cost. * - * Broadcast variables are created from a variable `v` by calling [[SparkContext#broadcast]]. + * Broadcast variables are created from a variable `v` by calling + * [[org.apache.spark.SparkContext#broadcast]]. * The broadcast variable is a wrapper around `v`, and its value can be accessed by calling the * `value` method. The interpreter session below shows this: * @@ -51,49 +51,80 @@ import org.apache.spark._ * @tparam T Type of the data contained in the broadcast variable. */ abstract class Broadcast[T](val id: Long) extends Serializable { - def value: T - // We cannot have an abstract readObject here due to some weird issues with - // readObject having to be 'private' in sub-classes. + /** + * Flag signifying whether the broadcast variable is valid + * (that is, not already destroyed) or not. + */ + @volatile private var _isValid = true - override def toString = "Broadcast(" + id + ")" -} - -private[spark] -class BroadcastManager(val _isDriver: Boolean, conf: SparkConf, securityManager: SecurityManager) - extends Logging with Serializable { - - private var initialized = false - private var broadcastFactory: BroadcastFactory = null - - initialize() - - // Called by SparkContext or Executor before using Broadcast - private def initialize() { - synchronized { - if (!initialized) { - val broadcastFactoryClass = conf.get( - "spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") - - broadcastFactory = - Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] + /** Get the broadcasted value. */ + def value: T = { + assertValid() + getValue() + } - // Initialize appropriate BroadcastFactory and BroadcastObject - broadcastFactory.initialize(isDriver, conf, securityManager) + /** + * Asynchronously delete cached copies of this broadcast on the executors. + * If the broadcast is used after this is called, it will need to be re-sent to each executor. + */ + def unpersist() { + unpersist(blocking = false) + } - initialized = true - } - } + /** + * Delete cached copies of this broadcast on the executors. If the broadcast is used after + * this is called, it will need to be re-sent to each executor. + * @param blocking Whether to block until unpersisting has completed + */ + def unpersist(blocking: Boolean) { + assertValid() + doUnpersist(blocking) } - def stop() { - broadcastFactory.stop() + /** + * Destroy all data and metadata related to this broadcast variable. Use this with caution; + * once a broadcast variable has been destroyed, it cannot be used again. + */ + private[spark] def destroy(blocking: Boolean) { + assertValid() + _isValid = false + doDestroy(blocking) } - private val nextBroadcastId = new AtomicLong(0) + /** + * Whether this Broadcast is actually usable. This should be false once persisted state is + * removed from the driver. + */ + private[spark] def isValid: Boolean = { + _isValid + } - def newBroadcast[T](value_ : T, isLocal: Boolean) = - broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) + /** + * Actually get the broadcasted value. Concrete implementations of Broadcast class must + * define their own way to get the value. + */ + private[spark] def getValue(): T + + /** + * Actually unpersist the broadcasted value on the executors. Concrete implementations of + * Broadcast class must define their own logic to unpersist their own data. + */ + private[spark] def doUnpersist(blocking: Boolean) + + /** + * Actually destroy all data and metadata related to this broadcast variable. + * Implementation of Broadcast class must define their own logic to destroy their own + * state. + */ + private[spark] def doDestroy(blocking: Boolean) + + /** Check if this broadcast is valid. If not valid, exception is thrown. */ + private[spark] def assertValid() { + if (!_isValid) { + throw new SparkException("Attempted to use %s after it has been destroyed!".format(toString)) + } + } - def isDriver = _isDriver + override def toString = "Broadcast(" + id + ")" } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index 6beecaeced5be..c7f7c59cfb449 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -27,7 +27,8 @@ import org.apache.spark.SparkConf * entire Spark job. */ trait BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean): Unit def stop(): Unit } diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala new file mode 100644 index 0000000000000..cf62aca4d45e8 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastManager.scala @@ -0,0 +1,66 @@ +/* + * 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.broadcast + +import java.util.concurrent.atomic.AtomicLong + +import org.apache.spark._ + +private[spark] class BroadcastManager( + val isDriver: Boolean, + conf: SparkConf, + securityManager: SecurityManager) + extends Logging { + + private var initialized = false + private var broadcastFactory: BroadcastFactory = null + + initialize() + + // Called by SparkContext or Executor before using Broadcast + private def initialize() { + synchronized { + if (!initialized) { + val broadcastFactoryClass = + conf.get("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") + + broadcastFactory = + Class.forName(broadcastFactoryClass).newInstance.asInstanceOf[BroadcastFactory] + + // Initialize appropriate BroadcastFactory and BroadcastObject + broadcastFactory.initialize(isDriver, conf, securityManager) + + initialized = true + } + } + } + + def stop() { + broadcastFactory.stop() + } + + private val nextBroadcastId = new AtomicLong(0) + + def newBroadcast[T](value_ : T, isLocal: Boolean) = { + broadcastFactory.newBroadcast[T](value_, isLocal, nextBroadcastId.getAndIncrement()) + } + + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + broadcastFactory.unbroadcast(id, removeFromDriver, blocking) + } +} diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index e8eb04bb10469..f6a8a8af91e4b 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -17,34 +17,65 @@ package org.apache.spark.broadcast -import java.io.{File, FileOutputStream, ObjectInputStream, OutputStream} -import java.net.{URL, URLConnection, URI} +import java.io.{File, FileOutputStream, ObjectInputStream, ObjectOutputStream, OutputStream} +import java.net.{URI, URL, URLConnection} import java.util.concurrent.TimeUnit -import it.unimi.dsi.fastutil.io.FastBufferedInputStream -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream +import it.unimi.dsi.fastutil.io.{FastBufferedInputStream, FastBufferedOutputStream} -import org.apache.spark.{SparkConf, HttpServer, Logging, SecurityManager, SparkEnv} +import org.apache.spark.{HttpServer, Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashSet, Utils} +/** + * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses HTTP server + * as a broadcast mechanism. The first time a HTTP broadcast variable (sent as part of a + * task) is deserialized in the executor, the broadcasted data is fetched from the driver + * (through a HTTP server running at the driver) and stored in the BlockManager of the + * executor to speed up future accesses. + */ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) extends Broadcast[T](id) with Logging with Serializable { - def value = value_ + def getValue = value_ - def blockId = BroadcastBlockId(id) + val blockId = BroadcastBlockId(id) + /* + * Broadcasted data is also stored in the BlockManager of the driver. The BlockManagerMaster + * does not need to be told about this block as not only need to know about this data block. + */ HttpBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) + SparkEnv.get.blockManager.putSingle( + blockId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) } if (!isLocal) { HttpBroadcast.write(id, value_) } - // Called by JVM when deserializing an object + /** + * Remove all persisted state associated with this HTTP broadcast on the executors. + */ + def doUnpersist(blocking: Boolean) { + HttpBroadcast.unpersist(id, removeFromDriver = false, blocking) + } + + /** + * Remove all persisted state associated with this HTTP broadcast on the executors and driver. + */ + def doDestroy(blocking: Boolean) { + HttpBroadcast.unpersist(id, removeFromDriver = true, blocking) + } + + /** Used by the JVM when serializing this object. */ + private def writeObject(out: ObjectOutputStream) { + assertValid() + out.defaultWriteObject() + } + + /** Used by the JVM when deserializing this object. */ private def readObject(in: ObjectInputStream) { in.defaultReadObject() HttpBroadcast.synchronized { @@ -54,7 +85,13 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea logInfo("Started reading broadcast variable " + id) val start = System.nanoTime value_ = HttpBroadcast.read[T](id) - SparkEnv.get.blockManager.putSingle(blockId, value_, StorageLevel.MEMORY_AND_DISK, false) + /* + * We cache broadcast data in the BlockManager so that subsequent tasks using it + * do not need to re-fetch. This data is only used locally and no other node + * needs to fetch this block, so we don't notify the master. + */ + SparkEnv.get.blockManager.putSingle( + blockId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) val time = (System.nanoTime - start) / 1e9 logInfo("Reading broadcast variable " + id + " took " + time + " s") } @@ -63,23 +100,8 @@ private[spark] class HttpBroadcast[T](@transient var value_ : T, isLocal: Boolea } } -/** - * A [[BroadcastFactory]] implementation that uses a HTTP server as the broadcast medium. - */ -class HttpBroadcastFactory extends BroadcastFactory { - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { - HttpBroadcast.initialize(isDriver, conf, securityMgr) - } - - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = - new HttpBroadcast[T](value_, isLocal, id) - - def stop() { HttpBroadcast.stop() } -} - -private object HttpBroadcast extends Logging { +private[spark] object HttpBroadcast extends Logging { private var initialized = false - private var broadcastDir: File = null private var compress: Boolean = false private var bufferSize: Int = 65536 @@ -89,11 +111,9 @@ private object HttpBroadcast extends Logging { // TODO: This shouldn't be a global variable so that multiple SparkContexts can coexist private val files = new TimeStampedHashSet[String] - private var cleaner: MetadataCleaner = null - private val httpReadTimeout = TimeUnit.MILLISECONDS.convert(5, TimeUnit.MINUTES).toInt - private var compressionCodec: CompressionCodec = null + private var cleaner: MetadataCleaner = null def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { synchronized { @@ -136,8 +156,10 @@ private object HttpBroadcast extends Logging { logInfo("Broadcast server started at " + serverUri) } + def getFile(id: Long) = new File(broadcastDir, BroadcastBlockId(id).name) + def write(id: Long, value: Any) { - val file = new File(broadcastDir, BroadcastBlockId(id).name) + val file = getFile(id) val out: OutputStream = { if (compress) { compressionCodec.compressedOutputStream(new FileOutputStream(file)) @@ -160,7 +182,7 @@ private object HttpBroadcast extends Logging { if (securityManager.isAuthenticationEnabled()) { logDebug("broadcast security enabled") val newuri = Utils.constructURIForAuthentication(new URI(url), securityManager) - uc = newuri.toURL().openConnection() + uc = newuri.toURL.openConnection() uc.setAllowUserInteraction(false) } else { logDebug("broadcast not using security") @@ -169,7 +191,7 @@ private object HttpBroadcast extends Logging { val in = { uc.setReadTimeout(httpReadTimeout) - val inputStream = uc.getInputStream(); + val inputStream = uc.getInputStream if (compress) { compressionCodec.compressedInputStream(inputStream) } else { @@ -183,20 +205,48 @@ private object HttpBroadcast extends Logging { obj } - def cleanup(cleanupTime: Long) { + /** + * Remove all persisted blocks associated with this HTTP broadcast on the executors. + * If removeFromDriver is true, also remove these persisted blocks on the driver + * and delete the associated broadcast file. + */ + def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = synchronized { + SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) + if (removeFromDriver) { + val file = getFile(id) + files.remove(file.toString) + deleteBroadcastFile(file) + } + } + + /** + * Periodically clean up old broadcasts by removing the associated map entries and + * deleting the associated files. + */ + private def cleanup(cleanupTime: Long) { val iterator = files.internalMap.entrySet().iterator() while(iterator.hasNext) { val entry = iterator.next() val (file, time) = (entry.getKey, entry.getValue) if (time < cleanupTime) { - try { - iterator.remove() - new File(file.toString).delete() - logInfo("Deleted broadcast file '" + file + "'") - } catch { - case e: Exception => logWarning("Could not delete broadcast file '" + file + "'", e) + iterator.remove() + deleteBroadcastFile(new File(file.toString)) + } + } + } + + private def deleteBroadcastFile(file: File) { + try { + if (file.exists) { + if (file.delete()) { + logInfo("Deleted broadcast file: %s".format(file)) + } else { + logWarning("Could not delete broadcast file: %s".format(file)) } } + } catch { + case e: Exception => + logError("Exception while deleting broadcast file: %s".format(file), e) } } } diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala new file mode 100644 index 0000000000000..e3f6cdc6154dd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcastFactory.scala @@ -0,0 +1,45 @@ +/* + * 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.broadcast + +import org.apache.spark.{SecurityManager, SparkConf} + +/** + * A [[org.apache.spark.broadcast.BroadcastFactory]] implementation that uses a + * HTTP server as the broadcast mechanism. Refer to + * [[org.apache.spark.broadcast.HttpBroadcast]] for more details about this mechanism. + */ +class HttpBroadcastFactory extends BroadcastFactory { + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + HttpBroadcast.initialize(isDriver, conf, securityMgr) + } + + def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + new HttpBroadcast[T](value_, isLocal, id) + + def stop() { HttpBroadcast.stop() } + + /** + * Remove all persisted state associated with the HTTP broadcast with the given ID. + * @param removeFromDriver Whether to remove state from the driver + * @param blocking Whether to block until unbroadcasted + */ + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + HttpBroadcast.unpersist(id, removeFromDriver, blocking) + } +} diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 2595c15104e87..2b32546c6854d 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -17,24 +17,43 @@ package org.apache.spark.broadcast -import java.io._ +import java.io.{ByteArrayInputStream, ObjectInputStream, ObjectOutputStream} import scala.math import scala.util.Random -import org.apache.spark._ -import org.apache.spark.storage.{BroadcastBlockId, BroadcastHelperBlockId, StorageLevel} +import org.apache.spark.{Logging, SparkConf, SparkEnv, SparkException} +import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} import org.apache.spark.util.Utils +/** + * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like + * protocol to do a distributed transfer of the broadcasted data to the executors. + * The mechanism is as follows. The driver divides the serializes the broadcasted data, + * divides it into smaller chunks, and stores them in the BlockManager of the driver. + * These chunks are reported to the BlockManagerMaster so that all the executors can + * learn the location of those chunks. The first time the broadcast variable (sent as + * part of task) is deserialized at a executor, all the chunks are fetched using + * the BlockManager. When all the chunks are fetched (initially from the driver's + * BlockManager), they are combined and deserialized to recreate the broadcasted data. + * However, the chunks are also stored in the BlockManager and reported to the + * BlockManagerMaster. As more executors fetch the chunks, BlockManagerMaster learns + * multiple locations for each chunk. Hence, subsequent fetches of each chunk will be + * made to other executors who already have those chunks, resulting in a distributed + * fetching. This prevents the driver from being the bottleneck in sending out multiple + * copies of the broadcast data (one per executor) as done by the + * [[org.apache.spark.broadcast.HttpBroadcast]]. + */ private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boolean, id: Long) -extends Broadcast[T](id) with Logging with Serializable { + extends Broadcast[T](id) with Logging with Serializable { - def value = value_ + def getValue = value_ - def broadcastId = BroadcastBlockId(id) + val broadcastId = BroadcastBlockId(id) TorrentBroadcast.synchronized { - SparkEnv.get.blockManager.putSingle(broadcastId, value_, StorageLevel.MEMORY_AND_DISK, false) + SparkEnv.get.blockManager.putSingle( + broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) } @transient var arrayOfBlocks: Array[TorrentBlock] = null @@ -46,32 +65,52 @@ extends Broadcast[T](id) with Logging with Serializable { sendBroadcast() } - def sendBroadcast() { - var tInfo = TorrentBroadcast.blockifyObject(value_) + /** + * Remove all persisted state associated with this Torrent broadcast on the executors. + */ + def doUnpersist(blocking: Boolean) { + TorrentBroadcast.unpersist(id, removeFromDriver = false, blocking) + } + + /** + * Remove all persisted state associated with this Torrent broadcast on the executors + * and driver. + */ + def doDestroy(blocking: Boolean) { + TorrentBroadcast.unpersist(id, removeFromDriver = true, blocking) + } + def sendBroadcast() { + val tInfo = TorrentBroadcast.blockifyObject(value_) totalBlocks = tInfo.totalBlocks totalBytes = tInfo.totalBytes hasBlocks = tInfo.totalBlocks // Store meta-info - val metaId = BroadcastHelperBlockId(broadcastId, "meta") + val metaId = BroadcastBlockId(id, "meta") val metaInfo = TorrentInfo(null, totalBlocks, totalBytes) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.putSingle( - metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, true) + metaId, metaInfo, StorageLevel.MEMORY_AND_DISK, tellMaster = true) } // Store individual pieces for (i <- 0 until totalBlocks) { - val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + i) + val pieceId = BroadcastBlockId(id, "piece" + i) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.putSingle( - pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, true) + pieceId, tInfo.arrayOfBlocks(i), StorageLevel.MEMORY_AND_DISK, tellMaster = true) } } } - // Called by JVM when deserializing an object + /** Used by the JVM when serializing this object. */ + private def writeObject(out: ObjectOutputStream) { + assertValid() + out.defaultWriteObject() + } + + /** Used by the JVM when deserializing this object. */ private def readObject(in: ObjectInputStream) { in.defaultReadObject() TorrentBroadcast.synchronized { @@ -86,18 +125,22 @@ extends Broadcast[T](id) with Logging with Serializable { // Initialize @transient variables that will receive garbage values from the master. resetWorkerVariables() - if (receiveBroadcast(id)) { + if (receiveBroadcast()) { value_ = TorrentBroadcast.unBlockifyObject[T](arrayOfBlocks, totalBytes, totalBlocks) - // Store the merged copy in cache so that the next worker doesn't need to rebuild it. - // This creates a tradeoff between memory usage and latency. - // Storing copy doubles the memory footprint; not storing doubles deserialization cost. + /* Store the merged copy in cache so that the next worker doesn't need to rebuild it. + * This creates a trade-off between memory usage and latency. Storing copy doubles + * the memory footprint; not storing doubles deserialization cost. Also, + * this does not need to be reported to BlockManagerMaster since other executors + * does not need to access this block (they only need to fetch the chunks, + * which are reported). + */ SparkEnv.get.blockManager.putSingle( - broadcastId, value_, StorageLevel.MEMORY_AND_DISK, false) + broadcastId, value_, StorageLevel.MEMORY_AND_DISK, tellMaster = false) // Remove arrayOfBlocks from memory once value_ is on local cache resetWorkerVariables() - } else { + } else { logError("Reading broadcast variable " + id + " failed") } @@ -114,9 +157,10 @@ extends Broadcast[T](id) with Logging with Serializable { hasBlocks = 0 } - def receiveBroadcast(variableID: Long): Boolean = { - // Receive meta-info - val metaId = BroadcastHelperBlockId(broadcastId, "meta") + def receiveBroadcast(): Boolean = { + // Receive meta-info about the size of broadcast data, + // the number of chunks it is divided into, etc. + val metaId = BroadcastBlockId(id, "meta") var attemptId = 10 while (attemptId > 0 && totalBlocks == -1) { TorrentBroadcast.synchronized { @@ -138,17 +182,21 @@ extends Broadcast[T](id) with Logging with Serializable { return false } - // Receive actual blocks + /* + * Fetch actual chunks of data. Note that all these chunks are stored in + * the BlockManager and reported to the master, so that other executors + * can find out and pull the chunks from this executor. + */ val recvOrder = new Random().shuffle(Array.iterate(0, totalBlocks)(_ + 1).toList) for (pid <- recvOrder) { - val pieceId = BroadcastHelperBlockId(broadcastId, "piece" + pid) + val pieceId = BroadcastBlockId(id, "piece" + pid) TorrentBroadcast.synchronized { SparkEnv.get.blockManager.getSingle(pieceId) match { case Some(x) => arrayOfBlocks(pid) = x.asInstanceOf[TorrentBlock] hasBlocks += 1 SparkEnv.get.blockManager.putSingle( - pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, true) + pieceId, arrayOfBlocks(pid), StorageLevel.MEMORY_AND_DISK, tellMaster = true) case None => throw new SparkException("Failed to get " + pieceId + " of " + broadcastId) @@ -156,16 +204,16 @@ extends Broadcast[T](id) with Logging with Serializable { } } - (hasBlocks == totalBlocks) + hasBlocks == totalBlocks } } -private object TorrentBroadcast -extends Logging { - +private[spark] object TorrentBroadcast extends Logging { + private lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 private var initialized = false private var conf: SparkConf = null + def initialize(_isDriver: Boolean, conf: SparkConf) { TorrentBroadcast.conf = conf // TODO: we might have to fix it in tests synchronized { @@ -179,39 +227,37 @@ extends Logging { initialized = false } - lazy val BLOCK_SIZE = conf.getInt("spark.broadcast.blockSize", 4096) * 1024 - def blockifyObject[T](obj: T): TorrentInfo = { val byteArray = Utils.serialize[T](obj) val bais = new ByteArrayInputStream(byteArray) - var blockNum = (byteArray.length / BLOCK_SIZE) + var blockNum = byteArray.length / BLOCK_SIZE if (byteArray.length % BLOCK_SIZE != 0) { blockNum += 1 } - var retVal = new Array[TorrentBlock](blockNum) - var blockID = 0 + val blocks = new Array[TorrentBlock](blockNum) + var blockId = 0 for (i <- 0 until (byteArray.length, BLOCK_SIZE)) { val thisBlockSize = math.min(BLOCK_SIZE, byteArray.length - i) - var tempByteArray = new Array[Byte](thisBlockSize) - val hasRead = bais.read(tempByteArray, 0, thisBlockSize) + val tempByteArray = new Array[Byte](thisBlockSize) + bais.read(tempByteArray, 0, thisBlockSize) - retVal(blockID) = new TorrentBlock(blockID, tempByteArray) - blockID += 1 + blocks(blockId) = new TorrentBlock(blockId, tempByteArray) + blockId += 1 } bais.close() - val tInfo = TorrentInfo(retVal, blockNum, byteArray.length) - tInfo.hasBlocks = blockNum - - tInfo + val info = TorrentInfo(blocks, blockNum, byteArray.length) + info.hasBlocks = blockNum + info } - def unBlockifyObject[T](arrayOfBlocks: Array[TorrentBlock], - totalBytes: Int, - totalBlocks: Int): T = { + def unBlockifyObject[T]( + arrayOfBlocks: Array[TorrentBlock], + totalBytes: Int, + totalBlocks: Int): T = { val retByteArray = new Array[Byte](totalBytes) for (i <- 0 until totalBlocks) { System.arraycopy(arrayOfBlocks(i).byteArray, 0, retByteArray, @@ -220,6 +266,13 @@ extends Logging { Utils.deserialize[T](retByteArray, Thread.currentThread.getContextClassLoader) } + /** + * Remove all persisted blocks associated with this torrent broadcast on the executors. + * If removeFromDriver is true, also remove these persisted blocks on the driver. + */ + def unpersist(id: Long, removeFromDriver: Boolean, blocking: Boolean) = synchronized { + SparkEnv.get.blockManager.master.removeBroadcast(id, removeFromDriver, blocking) + } } private[spark] case class TorrentBlock( @@ -228,25 +281,10 @@ private[spark] case class TorrentBlock( extends Serializable private[spark] case class TorrentInfo( - @transient arrayOfBlocks : Array[TorrentBlock], + @transient arrayOfBlocks: Array[TorrentBlock], totalBlocks: Int, totalBytes: Int) extends Serializable { @transient var hasBlocks = 0 } - -/** - * A [[BroadcastFactory]] that creates a torrent-based implementation of broadcast. - */ -class TorrentBroadcastFactory extends BroadcastFactory { - - def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { - TorrentBroadcast.initialize(isDriver, conf) - } - - def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = - new TorrentBroadcast[T](value_, isLocal, id) - - def stop() { TorrentBroadcast.stop() } -} diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.scala new file mode 100644 index 0000000000000..d216b58718148 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcastFactory.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.broadcast + +import org.apache.spark.{SecurityManager, SparkConf} + +/** + * A [[org.apache.spark.broadcast.Broadcast]] implementation that uses a BitTorrent-like + * protocol to do a distributed transfer of the broadcasted data to the executors. Refer to + * [[org.apache.spark.broadcast.TorrentBroadcast]] for more details. + */ +class TorrentBroadcastFactory extends BroadcastFactory { + + def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager) { + TorrentBroadcast.initialize(isDriver, conf) + } + + def newBroadcast[T](value_ : T, isLocal: Boolean, id: Long) = + new TorrentBroadcast[T](value_, isLocal, id) + + def stop() { TorrentBroadcast.stop() } + + /** + * Remove all persisted state associated with the torrent broadcast with the given ID. + * @param removeFromDriver Whether to remove state from the driver. + * @param blocking Whether to block until unbroadcasted + */ + def unbroadcast(id: Long, removeFromDriver: Boolean, blocking: Boolean) { + TorrentBroadcast.unpersist(id, removeFromDriver, blocking) + } +} diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index 6b0a972f0bbe0..bdf586351ac14 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -17,7 +17,6 @@ package org.apache.spark.network -import java.net._ import java.nio._ import java.nio.channels._ import java.nio.channels.spi._ diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index c43823bd769b7..bf3c57ad41eb2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -138,6 +138,8 @@ abstract class RDD[T: ClassTag]( "Cannot change storage level of an RDD after it was already assigned a level") } sc.persistRDD(this) + // Register the RDD with the ContextCleaner for automatic GC-based cleanup + sc.cleaner.foreach(_.registerRDDForCleanup(this)) storageLevel = newLevel this } @@ -156,7 +158,7 @@ abstract class RDD[T: ClassTag]( */ def unpersist(blocking: Boolean = true): RDD[T] = { logInfo("Removing RDD " + id + " from persistence list") - sc.unpersistRDD(this, blocking) + sc.unpersistRDD(id, blocking) storageLevel = StorageLevel.NONE this } @@ -1141,5 +1143,4 @@ abstract class RDD[T: ClassTag]( def toJavaRDD() : JavaRDD[T] = { new JavaRDD(this)(elementClassTag) } - } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 442a95bb2c44b..6368665f249ee 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -32,7 +32,7 @@ import org.apache.spark.executor.TaskMetrics import org.apache.spark.partial.{ApproximateActionListener, ApproximateEvaluator, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.{BlockId, BlockManager, BlockManagerMaster, RDDBlockId} -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap, Utils} +import org.apache.spark.util.Utils /** * The high-level scheduling layer that implements stage-oriented scheduling. It computes a DAG of @@ -80,13 +80,13 @@ class DAGScheduler( private[scheduler] def numTotalJobs: Int = nextJobId.get() private val nextStageId = new AtomicInteger(0) - private[scheduler] val jobIdToStageIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToJobIds = new TimeStampedHashMap[Int, HashSet[Int]] - private[scheduler] val stageIdToStage = new TimeStampedHashMap[Int, Stage] - private[scheduler] val shuffleToMapStage = new TimeStampedHashMap[Int, Stage] + private[scheduler] val jobIdToStageIds = new HashMap[Int, HashSet[Int]] + private[scheduler] val stageIdToJobIds = new HashMap[Int, HashSet[Int]] + private[scheduler] val stageIdToStage = new HashMap[Int, Stage] + private[scheduler] val shuffleToMapStage = new HashMap[Int, Stage] private[scheduler] val jobIdToActiveJob = new HashMap[Int, ActiveJob] private[scheduler] val resultStageToJob = new HashMap[Stage, ActiveJob] - private[scheduler] val stageToInfos = new TimeStampedHashMap[Stage, StageInfo] + private[scheduler] val stageToInfos = new HashMap[Stage, StageInfo] // Stages we need to run whose parents aren't done private[scheduler] val waitingStages = new HashSet[Stage] @@ -98,7 +98,7 @@ class DAGScheduler( private[scheduler] val failedStages = new HashSet[Stage] // Missing tasks from each stage - private[scheduler] val pendingTasks = new TimeStampedHashMap[Stage, HashSet[Task[_]]] + private[scheduler] val pendingTasks = new HashMap[Stage, HashSet[Task[_]]] private[scheduler] val activeJobs = new HashSet[ActiveJob] @@ -113,9 +113,6 @@ class DAGScheduler( // stray messages to detect. private val failedEpoch = new HashMap[String, Long] - private val metadataCleaner = - new MetadataCleaner(MetadataCleanerType.DAG_SCHEDULER, this.cleanup, env.conf) - taskScheduler.setDAGScheduler(this) /** @@ -258,7 +255,7 @@ class DAGScheduler( : Stage = { val stage = newStage(rdd, numTasks, Some(shuffleDep), jobId, callSite) - if (mapOutputTracker.has(shuffleDep.shuffleId)) { + if (mapOutputTracker.containsShuffle(shuffleDep.shuffleId)) { val serLocs = mapOutputTracker.getSerializedMapOutputStatuses(shuffleDep.shuffleId) val locs = MapOutputTracker.deserializeMapStatuses(serLocs) for (i <- 0 until locs.size) { @@ -390,6 +387,9 @@ class DAGScheduler( stageIdToStage -= stageId stageIdToJobIds -= stageId + ShuffleMapTask.removeStage(stageId) + ResultTask.removeStage(stageId) + logDebug("After removal of stage %d, remaining stages = %d" .format(stageId, stageIdToStage.size)) } @@ -1084,26 +1084,10 @@ class DAGScheduler( Nil } - private def cleanup(cleanupTime: Long) { - Map( - "stageIdToStage" -> stageIdToStage, - "shuffleToMapStage" -> shuffleToMapStage, - "pendingTasks" -> pendingTasks, - "stageToInfos" -> stageToInfos, - "jobIdToStageIds" -> jobIdToStageIds, - "stageIdToJobIds" -> stageIdToJobIds). - foreach { case (s, t) => - val sizeBefore = t.size - t.clearOldValues(cleanupTime) - logInfo("%s %d --> %d".format(s, sizeBefore, t.size)) - } - } - def stop() { if (eventProcessActor != null) { eventProcessActor ! StopDAGScheduler } - metadataCleaner.cancel() taskScheduler.stop() } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 3fc6cc9850feb..083fb895d8696 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -20,21 +20,17 @@ package org.apache.spark.scheduler import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} +import scala.collection.mutable.HashMap + import org.apache.spark._ -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.RDDCheckpointData -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} +import org.apache.spark.rdd.{RDD, RDDCheckpointData} private[spark] object ResultTask { // A simple map between the stage id to the serialized byte array of a task. // Served as a cache for task serialization because serialization can be // expensive on the master node if it needs to launch thousands of tasks. - val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - - // TODO: This object shouldn't have global variables - val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.RESULT_TASK, serializedInfoCache.clearOldValues, new SparkConf) + private val serializedInfoCache = new HashMap[Int, Array[Byte]] def serializeInfo(stageId: Int, rdd: RDD[_], func: (TaskContext, Iterator[_]) => _): Array[Byte] = { @@ -67,6 +63,10 @@ private[spark] object ResultTask { (rdd, func) } + def removeStage(stageId: Int) { + serializedInfoCache.remove(stageId) + } + def clearCache() { synchronized { serializedInfoCache.clear() diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 2a9edf4a76b97..23f3b3e824762 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -24,22 +24,16 @@ import scala.collection.mutable.HashMap import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics -import org.apache.spark.rdd.RDD -import org.apache.spark.rdd.RDDCheckpointData +import org.apache.spark.rdd.{RDD, RDDCheckpointData} import org.apache.spark.serializer.Serializer import org.apache.spark.storage._ -import org.apache.spark.util.{MetadataCleaner, MetadataCleanerType, TimeStampedHashMap} private[spark] object ShuffleMapTask { // A simple map between the stage id to the serialized byte array of a task. // Served as a cache for task serialization because serialization can be // expensive on the master node if it needs to launch thousands of tasks. - val serializedInfoCache = new TimeStampedHashMap[Int, Array[Byte]] - - // TODO: This object shouldn't have global variables - val metadataCleaner = new MetadataCleaner( - MetadataCleanerType.SHUFFLE_MAP_TASK, serializedInfoCache.clearOldValues, new SparkConf) + private val serializedInfoCache = new HashMap[Int, Array[Byte]] def serializeInfo(stageId: Int, rdd: RDD[_], dep: ShuffleDependency[_,_]): Array[Byte] = { synchronized { @@ -80,6 +74,10 @@ private[spark] object ShuffleMapTask { HashMap(set.toSeq: _*) } + def removeStage(stageId: Int) { + serializedInfoCache.remove(stageId) + } + def clearCache() { synchronized { serializedInfoCache.clear() diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index a92922166f595..acd152dda89d4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -42,7 +42,7 @@ import org.apache.spark.scheduler.SchedulingMode.SchedulingMode * * THREADING: SchedulerBackends and task-submitting clients can call this class from multiple * threads, so it needs locks in public API methods to maintain its state. In addition, some - * SchedulerBackends sycnchronize on themselves when they want to send events here, and then + * SchedulerBackends synchronize on themselves when they want to send events here, and then * acquire a lock on us, so we need to make sure that we don't try to lock the backend while * we are holding a lock on ourselves. */ diff --git a/core/src/main/scala/org/apache/spark/storage/BlockId.scala b/core/src/main/scala/org/apache/spark/storage/BlockId.scala index 301d784b350a3..cffea28fbf794 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockId.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockId.scala @@ -34,7 +34,7 @@ private[spark] sealed abstract class BlockId { def asRDDId = if (isRDD) Some(asInstanceOf[RDDBlockId]) else None def isRDD = isInstanceOf[RDDBlockId] def isShuffle = isInstanceOf[ShuffleBlockId] - def isBroadcast = isInstanceOf[BroadcastBlockId] || isInstanceOf[BroadcastHelperBlockId] + def isBroadcast = isInstanceOf[BroadcastBlockId] override def toString = name override def hashCode = name.hashCode @@ -48,18 +48,13 @@ private[spark] case class RDDBlockId(rddId: Int, splitIndex: Int) extends BlockI def name = "rdd_" + rddId + "_" + splitIndex } -private[spark] -case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) extends BlockId { +private[spark] case class ShuffleBlockId(shuffleId: Int, mapId: Int, reduceId: Int) + extends BlockId { def name = "shuffle_" + shuffleId + "_" + mapId + "_" + reduceId } -private[spark] case class BroadcastBlockId(broadcastId: Long) extends BlockId { - def name = "broadcast_" + broadcastId -} - -private[spark] -case class BroadcastHelperBlockId(broadcastId: BroadcastBlockId, hType: String) extends BlockId { - def name = broadcastId.name + "_" + hType +private[spark] case class BroadcastBlockId(broadcastId: Long, field: String = "") extends BlockId { + def name = "broadcast_" + broadcastId + (if (field == "") "" else "_" + field) } private[spark] case class TaskResultBlockId(taskId: Long) extends BlockId { @@ -83,8 +78,7 @@ private[spark] case class TestBlockId(id: String) extends BlockId { private[spark] object BlockId { val RDD = "rdd_([0-9]+)_([0-9]+)".r val SHUFFLE = "shuffle_([0-9]+)_([0-9]+)_([0-9]+)".r - val BROADCAST = "broadcast_([0-9]+)".r - val BROADCAST_HELPER = "broadcast_([0-9]+)_([A-Za-z0-9]+)".r + val BROADCAST = "broadcast_([0-9]+)([_A-Za-z0-9]*)".r val TASKRESULT = "taskresult_([0-9]+)".r val STREAM = "input-([0-9]+)-([0-9]+)".r val TEST = "test_(.*)".r @@ -95,10 +89,8 @@ private[spark] object BlockId { RDDBlockId(rddId.toInt, splitIndex.toInt) case SHUFFLE(shuffleId, mapId, reduceId) => ShuffleBlockId(shuffleId.toInt, mapId.toInt, reduceId.toInt) - case BROADCAST(broadcastId) => - BroadcastBlockId(broadcastId.toLong) - case BROADCAST_HELPER(broadcastId, hType) => - BroadcastHelperBlockId(BroadcastBlockId(broadcastId.toLong), hType) + case BROADCAST(broadcastId, field) => + BroadcastBlockId(broadcastId.toLong, field.stripPrefix("_")) case TASKRESULT(taskId) => TaskResultBlockId(taskId.toLong) case STREAM(streamId, uniqueId) => diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index 19138d9dde697..b021564477c47 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -19,20 +19,22 @@ package org.apache.spark.storage import java.io.{File, InputStream, OutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} + import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.{Await, Future} import scala.concurrent.duration._ import scala.util.Random + import akka.actor.{ActorSystem, Cancellable, Props} import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkEnv, SparkException} + +import org.apache.spark.{Logging, MapOutputTracker, SecurityManager, SparkConf, SparkEnv, SparkException} import org.apache.spark.io.CompressionCodec import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ - sealed trait Values case class ByteBufferValues(buffer: ByteBuffer) extends Values @@ -46,7 +48,8 @@ private[spark] class BlockManager( val defaultSerializer: Serializer, maxMemory: Long, val conf: SparkConf, - securityManager: SecurityManager) + securityManager: SecurityManager, + mapOutputTracker: MapOutputTracker) extends Logging { val shuffleBlockManager = new ShuffleBlockManager(this) @@ -55,7 +58,7 @@ private[spark] class BlockManager( private val blockInfo = new TimeStampedHashMap[BlockId, BlockInfo] - private[storage] val memoryStore: BlockStore = new MemoryStore(this, maxMemory) + private[storage] val memoryStore = new MemoryStore(this, maxMemory) private[storage] val diskStore = new DiskStore(this, diskBlockManager) var tachyonInitialized = false private[storage] lazy val tachyonStore: TachyonStore = { @@ -98,7 +101,7 @@ private[spark] class BlockManager( val heartBeatFrequency = BlockManager.getHeartBeatFrequency(conf) - val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this)), + val slaveActor = actorSystem.actorOf(Props(new BlockManagerSlaveActor(this, mapOutputTracker)), name = "BlockManagerActor" + BlockManager.ID_GENERATOR.next) // Pending re-registration action being executed asynchronously or null if none @@ -137,9 +140,10 @@ private[spark] class BlockManager( master: BlockManagerMaster, serializer: Serializer, conf: SparkConf, - securityManager: SecurityManager) = { + securityManager: SecurityManager, + mapOutputTracker: MapOutputTracker) = { this(execId, actorSystem, master, serializer, BlockManager.getMaxMemory(conf), - conf, securityManager) + conf, securityManager, mapOutputTracker) } /** @@ -217,9 +221,26 @@ private[spark] class BlockManager( } /** - * Get storage level of local block. If no info exists for the block, then returns null. + * Get the BlockStatus for the block identified by the given ID, if it exists. + * NOTE: This is mainly for testing, and it doesn't fetch information from Tachyon. + */ + def getStatus(blockId: BlockId): Option[BlockStatus] = { + blockInfo.get(blockId).map { info => + val memSize = if (memoryStore.contains(blockId)) memoryStore.getSize(blockId) else 0L + val diskSize = if (diskStore.contains(blockId)) diskStore.getSize(blockId) else 0L + // Assume that block is not in Tachyon + BlockStatus(info.level, memSize, diskSize, 0L) + } + } + + /** + * Get the ids of existing blocks that match the given filter. Note that this will + * query the blocks stored in the disk block manager (that the block manager + * may not know of). */ - def getLevel(blockId: BlockId): StorageLevel = blockInfo.get(blockId).map(_.level).orNull + def getMatchingBlockIds(filter: BlockId => Boolean): Seq[BlockId] = { + (blockInfo.keys ++ diskBlockManager.getAllBlocks()).filter(filter).toSeq + } /** * Tell the master about the current storage status of a block. This will send a block update @@ -525,9 +546,8 @@ private[spark] class BlockManager( /** * A short circuited method to get a block writer that can write data directly to disk. - * The Block will be appended to the File specified by filename. - * This is currently used for writing shuffle files out. Callers should handle error - * cases. + * The Block will be appended to the File specified by filename. This is currently used for + * writing shuffle files out. Callers should handle error cases. */ def getDiskWriter( blockId: BlockId, @@ -863,11 +883,22 @@ private[spark] class BlockManager( * @return The number of blocks removed. */ def removeRdd(rddId: Int): Int = { - // TODO: Instead of doing a linear scan on the blockInfo map, create another map that maps - // from RDD.id to blocks. + // TODO: Avoid a linear scan by creating another mapping of RDD.id to blocks. logInfo("Removing RDD " + rddId) val blocksToRemove = blockInfo.keys.flatMap(_.asRDDId).filter(_.rddId == rddId) - blocksToRemove.foreach(blockId => removeBlock(blockId, tellMaster = false)) + blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster = false) } + blocksToRemove.size + } + + /** + * Remove all blocks belonging to the given broadcast. + */ + def removeBroadcast(broadcastId: Long, tellMaster: Boolean): Int = { + logInfo("Removing broadcast " + broadcastId) + val blocksToRemove = blockInfo.keys.collect { + case bid @ BroadcastBlockId(`broadcastId`, _) => bid + } + blocksToRemove.foreach { blockId => removeBlock(blockId, tellMaster) } blocksToRemove.size } @@ -908,10 +939,10 @@ private[spark] class BlockManager( } private def dropOldBlocks(cleanupTime: Long, shouldDrop: (BlockId => Boolean)) { - val iterator = blockInfo.internalMap.entrySet().iterator() + val iterator = blockInfo.getEntrySet.iterator while (iterator.hasNext) { val entry = iterator.next() - val (id, info, time) = (entry.getKey, entry.getValue._1, entry.getValue._2) + val (id, info, time) = (entry.getKey, entry.getValue.value, entry.getValue.timestamp) if (time < cleanupTime && shouldDrop(id)) { info.synchronized { val level = info.level @@ -935,7 +966,7 @@ private[spark] class BlockManager( def shouldCompress(blockId: BlockId): Boolean = blockId match { case ShuffleBlockId(_, _, _) => compressShuffle - case BroadcastBlockId(_) => compressBroadcast + case BroadcastBlockId(_, _) => compressBroadcast case RDDBlockId(_, _) => compressRdds case TempBlockId(_) => compressShuffleSpill case _ => false diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala index 4bc1b407ad106..7897fade2df2b 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMaster.scala @@ -81,6 +81,14 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log askDriverWithReply[Seq[Seq[BlockManagerId]]](GetLocationsMultipleBlockIds(blockIds)) } + /** + * Check if block manager master has a block. Note that this can be used to check for only + * those blocks that are reported to block manager master. + */ + def contains(blockId: BlockId) = { + !getLocations(blockId).isEmpty + } + /** Get ids of other nodes in the cluster from the driver */ def getPeers(blockManagerId: BlockManagerId, numPeers: Int): Seq[BlockManagerId] = { val result = askDriverWithReply[Seq[BlockManagerId]](GetPeers(blockManagerId, numPeers)) @@ -99,12 +107,10 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log askDriverWithReply(RemoveBlock(blockId)) } - /** - * Remove all blocks belonging to the given RDD. - */ + /** Remove all blocks belonging to the given RDD. */ def removeRdd(rddId: Int, blocking: Boolean) { val future = askDriverWithReply[Future[Seq[Int]]](RemoveRdd(rddId)) - future onFailure { + future.onFailure { case e: Throwable => logError("Failed to remove RDD " + rddId, e) } if (blocking) { @@ -112,6 +118,31 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log } } + /** Remove all blocks belonging to the given shuffle. */ + def removeShuffle(shuffleId: Int, blocking: Boolean) { + val future = askDriverWithReply[Future[Seq[Boolean]]](RemoveShuffle(shuffleId)) + future.onFailure { + case e: Throwable => logError("Failed to remove shuffle " + shuffleId, e) + } + if (blocking) { + Await.result(future, timeout) + } + } + + /** Remove all blocks belonging to the given broadcast. */ + def removeBroadcast(broadcastId: Long, removeFromMaster: Boolean, blocking: Boolean) { + val future = askDriverWithReply[Future[Seq[Int]]]( + RemoveBroadcast(broadcastId, removeFromMaster)) + future.onFailure { + case e: Throwable => + logError("Failed to remove broadcast " + broadcastId + + " with removeFromMaster = " + removeFromMaster, e) + } + if (blocking) { + Await.result(future, timeout) + } + } + /** * Return the memory status for each block manager, in the form of a map from * the block manager's id to two long values. The first value is the maximum @@ -126,6 +157,51 @@ class BlockManagerMaster(var driverActor: ActorRef, conf: SparkConf) extends Log askDriverWithReply[Array[StorageStatus]](GetStorageStatus) } + /** + * Return the block's status on all block managers, if any. NOTE: This is a + * potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, this invokes the master to query each block manager for the most + * updated block statuses. This is useful when the master is not informed of the given block + * by all block managers. + */ + def getBlockStatus( + blockId: BlockId, + askSlaves: Boolean = true): Map[BlockManagerId, BlockStatus] = { + val msg = GetBlockStatus(blockId, askSlaves) + /* + * To avoid potential deadlocks, the use of Futures is necessary, because the master actor + * should not block on waiting for a block manager, which can in turn be waiting for the + * master actor for a response to a prior message. + */ + val response = askDriverWithReply[Map[BlockManagerId, Future[Option[BlockStatus]]]](msg) + val (blockManagerIds, futures) = response.unzip + val result = Await.result(Future.sequence(futures), timeout) + if (result == null) { + throw new SparkException("BlockManager returned null for BlockStatus query: " + blockId) + } + val blockStatus = result.asInstanceOf[Iterable[Option[BlockStatus]]] + blockManagerIds.zip(blockStatus).flatMap { case (blockManagerId, status) => + status.map { s => (blockManagerId, s) } + }.toMap + } + + /** + * Return a list of ids of existing blocks such that the ids match the given filter. NOTE: This + * is a potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, this invokes the master to query each block manager for the most + * updated block statuses. This is useful when the master is not informed of the given block + * by all block managers. + */ + def getMatchingBlockIds( + filter: BlockId => Boolean, + askSlaves: Boolean): Seq[BlockId] = { + val msg = GetMatchingBlockIds(filter, askSlaves) + val future = askDriverWithReply[Future[Seq[BlockId]]](msg) + Await.result(future, timeout) + } + /** Stop the driver actor, called only on the Spark driver node */ def stop() { if (driverActor != null) { diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index 378f4cadc17d7..c57b6e8391b13 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -94,9 +94,21 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus case GetStorageStatus => sender ! storageStatus + case GetBlockStatus(blockId, askSlaves) => + sender ! blockStatus(blockId, askSlaves) + + case GetMatchingBlockIds(filter, askSlaves) => + sender ! getMatchingBlockIds(filter, askSlaves) + case RemoveRdd(rddId) => sender ! removeRdd(rddId) + case RemoveShuffle(shuffleId) => + sender ! removeShuffle(shuffleId) + + case RemoveBroadcast(broadcastId, removeFromDriver) => + sender ! removeBroadcast(broadcastId, removeFromDriver) + case RemoveBlock(blockId) => removeBlockFromWorkers(blockId) sender ! true @@ -140,9 +152,41 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus // The dispatcher is used as an implicit argument into the Future sequence construction. import context.dispatcher val removeMsg = RemoveRdd(rddId) - Future.sequence(blockManagerInfo.values.map { bm => - bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] - }.toSeq) + Future.sequence( + blockManagerInfo.values.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + }.toSeq + ) + } + + private def removeShuffle(shuffleId: Int): Future[Seq[Boolean]] = { + // Nothing to do in the BlockManagerMasterActor data structures + import context.dispatcher + val removeMsg = RemoveShuffle(shuffleId) + Future.sequence( + blockManagerInfo.values.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Boolean] + }.toSeq + ) + } + + /** + * Delegate RemoveBroadcast messages to each BlockManager because the master may not notified + * of all broadcast blocks. If removeFromDriver is false, broadcast blocks are only removed + * from the executors, but not from the driver. + */ + private def removeBroadcast(broadcastId: Long, removeFromDriver: Boolean): Future[Seq[Int]] = { + // TODO: Consolidate usages of + import context.dispatcher + val removeMsg = RemoveBroadcast(broadcastId, removeFromDriver) + val requiredBlockManagers = blockManagerInfo.values.filter { info => + removeFromDriver || info.blockManagerId.executorId != "" + } + Future.sequence( + requiredBlockManagers.map { bm => + bm.slaveActor.ask(removeMsg)(akkaTimeout).mapTo[Int] + }.toSeq + ) } private def removeBlockManager(blockManagerId: BlockManagerId) { @@ -225,6 +269,61 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus }.toArray } + /** + * Return the block's status for all block managers, if any. NOTE: This is a + * potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, the master queries each block manager for the most updated block + * statuses. This is useful when the master is not informed of the given block by all block + * managers. + */ + private def blockStatus( + blockId: BlockId, + askSlaves: Boolean): Map[BlockManagerId, Future[Option[BlockStatus]]] = { + import context.dispatcher + val getBlockStatus = GetBlockStatus(blockId) + /* + * Rather than blocking on the block status query, master actor should simply return + * Futures to avoid potential deadlocks. This can arise if there exists a block manager + * that is also waiting for this master actor's response to a previous message. + */ + blockManagerInfo.values.map { info => + val blockStatusFuture = + if (askSlaves) { + info.slaveActor.ask(getBlockStatus)(akkaTimeout).mapTo[Option[BlockStatus]] + } else { + Future { info.getStatus(blockId) } + } + (info.blockManagerId, blockStatusFuture) + }.toMap + } + + /** + * Return the ids of blocks present in all the block managers that match the given filter. + * NOTE: This is a potentially expensive operation and should only be used for testing. + * + * If askSlaves is true, the master queries each block manager for the most updated block + * statuses. This is useful when the master is not informed of the given block by all block + * managers. + */ + private def getMatchingBlockIds( + filter: BlockId => Boolean, + askSlaves: Boolean): Future[Seq[BlockId]] = { + import context.dispatcher + val getMatchingBlockIds = GetMatchingBlockIds(filter) + Future.sequence( + blockManagerInfo.values.map { info => + val future = + if (askSlaves) { + info.slaveActor.ask(getMatchingBlockIds)(akkaTimeout).mapTo[Seq[BlockId]] + } else { + Future { info.blocks.keys.filter(filter).toSeq } + } + future + } + ).map(_.flatten.toSeq) + } + private def register(id: BlockManagerId, maxMemSize: Long, slaveActor: ActorRef) { if (!blockManagerInfo.contains(id)) { blockManagerIdByExecutor.get(id.executorId) match { @@ -334,6 +433,8 @@ private[spark] class BlockManagerInfo( logInfo("Registering block manager %s with %s RAM".format( blockManagerId.hostPort, Utils.bytesToString(maxMem))) + def getStatus(blockId: BlockId) = Option(_blocks.get(blockId)) + def updateLastSeenMs() { _lastSeenMs = System.currentTimeMillis() } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala index 8a36b5cc42dfd..2b53bf33b5fba 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMessages.scala @@ -34,6 +34,13 @@ private[storage] object BlockManagerMessages { // Remove all blocks belonging to a specific RDD. case class RemoveRdd(rddId: Int) extends ToBlockManagerSlave + // Remove all blocks belonging to a specific shuffle. + case class RemoveShuffle(shuffleId: Int) extends ToBlockManagerSlave + + // Remove all blocks belonging to a specific broadcast. + case class RemoveBroadcast(broadcastId: Long, removeFromDriver: Boolean = true) + extends ToBlockManagerSlave + ////////////////////////////////////////////////////////////////////////////////// // Messages from slaves to the master. @@ -80,7 +87,8 @@ private[storage] object BlockManagerMessages { } object UpdateBlockInfo { - def apply(blockManagerId: BlockManagerId, + def apply( + blockManagerId: BlockManagerId, blockId: BlockId, storageLevel: StorageLevel, memSize: Long, @@ -108,7 +116,13 @@ private[storage] object BlockManagerMessages { case object GetMemoryStatus extends ToBlockManagerMaster - case object ExpireDeadHosts extends ToBlockManagerMaster - case object GetStorageStatus extends ToBlockManagerMaster + + case class GetBlockStatus(blockId: BlockId, askSlaves: Boolean = true) + extends ToBlockManagerMaster + + case class GetMatchingBlockIds(filter: BlockId => Boolean, askSlaves: Boolean = true) + extends ToBlockManagerMaster + + case object ExpireDeadHosts extends ToBlockManagerMaster } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala index bcfb82d3c7336..6d4db064dff58 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerSlaveActor.scala @@ -17,8 +17,11 @@ package org.apache.spark.storage -import akka.actor.Actor +import scala.concurrent.Future +import akka.actor.{ActorRef, Actor} + +import org.apache.spark.{Logging, MapOutputTracker} import org.apache.spark.storage.BlockManagerMessages._ /** @@ -26,14 +29,59 @@ import org.apache.spark.storage.BlockManagerMessages._ * this is used to remove blocks from the slave's BlockManager. */ private[storage] -class BlockManagerSlaveActor(blockManager: BlockManager) extends Actor { - override def receive = { +class BlockManagerSlaveActor( + blockManager: BlockManager, + mapOutputTracker: MapOutputTracker) + extends Actor with Logging { + + import context.dispatcher + // Operations that involve removing blocks may be slow and should be done asynchronously + override def receive = { case RemoveBlock(blockId) => - blockManager.removeBlock(blockId) + doAsync[Boolean]("removing block " + blockId, sender) { + blockManager.removeBlock(blockId) + true + } case RemoveRdd(rddId) => - val numBlocksRemoved = blockManager.removeRdd(rddId) - sender ! numBlocksRemoved + doAsync[Int]("removing RDD " + rddId, sender) { + blockManager.removeRdd(rddId) + } + + case RemoveShuffle(shuffleId) => + doAsync[Boolean]("removing shuffle " + shuffleId, sender) { + if (mapOutputTracker != null) { + mapOutputTracker.unregisterShuffle(shuffleId) + } + blockManager.shuffleBlockManager.removeShuffle(shuffleId) + } + + case RemoveBroadcast(broadcastId, tellMaster) => + doAsync[Int]("removing broadcast " + broadcastId, sender) { + blockManager.removeBroadcast(broadcastId, tellMaster) + } + + case GetBlockStatus(blockId, _) => + sender ! blockManager.getStatus(blockId) + + case GetMatchingBlockIds(filter, _) => + sender ! blockManager.getMatchingBlockIds(filter) + } + + private def doAsync[T](actionMessage: String, responseActor: ActorRef)(body: => T) { + val future = Future { + logDebug(actionMessage) + body + } + future.onSuccess { case response => + logDebug("Done " + actionMessage + ", response is " + response) + responseActor ! response + logDebug("Sent response: " + response + " to " + responseActor) + } + future.onFailure { case t: Throwable => + logError("Error in " + actionMessage, t) + responseActor ! null.asInstanceOf[T] + } } } diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index f3e1c38744d78..7a24c8f57f43b 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -90,6 +90,20 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD def getFile(blockId: BlockId): File = getFile(blockId.name) + /** Check if disk block manager has a block. */ + def containsBlock(blockId: BlockId): Boolean = { + getBlockLocation(blockId).file.exists() + } + + /** List all the blocks currently stored on disk by the disk manager. */ + def getAllBlocks(): Seq[BlockId] = { + // Get all the files inside the array of array of directories + subDirs.flatten.filter(_ != null).flatMap { dir => + val files = dir.list() + if (files != null) files else Seq.empty + }.map(BlockId.apply) + } + /** Produces a unique block id and File suitable for intermediate results. */ def createTempBlock(): (TempBlockId, File) = { var blockId = new TempBlockId(UUID.randomUUID()) diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index bb07c8cb134cc..4cd4cdbd9909d 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -169,23 +169,43 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { throw new IllegalStateException("Failed to find shuffle block: " + id) } + /** Remove all the blocks / files and metadata related to a particular shuffle. */ + def removeShuffle(shuffleId: ShuffleId): Boolean = { + // Do not change the ordering of this, if shuffleStates should be removed only + // after the corresponding shuffle blocks have been removed + val cleaned = removeShuffleBlocks(shuffleId) + shuffleStates.remove(shuffleId) + cleaned + } + + /** Remove all the blocks / files related to a particular shuffle. */ + private def removeShuffleBlocks(shuffleId: ShuffleId): Boolean = { + shuffleStates.get(shuffleId) match { + case Some(state) => + if (consolidateShuffleFiles) { + for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { + file.delete() + } + } else { + for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { + val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) + blockManager.diskBlockManager.getFile(blockId).delete() + } + } + logInfo("Deleted all files for shuffle " + shuffleId) + true + case None => + logInfo("Could not find files for shuffle " + shuffleId + " for deleting") + false + } + } + private def physicalFileName(shuffleId: Int, bucketId: Int, fileId: Int) = { "merged_shuffle_%d_%d_%d".format(shuffleId, bucketId, fileId) } private def cleanup(cleanupTime: Long) { - shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => { - if (consolidateShuffleFiles) { - for (fileGroup <- state.allFileGroups; file <- fileGroup.files) { - file.delete() - } - } else { - for (mapId <- state.completedMapTasks; reduceId <- 0 until state.numBuckets) { - val blockId = new ShuffleBlockId(shuffleId, mapId, reduceId) - blockManager.diskBlockManager.getFile(blockId).delete() - } - } - }) + shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId)) } } diff --git a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala index 226ed2a132b00..a107c5182b3be 100644 --- a/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala +++ b/core/src/main/scala/org/apache/spark/storage/ThreadingTest.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ArrayBlockingQueue import akka.actor._ import util.Random -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.KryoSerializer @@ -48,7 +48,7 @@ private[spark] object ThreadingTest { val block = (1 to blockSize).map(_ => Random.nextInt()) val level = randomLevel() val startTime = System.currentTimeMillis() - manager.put(blockId, block.iterator, level, true) + manager.put(blockId, block.iterator, level, tellMaster = true) println("Pushed block " + blockId + " in " + (System.currentTimeMillis - startTime) + " ms") queue.add((blockId, block)) } @@ -101,7 +101,7 @@ private[spark] object ThreadingTest { conf) val blockManager = new BlockManager( "", actorSystem, blockManagerMaster, serializer, 1024 * 1024, conf, - new SecurityManager(conf)) + new SecurityManager(conf), new MapOutputTrackerMaster(conf)) val producers = (1 to numProducers).map(i => new ProducerThread(blockManager, i)) val consumers = producers.map(p => new ConsumerThread(blockManager, p.queue)) producers.foreach(_.start) diff --git a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala index 0448919e09161..7ebed5105b9fd 100644 --- a/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/MetadataCleaner.scala @@ -62,8 +62,8 @@ private[spark] class MetadataCleaner( private[spark] object MetadataCleanerType extends Enumeration { - val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, DAG_SCHEDULER, RESULT_TASK, - SHUFFLE_MAP_TASK, BLOCK_MANAGER, SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value + val MAP_OUTPUT_TRACKER, SPARK_CONTEXT, HTTP_BROADCAST, BLOCK_MANAGER, + SHUFFLE_BLOCK_MANAGER, BROADCAST_VARS = Value type MetadataCleanerType = Value @@ -78,15 +78,16 @@ private[spark] object MetadataCleaner { conf.getInt("spark.cleaner.ttl", -1) } - def getDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType): Int = - { - conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString) - .toInt + def getDelaySeconds( + conf: SparkConf, + cleanerType: MetadataCleanerType.MetadataCleanerType): Int = { + conf.get(MetadataCleanerType.systemProperty(cleanerType), getDelaySeconds(conf).toString).toInt } - def setDelaySeconds(conf: SparkConf, cleanerType: MetadataCleanerType.MetadataCleanerType, - delay: Int) - { + def setDelaySeconds( + conf: SparkConf, + cleanerType: MetadataCleanerType.MetadataCleanerType, + delay: Int) { conf.set(MetadataCleanerType.systemProperty(cleanerType), delay.toString) } diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala index ddbd084ed7f01..8de75ba9a9c92 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashMap.scala @@ -17,48 +17,54 @@ package org.apache.spark.util +import java.util.Set +import java.util.Map.Entry import java.util.concurrent.ConcurrentHashMap -import scala.collection.JavaConversions -import scala.collection.immutable -import scala.collection.mutable.Map +import scala.collection.{JavaConversions, mutable} import org.apache.spark.Logging +private[spark] case class TimeStampedValue[V](value: V, timestamp: Long) + /** * This is a custom implementation of scala.collection.mutable.Map which stores the insertion * timestamp along with each key-value pair. If specified, the timestamp of each pair can be * updated every time it is accessed. Key-value pairs whose timestamp are older than a particular * threshold time can then be removed using the clearOldValues method. This is intended to * be a drop-in replacement of scala.collection.mutable.HashMap. - * @param updateTimeStampOnGet When enabled, the timestamp of a pair will be - * updated when it is accessed + * + * @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed */ -class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) - extends Map[A, B]() with Logging { - val internalMap = new ConcurrentHashMap[A, (B, Long)]() +private[spark] class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) + extends mutable.Map[A, B]() with Logging { + + private val internalMap = new ConcurrentHashMap[A, TimeStampedValue[B]]() def get(key: A): Option[B] = { val value = internalMap.get(key) if (value != null && updateTimeStampOnGet) { - internalMap.replace(key, value, (value._1, currentTime)) + internalMap.replace(key, value, TimeStampedValue(value.value, currentTime)) } - Option(value).map(_._1) + Option(value).map(_.value) } def iterator: Iterator[(A, B)] = { - val jIterator = internalMap.entrySet().iterator() - JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue._1)) + val jIterator = getEntrySet.iterator + JavaConversions.asScalaIterator(jIterator).map(kv => (kv.getKey, kv.getValue.value)) } - override def + [B1 >: B](kv: (A, B1)): Map[A, B1] = { + def getEntrySet: Set[Entry[A, TimeStampedValue[B]]] = internalMap.entrySet + + override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = { val newMap = new TimeStampedHashMap[A, B1] - newMap.internalMap.putAll(this.internalMap) - newMap.internalMap.put(kv._1, (kv._2, currentTime)) + val oldInternalMap = this.internalMap.asInstanceOf[ConcurrentHashMap[A, TimeStampedValue[B1]]] + newMap.internalMap.putAll(oldInternalMap) + kv match { case (a, b) => newMap.internalMap.put(a, TimeStampedValue(b, currentTime)) } newMap } - override def - (key: A): Map[A, B] = { + override def - (key: A): mutable.Map[A, B] = { val newMap = new TimeStampedHashMap[A, B] newMap.internalMap.putAll(this.internalMap) newMap.internalMap.remove(key) @@ -66,17 +72,10 @@ class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) } override def += (kv: (A, B)): this.type = { - internalMap.put(kv._1, (kv._2, currentTime)) + kv match { case (a, b) => internalMap.put(a, TimeStampedValue(b, currentTime)) } this } - // Should we return previous value directly or as Option ? - def putIfAbsent(key: A, value: B): Option[B] = { - val prev = internalMap.putIfAbsent(key, (value, currentTime)) - if (prev != null) Some(prev._1) else None - } - - override def -= (key: A): this.type = { internalMap.remove(key) this @@ -87,53 +86,65 @@ class TimeStampedHashMap[A, B](updateTimeStampOnGet: Boolean = false) } override def apply(key: A): B = { - val value = internalMap.get(key) - if (value == null) throw new NoSuchElementException() - value._1 + get(key).getOrElse { throw new NoSuchElementException() } } - override def filter(p: ((A, B)) => Boolean): Map[A, B] = { - JavaConversions.mapAsScalaConcurrentMap(internalMap).map(kv => (kv._1, kv._2._1)).filter(p) + override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = { + JavaConversions.mapAsScalaConcurrentMap(internalMap) + .map { case (k, TimeStampedValue(v, t)) => (k, v) } + .filter(p) } - override def empty: Map[A, B] = new TimeStampedHashMap[A, B]() + override def empty: mutable.Map[A, B] = new TimeStampedHashMap[A, B]() override def size: Int = internalMap.size override def foreach[U](f: ((A, B)) => U) { - val iterator = internalMap.entrySet().iterator() - while(iterator.hasNext) { - val entry = iterator.next() - val kv = (entry.getKey, entry.getValue._1) + val it = getEntrySet.iterator + while(it.hasNext) { + val entry = it.next() + val kv = (entry.getKey, entry.getValue.value) f(kv) } } - def toMap: immutable.Map[A, B] = iterator.toMap + def putIfAbsent(key: A, value: B): Option[B] = { + val prev = internalMap.putIfAbsent(key, TimeStampedValue(value, currentTime)) + Option(prev).map(_.value) + } + + def putAll(map: Map[A, B]) { + map.foreach { case (k, v) => update(k, v) } + } + + def toMap: Map[A, B] = iterator.toMap - /** - * Removes old key-value pairs that have timestamp earlier than `threshTime`, - * calling the supplied function on each such entry before removing. - */ def clearOldValues(threshTime: Long, f: (A, B) => Unit) { - val iterator = internalMap.entrySet().iterator() - while (iterator.hasNext) { - val entry = iterator.next() - if (entry.getValue._2 < threshTime) { - f(entry.getKey, entry.getValue._1) + val it = getEntrySet.iterator + while (it.hasNext) { + val entry = it.next() + if (entry.getValue.timestamp < threshTime) { + f(entry.getKey, entry.getValue.value) logDebug("Removing key " + entry.getKey) - iterator.remove() + it.remove() } } } - /** - * Removes old key-value pairs that have timestamp earlier than `threshTime` - */ + /** Removes old key-value pairs that have timestamp earlier than `threshTime`. */ def clearOldValues(threshTime: Long) { clearOldValues(threshTime, (_, _) => ()) } - private def currentTime: Long = System.currentTimeMillis() + private def currentTime: Long = System.currentTimeMillis + // For testing + + def getTimeStampedValue(key: A): Option[TimeStampedValue[B]] = { + Option(internalMap.get(key)) + } + + def getTimestamp(key: A): Option[Long] = { + getTimeStampedValue(key).map(_.timestamp) + } } diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala new file mode 100644 index 0000000000000..b65017d6806c6 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala @@ -0,0 +1,170 @@ +/* + * 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.util + +import java.lang.ref.WeakReference +import java.util.concurrent.atomic.AtomicInteger + +import scala.collection.mutable + +import org.apache.spark.Logging + +/** + * A wrapper of TimeStampedHashMap that ensures the values are weakly referenced and timestamped. + * + * If the value is garbage collected and the weak reference is null, get() will return a + * non-existent value. These entries are removed from the map periodically (every N inserts), as + * their values are no longer strongly reachable. Further, key-value pairs whose timestamps are + * older than a particular threshold can be removed using the clearOldValues method. + * + * TimeStampedWeakValueHashMap exposes a scala.collection.mutable.Map interface, which allows it + * to be a drop-in replacement for Scala HashMaps. Internally, it uses a Java ConcurrentHashMap, + * so all operations on this HashMap are thread-safe. + * + * @param updateTimeStampOnGet Whether timestamp of a pair will be updated when it is accessed. + */ +private[spark] class TimeStampedWeakValueHashMap[A, B](updateTimeStampOnGet: Boolean = false) + extends mutable.Map[A, B]() with Logging { + + import TimeStampedWeakValueHashMap._ + + private val internalMap = new TimeStampedHashMap[A, WeakReference[B]](updateTimeStampOnGet) + private val insertCount = new AtomicInteger(0) + + /** Return a map consisting only of entries whose values are still strongly reachable. */ + private def nonNullReferenceMap = internalMap.filter { case (_, ref) => ref.get != null } + + def get(key: A): Option[B] = internalMap.get(key) + + def iterator: Iterator[(A, B)] = nonNullReferenceMap.iterator + + override def + [B1 >: B](kv: (A, B1)): mutable.Map[A, B1] = { + val newMap = new TimeStampedWeakValueHashMap[A, B1] + val oldMap = nonNullReferenceMap.asInstanceOf[mutable.Map[A, WeakReference[B1]]] + newMap.internalMap.putAll(oldMap.toMap) + newMap.internalMap += kv + newMap + } + + override def - (key: A): mutable.Map[A, B] = { + val newMap = new TimeStampedWeakValueHashMap[A, B] + newMap.internalMap.putAll(nonNullReferenceMap.toMap) + newMap.internalMap -= key + newMap + } + + override def += (kv: (A, B)): this.type = { + internalMap += kv + if (insertCount.incrementAndGet() % CLEAR_NULL_VALUES_INTERVAL == 0) { + clearNullValues() + } + this + } + + override def -= (key: A): this.type = { + internalMap -= key + this + } + + override def update(key: A, value: B) = this += ((key, value)) + + override def apply(key: A): B = internalMap.apply(key) + + override def filter(p: ((A, B)) => Boolean): mutable.Map[A, B] = nonNullReferenceMap.filter(p) + + override def empty: mutable.Map[A, B] = new TimeStampedWeakValueHashMap[A, B]() + + override def size: Int = internalMap.size + + override def foreach[U](f: ((A, B)) => U) = nonNullReferenceMap.foreach(f) + + def putIfAbsent(key: A, value: B): Option[B] = internalMap.putIfAbsent(key, value) + + def toMap: Map[A, B] = iterator.toMap + + /** Remove old key-value pairs with timestamps earlier than `threshTime`. */ + def clearOldValues(threshTime: Long) = internalMap.clearOldValues(threshTime) + + /** Remove entries with values that are no longer strongly reachable. */ + def clearNullValues() { + val it = internalMap.getEntrySet.iterator + while (it.hasNext) { + val entry = it.next() + if (entry.getValue.value.get == null) { + logDebug("Removing key " + entry.getKey + " because it is no longer strongly reachable.") + it.remove() + } + } + } + + // For testing + + def getTimestamp(key: A): Option[Long] = { + internalMap.getTimeStampedValue(key).map(_.timestamp) + } + + def getReference(key: A): Option[WeakReference[B]] = { + internalMap.getTimeStampedValue(key).map(_.value) + } +} + +/** + * Helper methods for converting to and from WeakReferences. + */ +private object TimeStampedWeakValueHashMap { + + // Number of inserts after which entries with null references are removed + val CLEAR_NULL_VALUES_INTERVAL = 100 + + /* Implicit conversion methods to WeakReferences. */ + + implicit def toWeakReference[V](v: V): WeakReference[V] = new WeakReference[V](v) + + implicit def toWeakReferenceTuple[K, V](kv: (K, V)): (K, WeakReference[V]) = { + kv match { case (k, v) => (k, toWeakReference(v)) } + } + + implicit def toWeakReferenceFunction[K, V, R](p: ((K, V)) => R): ((K, WeakReference[V])) => R = { + (kv: (K, WeakReference[V])) => p(kv) + } + + /* Implicit conversion methods from WeakReferences. */ + + implicit def fromWeakReference[V](ref: WeakReference[V]): V = ref.get + + implicit def fromWeakReferenceOption[V](v: Option[WeakReference[V]]): Option[V] = { + v match { + case Some(ref) => Option(fromWeakReference(ref)) + case None => None + } + } + + implicit def fromWeakReferenceTuple[K, V](kv: (K, WeakReference[V])): (K, V) = { + kv match { case (k, v) => (k, fromWeakReference(v)) } + } + + implicit def fromWeakReferenceIterator[K, V]( + it: Iterator[(K, WeakReference[V])]): Iterator[(K, V)] = { + it.map(fromWeakReferenceTuple) + } + + implicit def fromWeakReferenceMap[K, V]( + map: mutable.Map[K, WeakReference[V]]) : mutable.Map[K, V] = { + mutable.Map(map.mapValues(fromWeakReference).toSeq: _*) + } +} diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 4435b21a7505e..59da51f3e0297 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -499,10 +499,10 @@ private[spark] object Utils extends Logging { private val hostPortParseResults = new ConcurrentHashMap[String, (String, Int)]() def parseHostPort(hostPort: String): (String, Int) = { - { - // Check cache first. - val cached = hostPortParseResults.get(hostPort) - if (cached != null) return cached + // Check cache first. + val cached = hostPortParseResults.get(hostPort) + if (cached != null) { + return cached } val indx: Int = hostPort.lastIndexOf(':') diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala index d2e303d81c4c8..c5f24c66ce0c1 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -56,7 +56,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = securityManagerBad) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) @@ -93,7 +93,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) @@ -147,7 +147,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = goodconf, securityManager = securityManagerGood) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) @@ -200,7 +200,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) diff --git a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala index 96ba3929c1685..c9936256a5b95 100644 --- a/core/src/test/scala/org/apache/spark/BroadcastSuite.scala +++ b/core/src/test/scala/org/apache/spark/BroadcastSuite.scala @@ -19,68 +19,297 @@ package org.apache.spark import org.scalatest.FunSuite -class BroadcastSuite extends FunSuite with LocalSparkContext { +import org.apache.spark.storage._ +import org.apache.spark.broadcast.{Broadcast, HttpBroadcast} +import org.apache.spark.storage.BroadcastBlockId +class BroadcastSuite extends FunSuite with LocalSparkContext { - override def afterEach() { - super.afterEach() - System.clearProperty("spark.broadcast.factory") - } + private val httpConf = broadcastConf("HttpBroadcastFactory") + private val torrentConf = broadcastConf("TorrentBroadcastFactory") test("Using HttpBroadcast locally") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") - sc = new SparkContext("local", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 2).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === Set((1, 10), (2, 10))) + sc = new SparkContext("local", "test", httpConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 2).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === Set((1, 10), (2, 10))) } test("Accessing HttpBroadcast variables from multiple threads") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") - sc = new SparkContext("local[10]", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 10).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to 10).map(x => (x, 10)).toSet) + sc = new SparkContext("local[10]", "test", httpConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 10).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to 10).map(x => (x, 10)).toSet) } test("Accessing HttpBroadcast variables in a local cluster") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.HttpBroadcastFactory") val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to numSlaves).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", httpConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to numSlaves).map(x => (x, 10)).toSet) } test("Using TorrentBroadcast locally") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") - sc = new SparkContext("local", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 2).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === Set((1, 10), (2, 10))) + sc = new SparkContext("local", "test", torrentConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 2).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === Set((1, 10), (2, 10))) } test("Accessing TorrentBroadcast variables from multiple threads") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") - sc = new SparkContext("local[10]", "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to 10).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to 10).map(x => (x, 10)).toSet) + sc = new SparkContext("local[10]", "test", torrentConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to 10).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to 10).map(x => (x, 10)).toSet) } test("Accessing TorrentBroadcast variables in a local cluster") { - System.setProperty("spark.broadcast.factory", "org.apache.spark.broadcast.TorrentBroadcastFactory") val numSlaves = 4 - sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test") - val list = List(1, 2, 3, 4) - val listBroadcast = sc.broadcast(list) - val results = sc.parallelize(1 to numSlaves).map(x => (x, listBroadcast.value.sum)) - assert(results.collect.toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + sc = new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", torrentConf) + val list = List[Int](1, 2, 3, 4) + val broadcast = sc.broadcast(list) + val results = sc.parallelize(1 to numSlaves).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to numSlaves).map(x => (x, 10)).toSet) + } + + test("Unpersisting HttpBroadcast on executors only in local mode") { + testUnpersistHttpBroadcast(distributed = false, removeFromDriver = false) + } + + test("Unpersisting HttpBroadcast on executors and driver in local mode") { + testUnpersistHttpBroadcast(distributed = false, removeFromDriver = true) + } + + test("Unpersisting HttpBroadcast on executors only in distributed mode") { + testUnpersistHttpBroadcast(distributed = true, removeFromDriver = false) + } + + test("Unpersisting HttpBroadcast on executors and driver in distributed mode") { + testUnpersistHttpBroadcast(distributed = true, removeFromDriver = true) + } + + test("Unpersisting TorrentBroadcast on executors only in local mode") { + testUnpersistTorrentBroadcast(distributed = false, removeFromDriver = false) + } + + test("Unpersisting TorrentBroadcast on executors and driver in local mode") { + testUnpersistTorrentBroadcast(distributed = false, removeFromDriver = true) + } + + test("Unpersisting TorrentBroadcast on executors only in distributed mode") { + testUnpersistTorrentBroadcast(distributed = true, removeFromDriver = false) + } + + test("Unpersisting TorrentBroadcast on executors and driver in distributed mode") { + testUnpersistTorrentBroadcast(distributed = true, removeFromDriver = true) + } + /** + * Verify the persistence of state associated with an HttpBroadcast in either local mode or + * local-cluster mode (when distributed = true). + * + * This test creates a broadcast variable, uses it on all executors, and then unpersists it. + * In between each step, this test verifies that the broadcast blocks and the broadcast file + * are present only on the expected nodes. + */ + private def testUnpersistHttpBroadcast(distributed: Boolean, removeFromDriver: Boolean) { + val numSlaves = if (distributed) 2 else 0 + + def getBlockIds(id: Long) = Seq[BroadcastBlockId](BroadcastBlockId(id)) + + // Verify that the broadcast file is created, and blocks are persisted only on the driver + def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + assert(blockIds.size === 1) + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + assert(statuses.size === 1) + statuses.head match { case (bm, status) => + assert(bm.executorId === "", "Block should only be on the driver") + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store on the driver") + assert(status.diskSize === 0, "Block should not be in disk store on the driver") + } + if (distributed) { + // this file is only generated in distributed mode + assert(HttpBroadcast.getFile(blockIds.head.broadcastId).exists, "Broadcast file not found!") + } + } + + // Verify that blocks are persisted in both the executors and the driver + def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + assert(blockIds.size === 1) + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + assert(statuses.size === numSlaves + 1) + statuses.foreach { case (_, status) => + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store") + assert(status.diskSize === 0, "Block should not be in disk store") + } + } + + // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver + // is true. In the latter case, also verify that the broadcast file is deleted on the driver. + def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + assert(blockIds.size === 1) + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + val expectedNumBlocks = if (removeFromDriver) 0 else 1 + val possiblyNot = if (removeFromDriver) "" else " not" + assert(statuses.size === expectedNumBlocks, + "Block should%s be unpersisted on the driver".format(possiblyNot)) + if (distributed && removeFromDriver) { + // this file is only generated in distributed mode + assert(!HttpBroadcast.getFile(blockIds.head.broadcastId).exists, + "Broadcast file should%s be deleted".format(possiblyNot)) + } + } + + testUnpersistBroadcast(distributed, numSlaves, httpConf, getBlockIds, afterCreation, + afterUsingBroadcast, afterUnpersist, removeFromDriver) + } + + /** + * Verify the persistence of state associated with an TorrentBroadcast in a local-cluster. + * + * This test creates a broadcast variable, uses it on all executors, and then unpersists it. + * In between each step, this test verifies that the broadcast blocks are present only on the + * expected nodes. + */ + private def testUnpersistTorrentBroadcast(distributed: Boolean, removeFromDriver: Boolean) { + val numSlaves = if (distributed) 2 else 0 + + def getBlockIds(id: Long) = { + val broadcastBlockId = BroadcastBlockId(id) + val metaBlockId = BroadcastBlockId(id, "meta") + // Assume broadcast value is small enough to fit into 1 piece + val pieceBlockId = BroadcastBlockId(id, "piece0") + if (distributed) { + // the metadata and piece blocks are generated only in distributed mode + Seq[BroadcastBlockId](broadcastBlockId, metaBlockId, pieceBlockId) + } else { + Seq[BroadcastBlockId](broadcastBlockId) + } + } + + // Verify that blocks are persisted only on the driver + def afterCreation(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + blockIds.foreach { blockId => + val statuses = bmm.getBlockStatus(blockIds.head, askSlaves = true) + assert(statuses.size === 1) + statuses.head match { case (bm, status) => + assert(bm.executorId === "", "Block should only be on the driver") + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store on the driver") + assert(status.diskSize === 0, "Block should not be in disk store on the driver") + } + } + } + + // Verify that blocks are persisted in both the executors and the driver + def afterUsingBroadcast(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + blockIds.foreach { blockId => + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) + if (blockId.field == "meta") { + // Meta data is only on the driver + assert(statuses.size === 1) + statuses.head match { case (bm, _) => assert(bm.executorId === "") } + } else { + // Other blocks are on both the executors and the driver + assert(statuses.size === numSlaves + 1, + blockId + " has " + statuses.size + " statuses: " + statuses.mkString(",")) + statuses.foreach { case (_, status) => + assert(status.storageLevel === StorageLevel.MEMORY_AND_DISK) + assert(status.memSize > 0, "Block should be in memory store") + assert(status.diskSize === 0, "Block should not be in disk store") + } + } + } + } + + // Verify that blocks are unpersisted on all executors, and on all nodes if removeFromDriver + // is true. + def afterUnpersist(blockIds: Seq[BroadcastBlockId], bmm: BlockManagerMaster) { + val expectedNumBlocks = if (removeFromDriver) 0 else 1 + val possiblyNot = if (removeFromDriver) "" else " not" + blockIds.foreach { blockId => + val statuses = bmm.getBlockStatus(blockId, askSlaves = true) + assert(statuses.size === expectedNumBlocks, + "Block should%s be unpersisted on the driver".format(possiblyNot)) + } + } + + testUnpersistBroadcast(distributed, numSlaves, torrentConf, getBlockIds, afterCreation, + afterUsingBroadcast, afterUnpersist, removeFromDriver) + } + + /** + * This test runs in 4 steps: + * + * 1) Create broadcast variable, and verify that all state is persisted on the driver. + * 2) Use the broadcast variable on all executors, and verify that all state is persisted + * on both the driver and the executors. + * 3) Unpersist the broadcast, and verify that all state is removed where they should be. + * 4) [Optional] If removeFromDriver is false, we verify that the broadcast is re-usable. + */ + private def testUnpersistBroadcast( + distributed: Boolean, + numSlaves: Int, // used only when distributed = true + broadcastConf: SparkConf, + getBlockIds: Long => Seq[BroadcastBlockId], + afterCreation: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + afterUsingBroadcast: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + afterUnpersist: (Seq[BroadcastBlockId], BlockManagerMaster) => Unit, + removeFromDriver: Boolean) { + + sc = if (distributed) { + new SparkContext("local-cluster[%d, 1, 512]".format(numSlaves), "test", broadcastConf) + } else { + new SparkContext("local", "test", broadcastConf) + } + val blockManagerMaster = sc.env.blockManager.master + val list = List[Int](1, 2, 3, 4) + + // Create broadcast variable + val broadcast = sc.broadcast(list) + val blocks = getBlockIds(broadcast.id) + afterCreation(blocks, blockManagerMaster) + + // Use broadcast variable on all executors + val partitions = 10 + assert(partitions > numSlaves) + val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) + afterUsingBroadcast(blocks, blockManagerMaster) + + // Unpersist broadcast + if (removeFromDriver) { + broadcast.destroy(blocking = true) + } else { + broadcast.unpersist(blocking = true) + } + afterUnpersist(blocks, blockManagerMaster) + + // If the broadcast is removed from driver, all subsequent uses of the broadcast variable + // should throw SparkExceptions. Otherwise, the result should be the same as before. + if (removeFromDriver) { + // Using this variable on the executors crashes them, which hangs the test. + // Instead, crash the driver by directly accessing the broadcast value. + intercept[SparkException] { broadcast.value } + intercept[SparkException] { broadcast.unpersist() } + intercept[SparkException] { broadcast.destroy(blocking = true) } + } else { + val results = sc.parallelize(1 to partitions, partitions).map(x => (x, broadcast.value.sum)) + assert(results.collect().toSet === (1 to partitions).map(x => (x, list.sum)).toSet) + } } + /** Helper method to create a SparkConf that uses the given broadcast factory. */ + private def broadcastConf(factoryName: String): SparkConf = { + val conf = new SparkConf + conf.set("spark.broadcast.factory", "org.apache.spark.broadcast.%s".format(factoryName)) + conf + } } diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala new file mode 100644 index 0000000000000..e50981cf6fb20 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -0,0 +1,415 @@ +/* + * 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 + +import java.lang.ref.WeakReference + +import scala.collection.mutable.{HashSet, SynchronizedSet} +import scala.util.Random + +import org.scalatest.{BeforeAndAfter, FunSuite} +import org.scalatest.concurrent.Eventually +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +import org.apache.spark.SparkContext._ +import org.apache.spark.rdd.RDD +import org.apache.spark.storage.{BlockId, BroadcastBlockId, RDDBlockId, ShuffleBlockId} + +class ContextCleanerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { + + implicit val defaultTimeout = timeout(10000 millis) + val conf = new SparkConf() + .setMaster("local[2]") + .setAppName("ContextCleanerSuite") + .set("spark.cleaner.referenceTracking.blocking", "true") + + before { + sc = new SparkContext(conf) + } + + after { + if (sc != null) { + sc.stop() + sc = null + } + } + + + test("cleanup RDD") { + val rdd = newRDD.persist() + val collected = rdd.collect().toList + val tester = new CleanerTester(sc, rddIds = Seq(rdd.id)) + + // Explicit cleanup + cleaner.doCleanupRDD(rdd.id, blocking = true) + tester.assertCleanup() + + // Verify that RDDs can be re-executed after cleaning up + assert(rdd.collect().toList === collected) + } + + test("cleanup shuffle") { + val (rdd, shuffleDeps) = newRDDWithShuffleDependencies + val collected = rdd.collect().toList + val tester = new CleanerTester(sc, shuffleIds = shuffleDeps.map(_.shuffleId)) + + // Explicit cleanup + shuffleDeps.foreach(s => cleaner.doCleanupShuffle(s.shuffleId, blocking = true)) + tester.assertCleanup() + + // Verify that shuffles can be re-executed after cleaning up + assert(rdd.collect().toList === collected) + } + + test("cleanup broadcast") { + val broadcast = newBroadcast + val tester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) + + // Explicit cleanup + cleaner.doCleanupBroadcast(broadcast.id, blocking = true) + tester.assertCleanup() + } + + test("automatically cleanup RDD") { + var rdd = newRDD.persist() + rdd.count() + + // Test that GC does not cause RDD cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, rddIds = Seq(rdd.id)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes RDD cleanup after dereferencing the RDD + val postGCTester = new CleanerTester(sc, rddIds = Seq(rdd.id)) + rdd = null // Make RDD out of scope + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup shuffle") { + var rdd = newShuffleRDD + rdd.count() + + // Test that GC does not cause shuffle cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes shuffle cleanup after dereferencing the RDD + val postGCTester = new CleanerTester(sc, shuffleIds = Seq(0)) + rdd = null // Make RDD out of scope, so that corresponding shuffle goes out of scope + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup broadcast") { + var broadcast = newBroadcast + + // Test that GC does not cause broadcast cleanup due to a strong reference + val preGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC causes broadcast cleanup after dereferencing the broadcast variable + val postGCTester = new CleanerTester(sc, broadcastIds = Seq(broadcast.id)) + broadcast = null // Make broadcast variable out of scope + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup RDD + shuffle + broadcast") { + val numRdds = 100 + val numBroadcasts = 4 // Broadcasts are more costly + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddIds = sc.persistentRdds.keys.toSeq + val shuffleIds = 0 until sc.newShuffleId + val broadcastIds = 0L until numBroadcasts + + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC triggers the cleanup of all variables after the dereferencing them + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + broadcastBuffer.clear() + rddBuffer.clear() + runGC() + postGCTester.assertCleanup() + } + + test("automatically cleanup RDD + shuffle + broadcast in distributed mode") { + sc.stop() + + val conf2 = new SparkConf() + .setMaster("local-cluster[2, 1, 512]") + .setAppName("ContextCleanerSuite") + .set("spark.cleaner.referenceTracking.blocking", "true") + sc = new SparkContext(conf2) + + val numRdds = 10 + val numBroadcasts = 4 // Broadcasts are more costly + val rddBuffer = (1 to numRdds).map(i => randomRdd).toBuffer + val broadcastBuffer = (1 to numBroadcasts).map(i => randomBroadcast).toBuffer + val rddIds = sc.persistentRdds.keys.toSeq + val shuffleIds = 0 until sc.newShuffleId + val broadcastIds = 0L until numBroadcasts + + val preGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + runGC() + intercept[Exception] { + preGCTester.assertCleanup()(timeout(1000 millis)) + } + + // Test that GC triggers the cleanup of all variables after the dereferencing them + val postGCTester = new CleanerTester(sc, rddIds, shuffleIds, broadcastIds) + broadcastBuffer.clear() + rddBuffer.clear() + runGC() + postGCTester.assertCleanup() + } + + //------ Helper functions ------ + + def newRDD = sc.makeRDD(1 to 10) + def newPairRDD = newRDD.map(_ -> 1) + def newShuffleRDD = newPairRDD.reduceByKey(_ + _) + def newBroadcast = sc.broadcast(1 to 100) + def newRDDWithShuffleDependencies: (RDD[_], Seq[ShuffleDependency[_, _]]) = { + def getAllDependencies(rdd: RDD[_]): Seq[Dependency[_]] = { + rdd.dependencies ++ rdd.dependencies.flatMap { dep => + getAllDependencies(dep.rdd) + } + } + val rdd = newShuffleRDD + + // Get all the shuffle dependencies + val shuffleDeps = getAllDependencies(rdd) + .filter(_.isInstanceOf[ShuffleDependency[_, _]]) + .map(_.asInstanceOf[ShuffleDependency[_, _]]) + (rdd, shuffleDeps) + } + + def randomRdd = { + val rdd: RDD[_] = Random.nextInt(3) match { + case 0 => newRDD + case 1 => newShuffleRDD + case 2 => newPairRDD.join(newPairRDD) + } + if (Random.nextBoolean()) rdd.persist() + rdd.count() + rdd + } + + def randomBroadcast = { + sc.broadcast(Random.nextInt(Int.MaxValue)) + } + + /** Run GC and make sure it actually has run */ + def runGC() { + val weakRef = new WeakReference(new Object()) + val startTime = System.currentTimeMillis + System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. + // Wait until a weak reference object has been GCed + while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + System.gc() + Thread.sleep(200) + } + } + + def cleaner = sc.cleaner.get +} + + +/** Class to test whether RDDs, shuffles, etc. have been successfully cleaned. */ +class CleanerTester( + sc: SparkContext, + rddIds: Seq[Int] = Seq.empty, + shuffleIds: Seq[Int] = Seq.empty, + broadcastIds: Seq[Long] = Seq.empty) + extends Logging { + + val toBeCleanedRDDIds = new HashSet[Int] with SynchronizedSet[Int] ++= rddIds + val toBeCleanedShuffleIds = new HashSet[Int] with SynchronizedSet[Int] ++= shuffleIds + val toBeCleanedBroadcstIds = new HashSet[Long] with SynchronizedSet[Long] ++= broadcastIds + val isDistributed = !sc.isLocal + + val cleanerListener = new CleanerListener { + def rddCleaned(rddId: Int): Unit = { + toBeCleanedRDDIds -= rddId + logInfo("RDD "+ rddId + " cleaned") + } + + def shuffleCleaned(shuffleId: Int): Unit = { + toBeCleanedShuffleIds -= shuffleId + logInfo("Shuffle " + shuffleId + " cleaned") + } + + def broadcastCleaned(broadcastId: Long): Unit = { + toBeCleanedBroadcstIds -= broadcastId + logInfo("Broadcast" + broadcastId + " cleaned") + } + } + + val MAX_VALIDATION_ATTEMPTS = 10 + val VALIDATION_ATTEMPT_INTERVAL = 100 + + logInfo("Attempting to validate before cleanup:\n" + uncleanedResourcesToString) + preCleanupValidate() + sc.cleaner.get.attachListener(cleanerListener) + + /** Assert that all the stuff has been cleaned up */ + def assertCleanup()(implicit waitTimeout: Eventually.Timeout) { + try { + eventually(waitTimeout, interval(100 millis)) { + assert(isAllCleanedUp) + } + postCleanupValidate() + } finally { + logInfo("Resources left from cleaning up:\n" + uncleanedResourcesToString) + } + } + + /** Verify that RDDs, shuffles, etc. occupy resources */ + private def preCleanupValidate() { + assert(rddIds.nonEmpty || shuffleIds.nonEmpty || broadcastIds.nonEmpty, "Nothing to cleanup") + + // Verify the RDDs have been persisted and blocks are present + rddIds.foreach { rddId => + assert( + sc.persistentRdds.contains(rddId), + "RDD " + rddId + " have not been persisted, cannot start cleaner test" + ) + + assert( + !getRDDBlocks(rddId).isEmpty, + "Blocks of RDD " + rddId + " cannot be found in block manager, " + + "cannot start cleaner test" + ) + } + + // Verify the shuffle ids are registered and blocks are present + shuffleIds.foreach { shuffleId => + assert( + mapOutputTrackerMaster.containsShuffle(shuffleId), + "Shuffle " + shuffleId + " have not been registered, cannot start cleaner test" + ) + + assert( + !getShuffleBlocks(shuffleId).isEmpty, + "Blocks of shuffle " + shuffleId + " cannot be found in block manager, " + + "cannot start cleaner test" + ) + } + + // Verify that the broadcast blocks are present + broadcastIds.foreach { broadcastId => + assert( + !getBroadcastBlocks(broadcastId).isEmpty, + "Blocks of broadcast " + broadcastId + "cannot be found in block manager, " + + "cannot start cleaner test" + ) + } + } + + /** + * Verify that RDDs, shuffles, etc. do not occupy resources. Tests multiple times as there is + * as there is not guarantee on how long it will take clean up the resources. + */ + private def postCleanupValidate() { + // Verify the RDDs have been persisted and blocks are present + rddIds.foreach { rddId => + assert( + !sc.persistentRdds.contains(rddId), + "RDD " + rddId + " was not cleared from sc.persistentRdds" + ) + + assert( + getRDDBlocks(rddId).isEmpty, + "Blocks of RDD " + rddId + " were not cleared from block manager" + ) + } + + // Verify the shuffle ids are registered and blocks are present + shuffleIds.foreach { shuffleId => + assert( + !mapOutputTrackerMaster.containsShuffle(shuffleId), + "Shuffle " + shuffleId + " was not deregistered from map output tracker" + ) + + assert( + getShuffleBlocks(shuffleId).isEmpty, + "Blocks of shuffle " + shuffleId + " were not cleared from block manager" + ) + } + + // Verify that the broadcast blocks are present + broadcastIds.foreach { broadcastId => + assert( + getBroadcastBlocks(broadcastId).isEmpty, + "Blocks of broadcast " + broadcastId + " were not cleared from block manager" + ) + } + } + + private def uncleanedResourcesToString = { + s""" + |\tRDDs = ${toBeCleanedRDDIds.toSeq.sorted.mkString("[", ", ", "]")} + |\tShuffles = ${toBeCleanedShuffleIds.toSeq.sorted.mkString("[", ", ", "]")} + |\tBroadcasts = ${toBeCleanedBroadcstIds.toSeq.sorted.mkString("[", ", ", "]")} + """.stripMargin + } + + private def isAllCleanedUp = + toBeCleanedRDDIds.isEmpty && + toBeCleanedShuffleIds.isEmpty && + toBeCleanedBroadcstIds.isEmpty + + private def getRDDBlocks(rddId: Int): Seq[BlockId] = { + blockManager.master.getMatchingBlockIds( _ match { + case RDDBlockId(`rddId`, _) => true + case _ => false + }, askSlaves = true) + } + + private def getShuffleBlocks(shuffleId: Int): Seq[BlockId] = { + blockManager.master.getMatchingBlockIds( _ match { + case ShuffleBlockId(`shuffleId`, _, _) => true + case _ => false + }, askSlaves = true) + } + + private def getBroadcastBlocks(broadcastId: Long): Seq[BlockId] = { + blockManager.master.getMatchingBlockIds( _ match { + case BroadcastBlockId(`broadcastId`, _) => true + case _ => false + }, askSlaves = true) + } + + private def blockManager = sc.env.blockManager + private def mapOutputTrackerMaster = sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster] +} diff --git a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala index a5bd72eb0a122..6b2571cd9295e 100644 --- a/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala +++ b/core/src/test/scala/org/apache/spark/MapOutputTrackerSuite.scala @@ -57,12 +57,13 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.stop() } - test("master register and fetch") { + test("master register shuffle and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) tracker.registerShuffle(10, 2) + assert(tracker.containsShuffle(10)) val compressedSize1000 = MapOutputTracker.compressSize(1000L) val compressedSize10000 = MapOutputTracker.compressSize(10000L) val size1000 = MapOutputTracker.decompressSize(compressedSize1000) @@ -77,7 +78,25 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { tracker.stop() } - test("master register and unregister and fetch") { + test("master register and unregister shuffle") { + val actorSystem = ActorSystem("test") + val tracker = new MapOutputTrackerMaster(conf) + tracker.trackerActor = actorSystem.actorOf(Props(new MapOutputTrackerMasterActor(tracker, conf))) + tracker.registerShuffle(10, 2) + val compressedSize1000 = MapOutputTracker.compressSize(1000L) + val compressedSize10000 = MapOutputTracker.compressSize(10000L) + tracker.registerMapOutput(10, 0, new MapStatus(BlockManagerId("a", "hostA", 1000, 0), + Array(compressedSize1000, compressedSize10000))) + tracker.registerMapOutput(10, 1, new MapStatus(BlockManagerId("b", "hostB", 1000, 0), + Array(compressedSize10000, compressedSize1000))) + assert(tracker.containsShuffle(10)) + assert(tracker.getServerStatuses(10, 0).nonEmpty) + tracker.unregisterShuffle(10) + assert(!tracker.containsShuffle(10)) + assert(tracker.getServerStatuses(10, 0).isEmpty) + } + + test("master register shuffle and unregister map output and fetch") { val actorSystem = ActorSystem("test") val tracker = new MapOutputTrackerMaster(conf) tracker.trackerActor = @@ -114,7 +133,7 @@ class MapOutputTrackerSuite extends FunSuite with LocalSparkContext { val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = new SecurityManager(conf)) - val slaveTracker = new MapOutputTracker(conf) + val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index b6dd0526105a0..e10ec7d2624a0 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -28,7 +28,7 @@ import org.scalatest.concurrent.Timeouts._ import org.scalatest.matchers.ShouldMatchers._ import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{MapOutputTrackerMaster, SecurityManager, SparkConf} import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} @@ -42,6 +42,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT var oldArch: String = null conf.set("spark.authenticate", "false") val securityMgr = new SecurityManager(conf) + val mapOutputTracker = new MapOutputTrackerMaster(conf) // Reuse a serializer across tests to avoid creating a new thread-local buffer on each test conf.set("spark.kryoserializer.buffer.mb", "1") @@ -130,7 +131,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 1 manager interaction") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -160,9 +162,10 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("master + 2 managers interaction") { - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store2 = new BlockManager("exec2", actorSystem, master, new KryoSerializer(conf), 2000, conf, - securityMgr) + securityMgr, mapOutputTracker) val peers = master.getPeers(store.blockManagerId, 1) assert(peers.size === 1, "master did not return the other manager as a peer") @@ -177,7 +180,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing block") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -225,7 +229,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("removing rdd") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -257,9 +262,82 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT master.getLocations(rdd(0, 1)) should have size 0 } + test("removing broadcast") { + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) + val driverStore = store + val executorStore = new BlockManager("executor", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) + val a1 = new Array[Byte](400) + val a2 = new Array[Byte](400) + val a3 = new Array[Byte](400) + val a4 = new Array[Byte](400) + + val broadcast0BlockId = BroadcastBlockId(0) + val broadcast1BlockId = BroadcastBlockId(1) + val broadcast2BlockId = BroadcastBlockId(2) + val broadcast2BlockId2 = BroadcastBlockId(2, "_") + + // insert broadcast blocks in both the stores + Seq(driverStore, executorStore).foreach { case s => + s.putSingle(broadcast0BlockId, a1, StorageLevel.DISK_ONLY) + s.putSingle(broadcast1BlockId, a2, StorageLevel.DISK_ONLY) + s.putSingle(broadcast2BlockId, a3, StorageLevel.DISK_ONLY) + s.putSingle(broadcast2BlockId2, a4, StorageLevel.DISK_ONLY) + } + + // verify whether the blocks exist in both the stores + Seq(driverStore, executorStore).foreach { case s => + s.getLocal(broadcast0BlockId) should not be (None) + s.getLocal(broadcast1BlockId) should not be (None) + s.getLocal(broadcast2BlockId) should not be (None) + s.getLocal(broadcast2BlockId2) should not be (None) + } + + // remove broadcast 0 block only from executors + master.removeBroadcast(0, removeFromMaster = false, blocking = true) + + // only broadcast 0 block should be removed from the executor store + executorStore.getLocal(broadcast0BlockId) should be (None) + executorStore.getLocal(broadcast1BlockId) should not be (None) + executorStore.getLocal(broadcast2BlockId) should not be (None) + + // nothing should be removed from the driver store + driverStore.getLocal(broadcast0BlockId) should not be (None) + driverStore.getLocal(broadcast1BlockId) should not be (None) + driverStore.getLocal(broadcast2BlockId) should not be (None) + + // remove broadcast 0 block from the driver as well + master.removeBroadcast(0, removeFromMaster = true, blocking = true) + driverStore.getLocal(broadcast0BlockId) should be (None) + driverStore.getLocal(broadcast1BlockId) should not be (None) + + // remove broadcast 1 block from both the stores asynchronously + // and verify all broadcast 1 blocks have been removed + master.removeBroadcast(1, removeFromMaster = true, blocking = false) + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + driverStore.getLocal(broadcast1BlockId) should be (None) + executorStore.getLocal(broadcast1BlockId) should be (None) + } + + // remove broadcast 2 from both the stores asynchronously + // and verify all broadcast 2 blocks have been removed + master.removeBroadcast(2, removeFromMaster = true, blocking = false) + eventually(timeout(1000 milliseconds), interval(10 milliseconds)) { + driverStore.getLocal(broadcast2BlockId) should be (None) + driverStore.getLocal(broadcast2BlockId2) should be (None) + executorStore.getLocal(broadcast2BlockId) should be (None) + executorStore.getLocal(broadcast2BlockId2) should be (None) + } + executorStore.stop() + driverStore.stop() + store = null + } + test("reregistration on heart beat") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) store.putSingle("a1", a1, StorageLevel.MEMORY_ONLY) @@ -275,7 +353,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("reregistration on block update") { - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) @@ -294,7 +373,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("reregistration doesn't dead lock") { val heartBeat = PrivateMethod[Unit]('heartBeat) - store = new BlockManager("", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = List(new Array[Byte](400)) @@ -331,7 +411,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -350,7 +431,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -369,7 +451,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of same RDD") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -388,7 +471,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU for partitions of multiple RDDs") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(0, 2), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 1), new Array[Byte](400), StorageLevel.MEMORY_ONLY) @@ -414,7 +498,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT // TODO Make the spark.test.tachyon.enable true after using tachyon 0.5.0 testing jar. val tachyonUnitTestEnabled = conf.getBoolean("spark.test.tachyon.enable", false) if (tachyonUnitTestEnabled) { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -430,7 +515,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("on-disk storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -443,7 +529,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -458,7 +545,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -473,7 +561,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -488,7 +577,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("disk and memory storage with serialization and getLocalBytes") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -503,7 +593,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val a1 = new Array[Byte](400) val a2 = new Array[Byte](400) val a3 = new Array[Byte](400) @@ -525,7 +616,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("in-memory LRU with streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -549,7 +641,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("LRU with mixed storage levels and streams") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val list1 = List(new Array[Byte](200), new Array[Byte](200)) val list2 = List(new Array[Byte](200), new Array[Byte](200)) val list3 = List(new Array[Byte](200), new Array[Byte](200)) @@ -595,7 +688,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("overly large block") { - store = new BlockManager("", actorSystem, master, serializer, 500, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 500, conf, + securityMgr, mapOutputTracker) store.putSingle("a1", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.getSingle("a1") === None, "a1 was in store") store.putSingle("a2", new Array[Byte](1000), StorageLevel.MEMORY_AND_DISK) @@ -606,7 +700,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block compression") { try { conf.set("spark.shuffle.compress", "true") - store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec1", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) <= 100, "shuffle_0_0_0 was not compressed") @@ -614,7 +709,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.shuffle.compress", "false") - store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec2", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(ShuffleBlockId(0, 0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(ShuffleBlockId(0, 0, 0)) >= 1000, "shuffle_0_0_0 was compressed") @@ -622,7 +718,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.broadcast.compress", "true") - store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec3", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) <= 100, "broadcast_0 was not compressed") @@ -630,28 +727,32 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT store = null conf.set("spark.broadcast.compress", "false") - store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec4", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(BroadcastBlockId(0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(BroadcastBlockId(0)) >= 1000, "broadcast_0 was compressed") store.stop() store = null conf.set("spark.rdd.compress", "true") - store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec5", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) <= 100, "rdd_0_0 was not compressed") store.stop() store = null conf.set("spark.rdd.compress", "false") - store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec6", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 0), new Array[Byte](1000), StorageLevel.MEMORY_ONLY_SER) assert(store.memoryStore.getSize(rdd(0, 0)) >= 1000, "rdd_0_0 was compressed") store.stop() store = null // Check that any other block types are also kept uncompressed - store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, securityMgr) + store = new BlockManager("exec7", actorSystem, master, serializer, 2000, conf, + securityMgr, mapOutputTracker) store.putSingle("other_block", new Array[Byte](1000), StorageLevel.MEMORY_ONLY) assert(store.memoryStore.getSize("other_block") >= 1000, "other_block was compressed") store.stop() @@ -666,7 +767,7 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT test("block store put failure") { // Use Java serializer so we can create an unserializable error. store = new BlockManager("", actorSystem, master, new JavaSerializer(conf), 1200, conf, - securityMgr) + securityMgr, mapOutputTracker) // The put should fail since a1 is not serializable. class UnserializableClass @@ -682,7 +783,8 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT } test("updated block statuses") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) val list = List.fill(2)(new Array[Byte](200)) val bigList = List.fill(8)(new Array[Byte](200)) @@ -735,8 +837,83 @@ class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodT assert(!store.get("list5").isDefined, "list5 was in store") } + test("query block statuses") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) + val list = List.fill(2)(new Array[Byte](200)) + + // Tell master. By LRU, only list2 and list3 remains. + store.put("list1", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("list3", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + + // getLocations and getBlockStatus should yield the same locations + assert(store.master.getLocations("list1").size === 0) + assert(store.master.getLocations("list2").size === 1) + assert(store.master.getLocations("list3").size === 1) + assert(store.master.getBlockStatus("list1", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list2", askSlaves = false).size === 1) + assert(store.master.getBlockStatus("list3", askSlaves = false).size === 1) + assert(store.master.getBlockStatus("list1", askSlaves = true).size === 0) + assert(store.master.getBlockStatus("list2", askSlaves = true).size === 1) + assert(store.master.getBlockStatus("list3", askSlaves = true).size === 1) + + // This time don't tell master and see what happens. By LRU, only list5 and list6 remains. + store.put("list4", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + store.put("list5", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.put("list6", list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = false) + + // getLocations should return nothing because the master is not informed + // getBlockStatus without asking slaves should have the same result + // getBlockStatus with asking slaves, however, should return the actual block statuses + assert(store.master.getLocations("list4").size === 0) + assert(store.master.getLocations("list5").size === 0) + assert(store.master.getLocations("list6").size === 0) + assert(store.master.getBlockStatus("list4", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list5", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list6", askSlaves = false).size === 0) + assert(store.master.getBlockStatus("list4", askSlaves = true).size === 0) + assert(store.master.getBlockStatus("list5", askSlaves = true).size === 1) + assert(store.master.getBlockStatus("list6", askSlaves = true).size === 1) + } + + test("get matching blocks") { + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) + val list = List.fill(2)(new Array[Byte](10)) + + // insert some blocks + store.put("list1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("list2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("list3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + + // getLocations and getBlockStatus should yield the same locations + assert(store.master.getMatchingBlockIds(_.toString.contains("list"), askSlaves = false).size === 3) + assert(store.master.getMatchingBlockIds(_.toString.contains("list1"), askSlaves = false).size === 1) + + // insert some more blocks + store.put("newlist1", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = true) + store.put("newlist2", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + store.put("newlist3", list.iterator, StorageLevel.MEMORY_AND_DISK, tellMaster = false) + + // getLocations and getBlockStatus should yield the same locations + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = false).size === 1) + assert(store.master.getMatchingBlockIds(_.toString.contains("newlist"), askSlaves = true).size === 3) + + val blockIds = Seq(RDDBlockId(1, 0), RDDBlockId(1, 1), RDDBlockId(2, 0)) + blockIds.foreach { blockId => + store.put(blockId, list.iterator, StorageLevel.MEMORY_ONLY, tellMaster = true) + } + val matchedBlockIds = store.master.getMatchingBlockIds(_ match { + case RDDBlockId(1, _) => true + case _ => false + }, askSlaves = true) + assert(matchedBlockIds.toSet === Set(RDDBlockId(1, 0), RDDBlockId(1, 1))) + } + test("SPARK-1194 regression: fix the same-RDD rule for cache replacement") { - store = new BlockManager("", actorSystem, master, serializer, 1200, conf, securityMgr) + store = new BlockManager("", actorSystem, master, serializer, 1200, conf, + securityMgr, mapOutputTracker) store.putSingle(rdd(0, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) store.putSingle(rdd(1, 0), new Array[Byte](400), StorageLevel.MEMORY_ONLY) // Access rdd_1_0 to ensure it's not least recently used. diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 62f9b3cc7b2c1..808ddfdcf45d8 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -59,8 +59,16 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { val newFile = diskBlockManager.getFile(blockId) writeToFile(newFile, 10) assertSegmentEquals(blockId, blockId.name, 0, 10) - + assert(diskBlockManager.containsBlock(blockId)) newFile.delete() + assert(!diskBlockManager.containsBlock(blockId)) + } + + test("enumerating blocks") { + val ids = (1 to 100).map(i => TestBlockId("test_" + i)) + val files = ids.map(id => diskBlockManager.getFile(id)) + files.foreach(file => writeToFile(file, 10)) + assert(diskBlockManager.getAllBlocks.toSet === ids.toSet) } test("block appending") { diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 054eb01a64c11..7bab7da8fed68 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -108,8 +108,7 @@ class JsonProtocolSuite extends FunSuite { // BlockId testBlockId(RDDBlockId(1, 2)) testBlockId(ShuffleBlockId(1, 2, 3)) - testBlockId(BroadcastBlockId(1L)) - testBlockId(BroadcastHelperBlockId(BroadcastBlockId(2L), "Spark")) + testBlockId(BroadcastBlockId(1L, "insert_words_of_wisdom_here")) testBlockId(TaskResultBlockId(1L)) testBlockId(StreamBlockId(1, 2L)) } @@ -555,4 +554,4 @@ class JsonProtocolSuite extends FunSuite { {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} """ - } +} diff --git a/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala new file mode 100644 index 0000000000000..6a5653ed2fb54 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/util/TimeStampedHashMapSuite.scala @@ -0,0 +1,264 @@ +/* + * 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.util + +import java.lang.ref.WeakReference + +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer +import scala.util.Random + +import org.scalatest.FunSuite + +class TimeStampedHashMapSuite extends FunSuite { + + // Test the testMap function - a Scala HashMap should obviously pass + testMap(new mutable.HashMap[String, String]()) + + // Test TimeStampedHashMap basic functionality + testMap(new TimeStampedHashMap[String, String]()) + testMapThreadSafety(new TimeStampedHashMap[String, String]()) + + // Test TimeStampedWeakValueHashMap basic functionality + testMap(new TimeStampedWeakValueHashMap[String, String]()) + testMapThreadSafety(new TimeStampedWeakValueHashMap[String, String]()) + + test("TimeStampedHashMap - clearing by timestamp") { + // clearing by insertion time + val map = new TimeStampedHashMap[String, String](updateTimeStampOnGet = false) + map("k1") = "v1" + assert(map("k1") === "v1") + Thread.sleep(10) + val threshTime = System.currentTimeMillis + assert(map.getTimestamp("k1").isDefined) + assert(map.getTimestamp("k1").get < threshTime) + map.clearOldValues(threshTime) + assert(map.get("k1") === None) + + // clearing by modification time + val map1 = new TimeStampedHashMap[String, String](updateTimeStampOnGet = true) + map1("k1") = "v1" + map1("k2") = "v2" + assert(map1("k1") === "v1") + Thread.sleep(10) + val threshTime1 = System.currentTimeMillis + Thread.sleep(10) + assert(map1("k2") === "v2") // access k2 to update its access time to > threshTime + assert(map1.getTimestamp("k1").isDefined) + assert(map1.getTimestamp("k1").get < threshTime1) + assert(map1.getTimestamp("k2").isDefined) + assert(map1.getTimestamp("k2").get >= threshTime1) + map1.clearOldValues(threshTime1) //should only clear k1 + assert(map1.get("k1") === None) + assert(map1.get("k2").isDefined) + } + + test("TimeStampedWeakValueHashMap - clearing by timestamp") { + // clearing by insertion time + val map = new TimeStampedWeakValueHashMap[String, String](updateTimeStampOnGet = false) + map("k1") = "v1" + assert(map("k1") === "v1") + Thread.sleep(10) + val threshTime = System.currentTimeMillis + assert(map.getTimestamp("k1").isDefined) + assert(map.getTimestamp("k1").get < threshTime) + map.clearOldValues(threshTime) + assert(map.get("k1") === None) + + // clearing by modification time + val map1 = new TimeStampedWeakValueHashMap[String, String](updateTimeStampOnGet = true) + map1("k1") = "v1" + map1("k2") = "v2" + assert(map1("k1") === "v1") + Thread.sleep(10) + val threshTime1 = System.currentTimeMillis + Thread.sleep(10) + assert(map1("k2") === "v2") // access k2 to update its access time to > threshTime + assert(map1.getTimestamp("k1").isDefined) + assert(map1.getTimestamp("k1").get < threshTime1) + assert(map1.getTimestamp("k2").isDefined) + assert(map1.getTimestamp("k2").get >= threshTime1) + map1.clearOldValues(threshTime1) //should only clear k1 + assert(map1.get("k1") === None) + assert(map1.get("k2").isDefined) + } + + test("TimeStampedWeakValueHashMap - clearing weak references") { + var strongRef = new Object + val weakRef = new WeakReference(strongRef) + val map = new TimeStampedWeakValueHashMap[String, Object] + map("k1") = strongRef + map("k2") = "v2" + map("k3") = "v3" + assert(map("k1") === strongRef) + + // clear strong reference to "k1" + strongRef = null + val startTime = System.currentTimeMillis + System.gc() // Make a best effort to run the garbage collection. It *usually* runs GC. + System.runFinalization() // Make a best effort to call finalizer on all cleaned objects. + while(System.currentTimeMillis - startTime < 10000 && weakRef.get != null) { + System.gc() + System.runFinalization() + Thread.sleep(100) + } + assert(map.getReference("k1").isDefined) + val ref = map.getReference("k1").get + assert(ref.get === null) + assert(map.get("k1") === None) + + // operations should only display non-null entries + assert(map.iterator.forall { case (k, v) => k != "k1" }) + assert(map.filter { case (k, v) => k != "k2" }.size === 1) + assert(map.filter { case (k, v) => k != "k2" }.head._1 === "k3") + assert(map.toMap.size === 2) + assert(map.toMap.forall { case (k, v) => k != "k1" }) + val buffer = new ArrayBuffer[String] + map.foreach { case (k, v) => buffer += v.toString } + assert(buffer.size === 2) + assert(buffer.forall(_ != "k1")) + val plusMap = map + (("k4", "v4")) + assert(plusMap.size === 3) + assert(plusMap.forall { case (k, v) => k != "k1" }) + val minusMap = map - "k2" + assert(minusMap.size === 1) + assert(minusMap.head._1 == "k3") + + // clear null values - should only clear k1 + map.clearNullValues() + assert(map.getReference("k1") === None) + assert(map.get("k1") === None) + assert(map.get("k2").isDefined) + assert(map.get("k2").get === "v2") + } + + /** Test basic operations of a Scala mutable Map. */ + def testMap(hashMapConstructor: => mutable.Map[String, String]) { + def newMap() = hashMapConstructor + val testMap1 = newMap() + val testMap2 = newMap() + val name = testMap1.getClass.getSimpleName + + test(name + " - basic test") { + // put, get, and apply + testMap1 += (("k1", "v1")) + assert(testMap1.get("k1").isDefined) + assert(testMap1.get("k1").get === "v1") + testMap1("k2") = "v2" + assert(testMap1.get("k2").isDefined) + assert(testMap1.get("k2").get === "v2") + assert(testMap1("k2") === "v2") + testMap1.update("k3", "v3") + assert(testMap1.get("k3").isDefined) + assert(testMap1.get("k3").get === "v3") + + // remove + testMap1.remove("k1") + assert(testMap1.get("k1").isEmpty) + testMap1.remove("k2") + intercept[NoSuchElementException] { + testMap1("k2") // Map.apply() causes exception + } + testMap1 -= "k3" + assert(testMap1.get("k3").isEmpty) + + // multi put + val keys = (1 to 100).map(_.toString) + val pairs = keys.map(x => (x, x * 2)) + assert((testMap2 ++ pairs).iterator.toSet === pairs.toSet) + testMap2 ++= pairs + + // iterator + assert(testMap2.iterator.toSet === pairs.toSet) + + // filter + val filtered = testMap2.filter { case (_, v) => v.toInt % 2 == 0 } + val evenPairs = pairs.filter { case (_, v) => v.toInt % 2 == 0 } + assert(filtered.iterator.toSet === evenPairs.toSet) + + // foreach + val buffer = new ArrayBuffer[(String, String)] + testMap2.foreach(x => buffer += x) + assert(testMap2.toSet === buffer.toSet) + + // multi remove + testMap2("k1") = "v1" + testMap2 --= keys + assert(testMap2.size === 1) + assert(testMap2.iterator.toSeq.head === ("k1", "v1")) + + // + + val testMap3 = testMap2 + (("k0", "v0")) + assert(testMap3.size === 2) + assert(testMap3.get("k1").isDefined) + assert(testMap3.get("k1").get === "v1") + assert(testMap3.get("k0").isDefined) + assert(testMap3.get("k0").get === "v0") + + // - + val testMap4 = testMap3 - "k0" + assert(testMap4.size === 1) + assert(testMap4.get("k1").isDefined) + assert(testMap4.get("k1").get === "v1") + } + } + + /** Test thread safety of a Scala mutable map. */ + def testMapThreadSafety(hashMapConstructor: => mutable.Map[String, String]) { + def newMap() = hashMapConstructor + val name = newMap().getClass.getSimpleName + val testMap = newMap() + @volatile var error = false + + def getRandomKey(m: mutable.Map[String, String]): Option[String] = { + val keys = testMap.keysIterator.toSeq + if (keys.nonEmpty) { + Some(keys(Random.nextInt(keys.size))) + } else { + None + } + } + + val threads = (1 to 25).map(i => new Thread() { + override def run() { + try { + for (j <- 1 to 1000) { + Random.nextInt(3) match { + case 0 => + testMap(Random.nextString(10)) = Random.nextDouble().toString // put + case 1 => + getRandomKey(testMap).map(testMap.get) // get + case 2 => + getRandomKey(testMap).map(testMap.remove) // remove + } + } + } catch { + case t: Throwable => + error = true + throw t + } + } + }) + + test(name + " - threading safety test") { + threads.map(_.start) + threads.map(_.join) + assert(!error) + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d48b51aa69565..d043200f71a0b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -341,9 +341,11 @@ abstract class DStream[T: ClassTag] ( */ private[streaming] def clearMetadata(time: Time) { val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration)) + logDebug("Clearing references to old RDDs: [" + + oldRDDs.map(x => s"${x._1} -> ${x._2.id}").mkString(", ") + "]") generatedRDDs --= oldRDDs.keys if (ssc.conf.getBoolean("spark.streaming.unpersist", false)) { - logDebug("Unpersisting old RDDs: " + oldRDDs.keys.mkString(", ")) + logDebug("Unpersisting old RDDs: " + oldRDDs.values.map(_.id).mkString(", ")) oldRDDs.values.foreach(_.unpersist(false)) } logDebug("Cleared " + oldRDDs.size + " RDDs that were older than " + From 83ac9a4bbf272028d0c4639cbd1e12022b9ae77a Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 8 Apr 2014 00:00:17 -0700 Subject: [PATCH 231/397] [SPARK-1331] Added graceful shutdown to Spark Streaming Current version of StreamingContext.stop() directly kills all the data receivers (NetworkReceiver) without waiting for the data already received to be persisted and processed. This PR provides the fix. Now, when the StreamingContext.stop() is called, the following sequence of steps will happen. 1. The driver will send a stop signal to all the active receivers. 2. Each receiver, when it gets a stop signal from the driver, first stop receiving more data, then waits for the thread that persists data blocks to BlockManager to finish persisting all receive data, and finally quits. 3. After all the receivers have stopped, the driver will wait for the Job Generator and Job Scheduler to finish processing all the received data. It also fixes the semantics of StreamingContext.start and stop. It will throw appropriate errors and warnings if stop() is called before start(), stop() is called twice, etc. Author: Tathagata Das Closes #247 from tdas/graceful-shutdown and squashes the following commits: 61c0016 [Tathagata Das] Updated MIMA binary check excludes. ae1d39b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into graceful-shutdown 6b59cfc [Tathagata Das] Minor changes based on Andrew's comment on PR. d0b8d65 [Tathagata Das] Reduced time taken by graceful shutdown unit test. f55bc67 [Tathagata Das] Fix scalastyle c69b3a7 [Tathagata Das] Updates based on Patrick's comments. c43b8ae [Tathagata Das] Added graceful shutdown to Spark Streaming. --- project/MimaBuild.scala | 24 +-- .../apache/spark/streaming/Checkpoint.scala | 14 +- .../spark/streaming/StreamingContext.scala | 48 +++++- .../api/java/JavaStreamingContext.scala | 12 +- .../dstream/NetworkInputDStream.scala | 151 +++++++++++------ .../dstream/SocketInputDStream.scala | 1 - .../streaming/receivers/ActorReceiver.scala | 2 +- .../streaming/scheduler/JobGenerator.scala | 124 ++++++++++---- .../streaming/scheduler/JobScheduler.scala | 56 ++++--- .../scheduler/NetworkInputTracker.scala | 154 ++++++++++-------- .../apache/spark/streaming/util/Clock.scala | 5 +- .../spark/streaming/util/RecurringTimer.scala | 62 +++++-- .../streaming/BasicOperationsSuite.scala | 4 +- .../streaming/StreamingContextSuite.scala | 108 ++++++++++-- .../spark/streaming/TestSuiteBase.scala | 2 +- 15 files changed, 552 insertions(+), 215 deletions(-) diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index e7c9c47c960fa..5ea4817bfde18 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -58,17 +58,19 @@ object MimaBuild { SparkBuild.SPARK_VERSION match { case v if v.startsWith("1.0") => Seq( - excludePackage("org.apache.spark.api.java"), - excludePackage("org.apache.spark.streaming.api.java"), - excludePackage("org.apache.spark.mllib") - ) ++ - excludeSparkClass("rdd.ClassTags") ++ - excludeSparkClass("util.XORShiftRandom") ++ - excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ - excludeSparkClass("mllib.optimization.SquaredGradient") ++ - excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ - excludeSparkClass("mllib.regression.LassoWithSGD") ++ - excludeSparkClass("mllib.regression.LinearRegressionWithSGD") + excludePackage("org.apache.spark.api.java"), + excludePackage("org.apache.spark.streaming.api.java"), + excludePackage("org.apache.spark.mllib") + ) ++ + excludeSparkClass("rdd.ClassTags") ++ + excludeSparkClass("util.XORShiftRandom") ++ + excludeSparkClass("mllib.recommendation.MFDataGenerator") ++ + excludeSparkClass("mllib.optimization.SquaredGradient") ++ + excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ + excludeSparkClass("mllib.regression.LassoWithSGD") ++ + excludeSparkClass("mllib.regression.LinearRegressionWithSGD") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") case _ => Seq() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index baf80fe2a91b7..93023e8dced57 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -194,19 +194,19 @@ class CheckpointWriter( } } - def stop() { - synchronized { - if (stopped) { - return - } - stopped = true - } + def stop(): Unit = synchronized { + if (stopped) return + executor.shutdown() val startTime = System.currentTimeMillis() val terminated = executor.awaitTermination(10, java.util.concurrent.TimeUnit.SECONDS) + if (!terminated) { + executor.shutdownNow() + } val endTime = System.currentTimeMillis() logInfo("CheckpointWriter executor terminated ? " + terminated + ", waited for " + (endTime - startTime) + " ms.") + stopped = true } private def fs = synchronized { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index e198c69470c1f..a4e236c65ff86 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -158,6 +158,15 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter + /** Enumeration to identify current state of the StreamingContext */ + private[streaming] object StreamingContextState extends Enumeration { + type CheckpointState = Value + val Initialized, Started, Stopped = Value + } + + import StreamingContextState._ + private[streaming] var state = Initialized + /** * Return the associated Spark context */ @@ -405,9 +414,18 @@ class StreamingContext private[streaming] ( /** * Start the execution of the streams. */ - def start() = synchronized { + def start(): Unit = synchronized { + // Throw exception if the context has already been started once + // or if a stopped context is being started again + if (state == Started) { + throw new SparkException("StreamingContext has already been started") + } + if (state == Stopped) { + throw new SparkException("StreamingContext has already been stopped") + } validate() scheduler.start() + state = Started } /** @@ -428,14 +446,38 @@ class StreamingContext private[streaming] ( } /** - * Stop the execution of the streams. + * Stop the execution of the streams immediately (does not wait for all received data + * to be processed). * @param stopSparkContext Stop the associated SparkContext or not + * */ def stop(stopSparkContext: Boolean = true): Unit = synchronized { - scheduler.stop() + stop(stopSparkContext, false) + } + + /** + * Stop the execution of the streams, with option of ensuring all received data + * has been processed. + * @param stopSparkContext Stop the associated SparkContext or not + * @param stopGracefully Stop gracefully by waiting for the processing of all + * received data to be completed + */ + def stop(stopSparkContext: Boolean, stopGracefully: Boolean): Unit = synchronized { + // Warn (but not fail) if context is stopped twice, + // or context is stopped before starting + if (state == Initialized) { + logWarning("StreamingContext has not been started yet") + return + } + if (state == Stopped) { + logWarning("StreamingContext has already been stopped") + return + } // no need to throw an exception as its okay to stop twice + scheduler.stop(stopGracefully) logInfo("StreamingContext stopped successfully") waiter.notifyStop() if (stopSparkContext) sc.stop() + state = Stopped } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index b705d2ec9a58e..c800602d0959b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -509,8 +509,16 @@ class JavaStreamingContext(val ssc: StreamingContext) { * Stop the execution of the streams. * @param stopSparkContext Stop the associated SparkContext or not */ - def stop(stopSparkContext: Boolean): Unit = { - ssc.stop(stopSparkContext) + def stop(stopSparkContext: Boolean) = ssc.stop(stopSparkContext) + + /** + * Stop the execution of the streams. + * @param stopSparkContext Stop the associated SparkContext or not + * @param stopGracefully Stop gracefully by waiting for the processing of all + * received data to be completed + */ + def stop(stopSparkContext: Boolean, stopGracefully: Boolean) = { + ssc.stop(stopSparkContext, stopGracefully) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index 72ad0bae75bfb..d19a635fe8eca 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import java.util.concurrent.ArrayBlockingQueue +import java.util.concurrent.{TimeUnit, ArrayBlockingQueue} import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer @@ -34,6 +34,7 @@ import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.rdd.{RDD, BlockRDD} import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver} +import org.apache.spark.util.AkkaUtils /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -69,7 +70,7 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte // then this returns an empty RDD. This may happen when recovering from a // master failure if (validTime >= graph.startTime) { - val blockIds = ssc.scheduler.networkInputTracker.getBlockIds(id, validTime) + val blockIds = ssc.scheduler.networkInputTracker.getBlocks(id, validTime) Some(new BlockRDD[T](ssc.sc, blockIds)) } else { Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) @@ -79,7 +80,7 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte private[streaming] sealed trait NetworkReceiverMessage -private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage +private[streaming] case class StopReceiver() extends NetworkReceiverMessage private[streaming] case class ReportBlock(blockId: BlockId, metadata: Any) extends NetworkReceiverMessage private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage @@ -90,13 +91,31 @@ private[streaming] case class ReportError(msg: String) extends NetworkReceiverMe */ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging { + /** Local SparkEnv */ lazy protected val env = SparkEnv.get + /** Remote Akka actor for the NetworkInputTracker */ + lazy protected val trackerActor = { + val ip = env.conf.get("spark.driver.host", "localhost") + val port = env.conf.getInt("spark.driver.port", 7077) + val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) + env.actorSystem.actorSelection(url) + } + + /** Akka actor for receiving messages from the NetworkInputTracker in the driver */ lazy protected val actor = env.actorSystem.actorOf( Props(new NetworkReceiverActor()), "NetworkReceiver-" + streamId) + /** Timeout for Akka actor messages */ + lazy protected val askTimeout = AkkaUtils.askTimeout(env.conf) + + /** Thread that starts the receiver and stays blocked while data is being received */ lazy protected val receivingThread = Thread.currentThread() + /** Exceptions that occurs while receiving data */ + protected lazy val exceptions = new ArrayBuffer[Exception] + + /** Identifier of the stream this receiver is associated with */ protected var streamId: Int = -1 /** @@ -112,7 +131,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging def getLocationPreference() : Option[String] = None /** - * Starts the receiver. First is accesses all the lazy members to + * Start the receiver. First is accesses all the lazy members to * materialize them. Then it calls the user-defined onStart() method to start * other threads, etc required to receiver the data. */ @@ -124,83 +143,107 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging receivingThread // Call user-defined onStart() + logInfo("Starting receiver") onStart() + + // Wait until interrupt is called on this thread + while(true) Thread.sleep(100000) } catch { case ie: InterruptedException => - logInfo("Receiving thread interrupted") + logInfo("Receiving thread has been interrupted, receiver " + streamId + " stopped") case e: Exception => - stopOnError(e) + logError("Error receiving data in receiver " + streamId, e) + exceptions += e + } + + // Call user-defined onStop() + logInfo("Stopping receiver") + try { + onStop() + } catch { + case e: Exception => + logError("Error stopping receiver " + streamId, e) + exceptions += e + } + + val message = if (exceptions.isEmpty) { + null + } else if (exceptions.size == 1) { + val e = exceptions.head + "Exception in receiver " + streamId + ": " + e.getMessage + "\n" + e.getStackTraceString + } else { + "Multiple exceptions in receiver " + streamId + "(" + exceptions.size + "):\n" + exceptions.zipWithIndex.map { + case (e, i) => "Exception " + i + ": " + e.getMessage + "\n" + e.getStackTraceString + }.mkString("\n") } + logInfo("Deregistering receiver " + streamId) + val future = trackerActor.ask(DeregisterReceiver(streamId, message))(askTimeout) + Await.result(future, askTimeout) + logInfo("Deregistered receiver " + streamId) + env.actorSystem.stop(actor) + logInfo("Stopped receiver " + streamId) } /** - * Stops the receiver. First it interrupts the main receiving thread, - * that is, the thread that called receiver.start(). Then it calls the user-defined - * onStop() method to stop other threads and/or do cleanup. + * Stop the receiver. First it interrupts the main receiving thread, + * that is, the thread that called receiver.start(). */ def stop() { + // Stop receiving by interrupting the receiving thread receivingThread.interrupt() - onStop() - // TODO: terminate the actor + logInfo("Interrupted receiving thread " + receivingThread + " for stopping") } /** - * Stops the receiver and reports exception to the tracker. + * Stop the receiver and reports exception to the tracker. * This should be called whenever an exception is to be handled on any thread * of the receiver. */ protected def stopOnError(e: Exception) { logError("Error receiving data", e) + exceptions += e stop() - actor ! ReportError(e.toString) } - /** - * Pushes a block (as an ArrayBuffer filled with data) into the block manager. + * Push a block (as an ArrayBuffer filled with data) into the block manager. */ def pushBlock(blockId: BlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) - actor ! ReportBlock(blockId, metadata) + trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + logDebug("Pushed block " + blockId) } /** - * Pushes a block (as bytes) into the block manager. + * Push a block (as bytes) into the block manager. */ def pushBlock(blockId: BlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) { env.blockManager.putBytes(blockId, bytes, level) - actor ! ReportBlock(blockId, metadata) + trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + } + + /** Set the ID of the DStream that this receiver is associated with */ + protected[streaming] def setStreamId(id: Int) { + streamId = id } /** A helper actor that communicates with the NetworkInputTracker */ private class NetworkReceiverActor extends Actor { - logInfo("Attempting to register with tracker") - val ip = env.conf.get("spark.driver.host", "localhost") - val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) - val tracker = env.actorSystem.actorSelection(url) - val timeout = 5.seconds override def preStart() { - val future = tracker.ask(RegisterReceiver(streamId, self))(timeout) - Await.result(future, timeout) + logInfo("Registered receiver " + streamId) + val future = trackerActor.ask(RegisterReceiver(streamId, self))(askTimeout) + Await.result(future, askTimeout) } override def receive() = { - case ReportBlock(blockId, metadata) => - tracker ! AddBlocks(streamId, Array(blockId), metadata) - case ReportError(msg) => - tracker ! DeregisterReceiver(streamId, msg) - case StopReceiver(msg) => + case StopReceiver => + logInfo("Received stop signal") stop() - tracker ! DeregisterReceiver(streamId, msg) } } - protected[streaming] def setStreamId(id: Int) { - streamId = id - } - /** * Batches objects created by a [[org.apache.spark.streaming.dstream.NetworkReceiver]] and puts * them into appropriately named blocks at regular intervals. This class starts two threads, @@ -214,23 +257,26 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging val clock = new SystemClock() val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) - val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer) + val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer, + "BlockGenerator") val blockStorageLevel = storageLevel val blocksForPushing = new ArrayBlockingQueue[Block](1000) val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } var currentBuffer = new ArrayBuffer[T] + var stopped = false def start() { blockIntervalTimer.start() blockPushingThread.start() - logInfo("Data handler started") + logInfo("Started BlockGenerator") } def stop() { - blockIntervalTimer.stop() - blockPushingThread.interrupt() - logInfo("Data handler stopped") + blockIntervalTimer.stop(false) + stopped = true + blockPushingThread.join() + logInfo("Stopped BlockGenerator") } def += (obj: T): Unit = synchronized { @@ -248,24 +294,35 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging } } catch { case ie: InterruptedException => - logInfo("Block interval timer thread interrupted") + logInfo("Block updating timer thread was interrupted") case e: Exception => - NetworkReceiver.this.stop() + NetworkReceiver.this.stopOnError(e) } } private def keepPushingBlocks() { - logInfo("Block pushing thread started") + logInfo("Started block pushing thread") try { - while(true) { + while(!stopped) { + Option(blocksForPushing.poll(100, TimeUnit.MILLISECONDS)) match { + case Some(block) => + NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) + case None => + } + } + // Push out the blocks that are still left + logInfo("Pushing out the last " + blocksForPushing.size() + " blocks") + while (!blocksForPushing.isEmpty) { val block = blocksForPushing.take() NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) + logInfo("Blocks left to push " + blocksForPushing.size()) } + logInfo("Stopped blocks pushing thread") } catch { case ie: InterruptedException => - logInfo("Block pushing thread interrupted") + logInfo("Block pushing thread was interrupted") case e: Exception => - NetworkReceiver.this.stop() + NetworkReceiver.this.stopOnError(e) } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 2cdd13f205313..63d94d1cc670a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -67,7 +67,6 @@ class SocketReceiver[T: ClassTag]( protected def onStop() { blockGenerator.stop() } - } private[streaming] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index bd78bae8a5c51..44eb2750c6c7a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -174,10 +174,10 @@ private[streaming] class ActorReceiver[T: ClassTag]( blocksGenerator.start() supervisor logInfo("Supervision tree for receivers initialized at:" + supervisor.path) + } protected def onStop() = { supervisor ! PoisonPill } - } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index c7306248b1950..92d885c4bc5a5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -39,16 +39,22 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { private val ssc = jobScheduler.ssc private val graph = ssc.graph + val clock = { val clockClass = ssc.sc.conf.get( "spark.streaming.clock", "org.apache.spark.streaming.util.SystemClock") Class.forName(clockClass).newInstance().asInstanceOf[Clock] } + private val timer = new RecurringTimer(clock, ssc.graph.batchDuration.milliseconds, - longTime => eventActor ! GenerateJobs(new Time(longTime))) - private lazy val checkpointWriter = - if (ssc.checkpointDuration != null && ssc.checkpointDir != null) { - new CheckpointWriter(this, ssc.conf, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration) + longTime => eventActor ! GenerateJobs(new Time(longTime)), "JobGenerator") + + // This is marked lazy so that this is initialized after checkpoint duration has been set + // in the context and the generator has been started. + private lazy val shouldCheckpoint = ssc.checkpointDuration != null && ssc.checkpointDir != null + + private lazy val checkpointWriter = if (shouldCheckpoint) { + new CheckpointWriter(this, ssc.conf, ssc.checkpointDir, ssc.sparkContext.hadoopConfiguration) } else { null } @@ -57,17 +63,16 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // This not being null means the scheduler has been started and not stopped private var eventActor: ActorRef = null + // last batch whose completion,checkpointing and metadata cleanup has been completed + private var lastProcessedBatch: Time = null + /** Start generation of jobs */ - def start() = synchronized { - if (eventActor != null) { - throw new SparkException("JobGenerator already started") - } + def start(): Unit = synchronized { + if (eventActor != null) return // generator has already been started eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { def receive = { - case event: JobGeneratorEvent => - logDebug("Got event of type " + event.getClass.getName) - processEvent(event) + case event: JobGeneratorEvent => processEvent(event) } }), "JobGenerator") if (ssc.isCheckpointPresent) { @@ -77,30 +82,79 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { } } - /** Stop generation of jobs */ - def stop() = synchronized { - if (eventActor != null) { - timer.stop() - ssc.env.actorSystem.stop(eventActor) - if (checkpointWriter != null) checkpointWriter.stop() - ssc.graph.stop() - logInfo("JobGenerator stopped") + /** + * Stop generation of jobs. processReceivedData = true makes this wait until jobs + * of current ongoing time interval has been generated, processed and corresponding + * checkpoints written. + */ + def stop(processReceivedData: Boolean): Unit = synchronized { + if (eventActor == null) return // generator has already been stopped + + if (processReceivedData) { + logInfo("Stopping JobGenerator gracefully") + val timeWhenStopStarted = System.currentTimeMillis() + val stopTimeout = 10 * ssc.graph.batchDuration.milliseconds + val pollTime = 100 + + // To prevent graceful stop to get stuck permanently + def hasTimedOut = { + val timedOut = System.currentTimeMillis() - timeWhenStopStarted > stopTimeout + if (timedOut) logWarning("Timed out while stopping the job generator") + timedOut + } + + // Wait until all the received blocks in the network input tracker has + // been consumed by network input DStreams, and jobs have been generated with them + logInfo("Waiting for all received blocks to be consumed for job generation") + while(!hasTimedOut && jobScheduler.networkInputTracker.hasMoreReceivedBlockIds) { + Thread.sleep(pollTime) + } + logInfo("Waited for all received blocks to be consumed for job generation") + + // Stop generating jobs + val stopTime = timer.stop(false) + graph.stop() + logInfo("Stopped generation timer") + + // Wait for the jobs to complete and checkpoints to be written + def haveAllBatchesBeenProcessed = { + lastProcessedBatch != null && lastProcessedBatch.milliseconds == stopTime + } + logInfo("Waiting for jobs to be processed and checkpoints to be written") + while (!hasTimedOut && !haveAllBatchesBeenProcessed) { + Thread.sleep(pollTime) + } + logInfo("Waited for jobs to be processed and checkpoints to be written") + } else { + logInfo("Stopping JobGenerator immediately") + // Stop timer and graph immediately, ignore unprocessed data and pending jobs + timer.stop(true) + graph.stop() } + + // Stop the actor and checkpoint writer + if (shouldCheckpoint) checkpointWriter.stop() + ssc.env.actorSystem.stop(eventActor) + logInfo("Stopped JobGenerator") } /** - * On batch completion, clear old metadata and checkpoint computation. + * Callback called when a batch has been completely processed. */ def onBatchCompletion(time: Time) { eventActor ! ClearMetadata(time) } - + + /** + * Callback called when the checkpoint of a batch has been written. + */ def onCheckpointCompletion(time: Time) { eventActor ! ClearCheckpointData(time) } /** Processes all events */ private def processEvent(event: JobGeneratorEvent) { + logDebug("Got event " + event) event match { case GenerateJobs(time) => generateJobs(time) case ClearMetadata(time) => clearMetadata(time) @@ -114,7 +168,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { val startTime = new Time(timer.getStartTime()) graph.start(startTime - graph.batchDuration) timer.start(startTime.milliseconds) - logInfo("JobGenerator started at " + startTime) + logInfo("Started JobGenerator at " + startTime) } /** Restarts the generator based on the information in checkpoint */ @@ -152,15 +206,17 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { // Restart the timer timer.start(restartTime.milliseconds) - logInfo("JobGenerator restarted at " + restartTime) + logInfo("Restarted JobGenerator at " + restartTime) } /** Generate jobs and perform checkpoint for the given `time`. */ private def generateJobs(time: Time) { SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { - case Success(jobs) => jobScheduler.runJobs(time, jobs) - case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) + case Success(jobs) => + jobScheduler.runJobs(time, jobs) + case Failure(e) => + jobScheduler.reportError("Error generating jobs for time " + time, e) } eventActor ! DoCheckpoint(time) } @@ -168,20 +224,32 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { /** Clear DStream metadata for the given `time`. */ private def clearMetadata(time: Time) { ssc.graph.clearMetadata(time) - eventActor ! DoCheckpoint(time) + + // If checkpointing is enabled, then checkpoint, + // else mark batch to be fully processed + if (shouldCheckpoint) { + eventActor ! DoCheckpoint(time) + } else { + markBatchFullyProcessed(time) + } } /** Clear DStream checkpoint data for the given `time`. */ private def clearCheckpointData(time: Time) { ssc.graph.clearCheckpointData(time) + markBatchFullyProcessed(time) } /** Perform checkpoint for the give `time`. */ - private def doCheckpoint(time: Time) = synchronized { - if (checkpointWriter != null && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { + private def doCheckpoint(time: Time) { + if (shouldCheckpoint && (time - graph.zeroTime).isMultipleOf(ssc.checkpointDuration)) { logInfo("Checkpointing graph for time " + time) ssc.graph.updateCheckpointData(time) checkpointWriter.write(new Checkpoint(ssc, time)) } } + + private def markBatchFullyProcessed(time: Time) { + lastProcessedBatch = time + } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index de675d3c7fb94..04e0a6a283cfb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -39,7 +39,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private val jobSets = new ConcurrentHashMap[Time, JobSet] private val numConcurrentJobs = ssc.conf.getInt("spark.streaming.concurrentJobs", 1) - private val executor = Executors.newFixedThreadPool(numConcurrentJobs) + private val jobExecutor = Executors.newFixedThreadPool(numConcurrentJobs) private val jobGenerator = new JobGenerator(this) val clock = jobGenerator.clock val listenerBus = new StreamingListenerBus() @@ -50,36 +50,54 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { private var eventActor: ActorRef = null - def start() = synchronized { - if (eventActor != null) { - throw new SparkException("JobScheduler already started") - } + def start(): Unit = synchronized { + if (eventActor != null) return // scheduler has already been started + logDebug("Starting JobScheduler") eventActor = ssc.env.actorSystem.actorOf(Props(new Actor { def receive = { case event: JobSchedulerEvent => processEvent(event) } }), "JobScheduler") + listenerBus.start() networkInputTracker = new NetworkInputTracker(ssc) networkInputTracker.start() - Thread.sleep(1000) jobGenerator.start() - logInfo("JobScheduler started") + logInfo("Started JobScheduler") } - def stop() = synchronized { - if (eventActor != null) { - jobGenerator.stop() - networkInputTracker.stop() - executor.shutdown() - if (!executor.awaitTermination(2, TimeUnit.SECONDS)) { - executor.shutdownNow() - } - listenerBus.stop() - ssc.env.actorSystem.stop(eventActor) - logInfo("JobScheduler stopped") + def stop(processAllReceivedData: Boolean): Unit = synchronized { + if (eventActor == null) return // scheduler has already been stopped + logDebug("Stopping JobScheduler") + + // First, stop receiving + networkInputTracker.stop() + + // Second, stop generating jobs. If it has to process all received data, + // then this will wait for all the processing through JobScheduler to be over. + jobGenerator.stop(processAllReceivedData) + + // Stop the executor for receiving new jobs + logDebug("Stopping job executor") + jobExecutor.shutdown() + + // Wait for the queued jobs to complete if indicated + val terminated = if (processAllReceivedData) { + jobExecutor.awaitTermination(1, TimeUnit.HOURS) // just a very large period of time + } else { + jobExecutor.awaitTermination(2, TimeUnit.SECONDS) } + if (!terminated) { + jobExecutor.shutdownNow() + } + logDebug("Stopped job executor") + + // Stop everything else + listenerBus.stop() + ssc.env.actorSystem.stop(eventActor) + eventActor = null + logInfo("Stopped JobScheduler") } def runJobs(time: Time, jobs: Seq[Job]) { @@ -88,7 +106,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { } else { val jobSet = new JobSet(time, jobs) jobSets.put(time, jobSet) - jobSet.jobs.foreach(job => executor.execute(new JobHandler(job))) + jobSet.jobs.foreach(job => jobExecutor.execute(new JobHandler(job))) logInfo("Added jobs for time " + time) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index cad68e248ab29..067e804202236 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -17,20 +17,14 @@ package org.apache.spark.streaming.scheduler -import org.apache.spark.streaming.dstream.{NetworkInputDStream, NetworkReceiver} -import org.apache.spark.streaming.dstream.{StopReceiver, ReportBlock, ReportError} -import org.apache.spark.{SparkException, Logging, SparkEnv} -import org.apache.spark.SparkContext._ - -import scala.collection.mutable.HashMap -import scala.collection.mutable.Queue -import scala.concurrent.duration._ +import scala.collection.mutable.{HashMap, Queue, SynchronizedMap} import akka.actor._ -import akka.pattern.ask -import akka.dispatch._ +import org.apache.spark.{Logging, SparkEnv, SparkException} +import org.apache.spark.SparkContext._ import org.apache.spark.storage.BlockId -import org.apache.spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.{StreamingContext, Time} +import org.apache.spark.streaming.dstream.{NetworkReceiver, StopReceiver} import org.apache.spark.util.AkkaUtils private[streaming] sealed trait NetworkInputTrackerMessage @@ -52,8 +46,8 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { val networkInputStreams = ssc.graph.getNetworkInputStreams() val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) val receiverExecutor = new ReceiverExecutor() - val receiverInfo = new HashMap[Int, ActorRef] - val receivedBlockIds = new HashMap[Int, Queue[BlockId]] + val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] + val receivedBlockIds = new HashMap[Int, Queue[BlockId]] with SynchronizedMap[Int, Queue[BlockId]] val timeout = AkkaUtils.askTimeout(ssc.conf) @@ -63,7 +57,7 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { var currentTime: Time = null /** Start the actor and receiver execution thread. */ - def start() { + def start() = synchronized { if (actor != null) { throw new SparkException("NetworkInputTracker already started") } @@ -77,72 +71,99 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } /** Stop the receiver execution thread. */ - def stop() { + def stop() = synchronized { if (!networkInputStreams.isEmpty && actor != null) { - receiverExecutor.interrupt() - receiverExecutor.stopReceivers() + // First, stop the receivers + receiverExecutor.stop() + + // Finally, stop the actor ssc.env.actorSystem.stop(actor) + actor = null logInfo("NetworkInputTracker stopped") } } - /** Return all the blocks received from a receiver. */ - def getBlockIds(receiverId: Int, time: Time): Array[BlockId] = synchronized { - val queue = receivedBlockIds.synchronized { - receivedBlockIds.getOrElse(receiverId, new Queue[BlockId]()) + /** Register a receiver */ + def registerReceiver(streamId: Int, receiverActor: ActorRef, sender: ActorRef) { + if (!networkInputStreamMap.contains(streamId)) { + throw new Exception("Register received for unexpected id " + streamId) } - val result = queue.synchronized { - queue.dequeueAll(x => true) - } - logInfo("Stream " + receiverId + " received " + result.size + " blocks") - result.toArray + receiverInfo += ((streamId, receiverActor)) + logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) + } + + /** Deregister a receiver */ + def deregisterReceiver(streamId: Int, message: String) { + receiverInfo -= streamId + logError("Deregistered receiver for network stream " + streamId + " with message:\n" + message) + } + + /** Get all the received blocks for the given stream. */ + def getBlocks(streamId: Int, time: Time): Array[BlockId] = { + val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]()) + val result = queue.dequeueAll(x => true).toArray + logInfo("Stream " + streamId + " received " + result.size + " blocks") + result + } + + /** Add new blocks for the given stream */ + def addBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) = { + val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]) + queue ++= blockIds + networkInputStreamMap(streamId).addMetadata(metadata) + logDebug("Stream " + streamId + " received new blocks: " + blockIds.mkString("[", ", ", "]")) + } + + /** Check if any blocks are left to be processed */ + def hasMoreReceivedBlockIds: Boolean = { + !receivedBlockIds.forall(_._2.isEmpty) } /** Actor to receive messages from the receivers. */ private class NetworkInputTrackerActor extends Actor { def receive = { - case RegisterReceiver(streamId, receiverActor) => { - if (!networkInputStreamMap.contains(streamId)) { - throw new Exception("Register received for unexpected id " + streamId) - } - receiverInfo += ((streamId, receiverActor)) - logInfo("Registered receiver for network stream " + streamId + " from " - + sender.path.address) + case RegisterReceiver(streamId, receiverActor) => + registerReceiver(streamId, receiverActor, sender) + sender ! true + case AddBlocks(streamId, blockIds, metadata) => + addBlocks(streamId, blockIds, metadata) + case DeregisterReceiver(streamId, message) => + deregisterReceiver(streamId, message) sender ! true - } - case AddBlocks(streamId, blockIds, metadata) => { - val tmp = receivedBlockIds.synchronized { - if (!receivedBlockIds.contains(streamId)) { - receivedBlockIds += ((streamId, new Queue[BlockId])) - } - receivedBlockIds(streamId) - } - tmp.synchronized { - tmp ++= blockIds - } - networkInputStreamMap(streamId).addMetadata(metadata) - } - case DeregisterReceiver(streamId, msg) => { - receiverInfo -= streamId - logError("De-registered receiver for network stream " + streamId - + " with message " + msg) - // TODO: Do something about the corresponding NetworkInputDStream - } } } /** This thread class runs all the receivers on the cluster. */ - class ReceiverExecutor extends Thread { - val env = ssc.env - - override def run() { - try { - SparkEnv.set(env) - startReceivers() - } catch { - case ie: InterruptedException => logInfo("ReceiverExecutor interrupted") - } finally { - stopReceivers() + class ReceiverExecutor { + @transient val env = ssc.env + @transient val thread = new Thread() { + override def run() { + try { + SparkEnv.set(env) + startReceivers() + } catch { + case ie: InterruptedException => logInfo("ReceiverExecutor interrupted") + } + } + } + + def start() { + thread.start() + } + + def stop() { + // Send the stop signal to all the receivers + stopReceivers() + + // Wait for the Spark job that runs the receivers to be over + // That is, for the receivers to quit gracefully. + thread.join(10000) + + // Check if all the receivers have been deregistered or not + if (!receiverInfo.isEmpty) { + logWarning("All of the receivers have not deregistered, " + receiverInfo) + } else { + logInfo("All of the receivers have deregistered successfully") } } @@ -150,7 +171,7 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { * Get the receivers from the NetworkInputDStreams, distributes them to the * worker nodes as a parallel collection, and runs them. */ - def startReceivers() { + private def startReceivers() { val receivers = networkInputStreams.map(nis => { val rcvr = nis.getReceiver() rcvr.setStreamId(nis.id) @@ -186,13 +207,16 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } // Distribute the receivers and start them + logInfo("Starting " + receivers.length + " receivers") ssc.sparkContext.runJob(tempRDD, startReceiver) + logInfo("All of the receivers have been terminated") } /** Stops the receivers. */ - def stopReceivers() { + private def stopReceivers() { // Signal the receivers to stop receiverInfo.values.foreach(_ ! StopReceiver) + logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index c3a849d2769a7..c5ef2cc8c390d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -48,14 +48,11 @@ class SystemClock() extends Clock { minPollTime } } - - + while (true) { currentTime = System.currentTimeMillis() waitTime = targetTime - currentTime - if (waitTime <= 0) { - return currentTime } val sleepTime = diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index 559c2473851b3..f71938ac55ccb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -17,44 +17,84 @@ package org.apache.spark.streaming.util +import org.apache.spark.Logging + private[streaming] -class RecurringTimer(val clock: Clock, val period: Long, val callback: (Long) => Unit) { +class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: String) + extends Logging { - private val thread = new Thread("RecurringTimer") { + private val thread = new Thread("RecurringTimer - " + name) { + setDaemon(true) override def run() { loop } } - - private var nextTime = 0L + @volatile private var prevTime = -1L + @volatile private var nextTime = -1L + @volatile private var stopped = false + + /** + * Get the time when this timer will fire if it is started right now. + * The time will be a multiple of this timer's period and more than + * current system time. + */ def getStartTime(): Long = { (math.floor(clock.currentTime.toDouble / period) + 1).toLong * period } + /** + * Get the time when the timer will fire if it is restarted right now. + * This time depends on when the timer was started the first time, and was stopped + * for whatever reason. The time must be a multiple of this timer's period and + * more than current time. + */ def getRestartTime(originalStartTime: Long): Long = { val gap = clock.currentTime - originalStartTime (math.floor(gap.toDouble / period).toLong + 1) * period + originalStartTime } - def start(startTime: Long): Long = { + /** + * Start at the given start time. + */ + def start(startTime: Long): Long = synchronized { nextTime = startTime thread.start() + logInfo("Started timer for " + name + " at time " + nextTime) nextTime } + /** + * Start at the earliest time it can start based on the period. + */ def start(): Long = { start(getStartTime()) } - def stop() { - thread.interrupt() + /** + * Stop the timer, and return the last time the callback was made. + * interruptTimer = true will interrupt the callback + * if it is in progress (not guaranteed to give correct time in this case). + */ + def stop(interruptTimer: Boolean): Long = synchronized { + if (!stopped) { + stopped = true + if (interruptTimer) thread.interrupt() + thread.join() + logInfo("Stopped timer for " + name + " after time " + prevTime) + } + prevTime } - + + /** + * Repeatedly call the callback every interval. + */ private def loop() { try { - while (true) { + while (!stopped) { clock.waitTillTime(nextTime) callback(nextTime) + prevTime = nextTime nextTime += period + logDebug("Callback for " + name + " called at time " + prevTime) } } catch { case e: InterruptedException => @@ -74,10 +114,10 @@ object RecurringTimer { println("" + currentTime + ": " + (currentTime - lastRecurTime)) lastRecurTime = currentTime } - val timer = new RecurringTimer(new SystemClock(), period, onRecur) + val timer = new RecurringTimer(new SystemClock(), period, onRecur, "Test") timer.start() Thread.sleep(30 * 1000) - timer.stop() + timer.stop(true) } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index bcb0c28bf07a0..bb73dbf29b649 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -324,7 +324,7 @@ class BasicOperationsSuite extends TestSuiteBase { val updateStateOperation = (s: DStream[String]) => { val updateFunc = (values: Seq[Int], state: Option[Int]) => { - Some(values.foldLeft(0)(_ + _) + state.getOrElse(0)) + Some(values.sum + state.getOrElse(0)) } s.map(x => (x, 1)).updateStateByKey[Int](updateFunc) } @@ -359,7 +359,7 @@ class BasicOperationsSuite extends TestSuiteBase { // updateFunc clears a state when a StateObject is seen without new values twice in a row val updateFunc = (values: Seq[Int], state: Option[StateObject]) => { val stateObj = state.getOrElse(new StateObject) - values.foldLeft(0)(_ + _) match { + values.sum match { case 0 => stateObj.expireCounter += 1 // no new values case n => { // has new values, increment and reset expireCounter stateObj.counter += n diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 717da8e00462b..9cc27ef7f03b5 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -17,19 +17,22 @@ package org.apache.spark.streaming -import org.scalatest.{FunSuite, BeforeAndAfter} -import org.scalatest.exceptions.TestFailedDueToTimeoutException +import java.util.concurrent.atomic.AtomicInteger + +import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} +import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.{DStream, NetworkReceiver} +import org.apache.spark.util.{MetadataCleaner, Utils} +import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Timeouts +import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ -import org.apache.spark.{SparkException, SparkConf, SparkContext} -import org.apache.spark.util.{Utils, MetadataCleaner} -import org.apache.spark.streaming.dstream.DStream -class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { +class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { val master = "local[2]" val appName = this.getClass.getSimpleName - val batchDuration = Seconds(1) + val batchDuration = Milliseconds(500) val sparkHome = "someDir" val envPair = "key" -> "value" val ttl = StreamingContext.DEFAULT_CLEANER_TTL + 100 @@ -108,19 +111,31 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) myConf.set("spark.cleaner.ttl", ttl.toString) val ssc1 = new StreamingContext(myConf, batchDuration) + addInputStream(ssc1).register + ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) assert(MetadataCleaner.getDelaySeconds(cp.sparkConf) === ttl) ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) assert(MetadataCleaner.getDelaySeconds(newCp.sparkConf) === ttl) - ssc = new StreamingContext(null, cp, null) + ssc = new StreamingContext(null, newCp, null) assert(MetadataCleaner.getDelaySeconds(ssc.conf) === ttl) } - test("start multiple times") { + test("start and stop state check") { ssc = new StreamingContext(master, appName, batchDuration) addInputStream(ssc).register + assert(ssc.state === ssc.StreamingContextState.Initialized) + ssc.start() + assert(ssc.state === ssc.StreamingContextState.Started) + ssc.stop() + assert(ssc.state === ssc.StreamingContextState.Stopped) + } + + test("start multiple times") { + ssc = new StreamingContext(master, appName, batchDuration) + addInputStream(ssc).register ssc.start() intercept[SparkException] { ssc.start() @@ -133,18 +148,61 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { ssc.start() ssc.stop() ssc.stop() - ssc = null } + test("stop before start and start after stop") { + ssc = new StreamingContext(master, appName, batchDuration) + addInputStream(ssc).register + ssc.stop() // stop before start should not throw exception + ssc.start() + ssc.stop() + intercept[SparkException] { + ssc.start() // start after stop should throw exception + } + } + + test("stop only streaming context") { ssc = new StreamingContext(master, appName, batchDuration) sc = ssc.sparkContext addInputStream(ssc).register ssc.start() ssc.stop(false) - ssc = null assert(sc.makeRDD(1 to 100).collect().size === 100) ssc = new StreamingContext(sc, batchDuration) + addInputStream(ssc).register + ssc.start() + ssc.stop() + } + + test("stop gracefully") { + val conf = new SparkConf().setMaster(master).setAppName(appName) + conf.set("spark.cleaner.ttl", "3600") + sc = new SparkContext(conf) + for (i <- 1 to 4) { + logInfo("==================================") + ssc = new StreamingContext(sc, batchDuration) + var runningCount = 0 + TestReceiver.counter.set(1) + val input = ssc.networkStream(new TestReceiver) + input.count.foreachRDD(rdd => { + val count = rdd.first() + logInfo("Count = " + count) + runningCount += count.toInt + }) + ssc.start() + ssc.awaitTermination(500) + ssc.stop(stopSparkContext = false, stopGracefully = true) + logInfo("Running count = " + runningCount) + logInfo("TestReceiver.counter = " + TestReceiver.counter.get()) + assert(runningCount > 0) + assert( + (TestReceiver.counter.get() == runningCount + 1) || + (TestReceiver.counter.get() == runningCount + 2), + "Received records = " + TestReceiver.counter.get() + ", " + + "processed records = " + runningCount + ) + } } test("awaitTermination") { @@ -199,7 +257,6 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { test("awaitTermination with error in job generation") { ssc = new StreamingContext(master, appName, batchDuration) val inputStream = addInputStream(ssc) - inputStream.transform(rdd => { throw new TestException("error in transform"); rdd }).register val exception = intercept[TestException] { ssc.start() @@ -215,4 +272,29 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts { } } -class TestException(msg: String) extends Exception(msg) \ No newline at end of file +class TestException(msg: String) extends Exception(msg) + +/** Custom receiver for testing whether all data received by a receiver gets processed or not */ +class TestReceiver extends NetworkReceiver[Int] { + protected lazy val blockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY) + protected def onStart() { + blockGenerator.start() + logInfo("BlockGenerator started on thread " + receivingThread) + try { + while(true) { + blockGenerator += TestReceiver.counter.getAndIncrement + Thread.sleep(0) + } + } finally { + logInfo("Receiving stopped at count value of " + TestReceiver.counter.get()) + } + } + + protected def onStop() { + blockGenerator.stop() + } +} + +object TestReceiver { + val counter = new AtomicInteger(1) +} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index 201630672ab4c..aa2d5c2fc2454 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -277,7 +277,7 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { assert(timeTaken < maxWaitTimeMillis, "Operation timed out after " + timeTaken + " ms") assert(output.size === numExpectedOutput, "Unexpected number of outputs generated") - Thread.sleep(500) // Give some time for the forgetting old RDDs to complete + Thread.sleep(100) // Give some time for the forgetting old RDDs to complete } catch { case e: Exception => {e.printStackTrace(); throw e} } finally { From 6dc5f5849c0e0378abc6648c919412827d831641 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 8 Apr 2014 01:03:33 -0700 Subject: [PATCH 232/397] [SPARK-1396] Properly cleanup DAGScheduler on job cancellation. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Previously, when jobs were cancelled, not all of the state in the DAGScheduler was cleaned up, leading to a slow memory leak in the DAGScheduler. As we expose easier ways to cancel jobs, it's more important to fix these issues. This commit also fixes a second and less serious problem, which is that previously, when a stage failed, not all of the appropriate stages were cancelled. See the "failure of stage used by two jobs" test for an example of this. This just meant that extra work was done, and is not a correctness problem. This commit adds 3 tests. “run shuffle with map stage failure†is a new test to more thoroughly test this functionality, and passes on both the old and new versions of the code. “trivial job cancellation†fails on the old code because all state wasn’t cleaned up correctly when jobs were cancelled (we didn’t remove the job from resultStageToJob). “failure of stage used by two jobs†fails on the old code because taskScheduler.cancelTasks wasn’t called for one of the stages (see test comments). This should be checked in before #246, which makes it easier to cancel stages / jobs. Author: Kay Ousterhout Closes #305 from kayousterhout/incremental_abort_fix and squashes the following commits: f33d844 [Kay Ousterhout] Mark review comments 9217080 [Kay Ousterhout] Properly cleanup DAGScheduler on job cancellation. --- .../apache/spark/scheduler/DAGScheduler.scala | 44 +++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 92 ++++++++++++++++++- 2 files changed, 115 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index 6368665f249ee..c96d7435a7ed4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -982,15 +982,7 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - val independentStages = removeJobAndIndependentStages(jobId) - independentStages.foreach(taskScheduler.cancelTasks) - val error = new SparkException("Job %d cancelled".format(jobId)) - val job = jobIdToActiveJob(jobId) - job.listener.jobFailed(error) - jobIdToStageIds -= jobId - activeJobs -= job - jobIdToActiveJob -= jobId - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled") } } @@ -1007,19 +999,39 @@ class DAGScheduler( stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) - val error = new SparkException("Job aborted: " + reason) - job.listener.jobFailed(error) - jobIdToStageIdsRemove(job.jobId) - jobIdToActiveJob -= resultStage.jobId - activeJobs -= job - resultStageToJob -= resultStage - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, failedStage.id))) + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") } } + /** + * Fails a job and all stages that are only used by that job, and cleans up relevant state. + */ + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { + val error = new SparkException(failureReason) + job.listener.jobFailed(error) + + // Cancel all tasks in independent stages. + val independentStages = removeJobAndIndependentStages(job.jobId) + independentStages.foreach(taskScheduler.cancelTasks) + + // Clean up remaining state we store for the job. + jobIdToActiveJob -= job.jobId + activeJobs -= job + jobIdToStageIds -= job.jobId + val resultStagesForJob = resultStageToJob.keySet.filter( + stage => resultStageToJob(stage).jobId == job.jobId) + if (resultStagesForJob.size != 1) { + logWarning( + s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + } + resultStageToJob --= resultStagesForJob + + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + } + /** * Return true if one of stage's ancestors is target. */ diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ce567b0cde85d..2e3026bffba2f 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -18,7 +18,7 @@ package org.apache.spark.scheduler import scala.Tuple2 -import scala.collection.mutable.{HashMap, Map} +import scala.collection.mutable.{HashSet, HashMap, Map} import org.scalatest.{BeforeAndAfter, FunSuite} @@ -43,6 +43,10 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() + + /** Stages for which the DAGScheduler has called TaskScheduler.cancelTasks(). */ + val cancelledStages = new HashSet[Int]() + val taskScheduler = new TaskScheduler() { override def rootPool: Pool = null override def schedulingMode: SchedulingMode = SchedulingMode.NONE @@ -53,7 +57,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) taskSets += taskSet } - override def cancelTasks(stageId: Int) {} + override def cancelTasks(stageId: Int) { + cancelledStages += stageId + } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} override def defaultParallelism() = 2 } @@ -91,6 +97,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont before { sc = new SparkContext("local", "DAGSchedulerSuite") taskSets.clear() + cancelledStages.clear() cacheLocations.clear() results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) @@ -174,15 +181,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } } - /** Sends the rdd to the scheduler for scheduling. */ + /** Sends the rdd to the scheduler for scheduling and returns the job id. */ private def submit( rdd: RDD[_], partitions: Array[Int], func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, allowLocal: Boolean = false, - listener: JobListener = listener) { + listener: JobListener = listener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) + return jobId } /** Sends TaskSetFailed to the scheduler. */ @@ -190,6 +198,11 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runEvent(TaskSetFailed(taskSet, message)) } + /** Sends JobCancelled to the DAG scheduler. */ + private def cancel(jobId: Int) { + runEvent(JobCancelled(jobId)) + } + test("zero split job") { val rdd = makeRdd(0, Nil) var numResults = 0 @@ -248,7 +261,15 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont test("trivial job failure") { submit(makeRdd(1, Nil), Array(0)) failed(taskSets(0), "some failure") - assert(failure.getMessage === "Job aborted: some failure") + assert(failure.getMessage === "Job aborted due to stage failure: some failure") + assertDataStructuresEmpty + } + + test("trivial job cancellation") { + val rdd = makeRdd(1, Nil) + val jobId = submit(rdd, Array(0)) + cancel(jobId) + assert(failure.getMessage === s"Job $jobId cancelled") assertDataStructuresEmpty } @@ -323,6 +344,67 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertDataStructuresEmpty } + test("run shuffle with map stage failure") { + val shuffleMapRdd = makeRdd(2, Nil) + val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) + val reduceRdd = makeRdd(2, List(shuffleDep)) + submit(reduceRdd, Array(0, 1)) + + // Fail the map stage. This should cause the entire job to fail. + val stageFailureMessage = "Exception failure in map stage" + failed(taskSets(0), stageFailureMessage) + assert(failure.getMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assertDataStructuresEmpty + } + + /** + * Makes sure that failures of stage used by multiple jobs are correctly handled. + * + * This test creates the following dependency graph: + * + * shuffleMapRdd1 shuffleMapRDD2 + * | \ | + * | \ | + * | \ | + * | \ | + * reduceRdd1 reduceRdd2 + * + * We start both shuffleMapRdds and then fail shuffleMapRdd1. As a result, the job listeners for + * reduceRdd1 and reduceRdd2 should both be informed that the job failed. shuffleMapRDD2 should + * also be cancelled, because it is only used by reduceRdd2 and reduceRdd2 cannot complete + * without shuffleMapRdd1. + */ + test("failure of stage used by two jobs") { + val shuffleMapRdd1 = makeRdd(2, Nil) + val shuffleDep1 = new ShuffleDependency(shuffleMapRdd1, null) + val shuffleMapRdd2 = makeRdd(2, Nil) + val shuffleDep2 = new ShuffleDependency(shuffleMapRdd2, null) + + val reduceRdd1 = makeRdd(2, List(shuffleDep1)) + val reduceRdd2 = makeRdd(2, List(shuffleDep1, shuffleDep2)) + + // We need to make our own listeners for this test, since by default submit uses the same + // listener for all jobs, and here we want to capture the failure for each job separately. + class FailureRecordingJobListener() extends JobListener { + var failureMessage: String = _ + override def taskSucceeded(index: Int, result: Any) {} + override def jobFailed(exception: Exception) = { failureMessage = exception.getMessage } + } + val listener1 = new FailureRecordingJobListener() + val listener2 = new FailureRecordingJobListener() + + submit(reduceRdd1, Array(0, 1), listener=listener1) + submit(reduceRdd2, Array(0, 1), listener=listener2) + + val stageFailureMessage = "Exception failure in map stage" + failed(taskSets(0), stageFailureMessage) + + assert(cancelledStages.contains(1)) + assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") + assertDataStructuresEmpty + } + test("run trivial shuffle with out-of-band failure and retry") { val shuffleMapRdd = makeRdd(2, Nil) val shuffleDep = new ShuffleDependency(shuffleMapRdd, null) From 3bc054893bf2decdafa97a1e149e489ad154f066 Mon Sep 17 00:00:00 2001 From: Henry Saputra Date: Tue, 8 Apr 2014 14:23:16 -0700 Subject: [PATCH 233/397] Remove extra semicolon in import statement and unused import in ApplicationMaster Small nit cleanup to remove extra semicolon and unused import in Yarn's stable ApplicationMaster (it bothers me every time I saw it) Author: Henry Saputra Closes #358 from hsaputra/nitcleanup_removesemicolon_import_applicationmaster and squashes the following commits: bffb685 [Henry Saputra] Remove extra semicolon in import statement and unused import in ApplicationMaster.scala --- .../scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala | 3 +-- 1 file changed, 1 insertion(+), 2 deletions(-) diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 30735cbfdf26e..c8a4d2e647cbd 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -18,7 +18,6 @@ package org.apache.spark.deploy.yarn import java.io.IOException -import java.net.Socket import java.util.concurrent.CopyOnWriteArrayList import java.util.concurrent.atomic.{AtomicInteger, AtomicReference} @@ -36,7 +35,7 @@ import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest import org.apache.hadoop.yarn.conf.YarnConfiguration import org.apache.hadoop.yarn.ipc.YarnRPC import org.apache.hadoop.yarn.util.{ConverterUtils, Records} -import org.apache.hadoop.yarn.webapp.util.WebAppUtils; +import org.apache.hadoop.yarn.webapp.util.WebAppUtils import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil From a8d86b080ae26c96b078ba14dc60f3b528c07787 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Tue, 8 Apr 2014 14:30:24 -0700 Subject: [PATCH 234/397] SPARK-1348 binding Master, Worker, and App Web UI to all interfaces Author: Kan Zhang Closes #318 from kanzhang/SPARK-1348 and squashes the following commits: e625a5f [Kan Zhang] reverting the changes to startJettyServer() 7a8084e [Kan Zhang] SPARK-1348 binding Master, Worker, and App Web UI to all interfaces --- .../scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala | 2 +- .../scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala | 2 +- core/src/main/scala/org/apache/spark/ui/SparkUI.scala | 2 +- 3 files changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index bd75b2dfd0e07..01d9f52f4b7b4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -59,7 +59,7 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { def bind() { try { - serverInfo = Some(startJettyServer(host, port, handlers, master.conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index de76a5d5eb7bc..650f3da5ce3ff 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -60,7 +60,7 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I def bind() { try { - serverInfo = Some(JettyUtils.startJettyServer(host, port, handlers, worker.conf)) + serverInfo = Some(JettyUtils.startJettyServer("0.0.0.0", port, handlers, worker.conf)) logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index ef1ad872c8ef7..f53df7fbedf39 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -80,7 +80,7 @@ private[spark] class SparkUI( /** Bind the HTTP server which backs this web interface */ def bind() { try { - serverInfo = Some(startJettyServer(bindHost, port, handlers, sc.conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) } catch { case e: Exception => From e25b593447a2e0aab9e5066f755e41be9068ecdc Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Tue, 8 Apr 2014 14:40:20 -0700 Subject: [PATCH 235/397] SPARK-1445: compute-classpath should not print error if lib_managed not found This was added to the check for the assembly jar, forgot it for the datanucleus jars. Author: Aaron Davidson Closes #361 from aarondav/cc and squashes the following commits: 8facc16 [Aaron Davidson] SPARK-1445: compute-classpath should not print error if lib_managed not found --- bin/compute-classpath.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index be37102dc069a..2a2bb376fd71f 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -63,7 +63,7 @@ fi # built with Hive, so first check if the datanucleus jars exist, and then ensure the current Spark # assembly is built for Hive, before actually populating the CLASSPATH with the jars. # Note that this check order is faster (by up to half a second) in the case where Hive is not used. -num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ | grep "datanucleus-.*\\.jar" | wc -l) +num_datanucleus_jars=$(ls "$FWDIR"/lib_managed/jars/ 2>/dev/null | grep "datanucleus-.*\\.jar" | wc -l) if [ $num_datanucleus_jars -gt 0 ]; then AN_ASSEMBLY_JAR=${ASSEMBLY_JAR:-$DEPS_ASSEMBLY_JAR} num_hive_files=$(jar tvf "$AN_ASSEMBLY_JAR" org/apache/hadoop/hive/ql/exec 2>/dev/null | wc -l) From fac6085cd774a4dba73ad1618537ef1817b2bcf3 Mon Sep 17 00:00:00 2001 From: Kay Ousterhout Date: Tue, 8 Apr 2014 14:42:02 -0700 Subject: [PATCH 236/397] [SPARK-1397] Notify SparkListeners when stages fail or are cancelled. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit [I wanted to post this for folks to comment but it depends on (and thus includes the changes in) a currently outstanding PR, #305. You can look at just the second commit: https://github.com/kayousterhout/spark-1/commit/93f08baf731b9eaf5c9792a5373560526e2bccac to see just the changes relevant to this PR] Previously, when stages fail or get cancelled, the SparkListener is only notified indirectly through the SparkListenerJobEnd, where we sometimes pass in a single stage that failed. This worked before job cancellation, because jobs would only fail due to a single stage failure. However, with job cancellation, multiple running stages can fail when a job gets cancelled. Right now, this is not handled correctly, which results in stages that get stuck in the “Running Stages†window in the UI even though they’re dead. This PR changes the SparkListenerStageCompleted event to a SparkListenerStageEnded event, and uses this event to tell SparkListeners when stages fail in addition to when they complete successfully. This change is NOT publicly backward compatible for two reasons. First, it changes the SparkListener interface. We could alternately add a new event, SparkListenerStageFailed, and keep the existing SparkListenerStageCompleted. However, this is less consistent with the listener events for tasks / jobs ending, and will result in some code duplication for listeners (because failed and completed stages are handled in similar ways). Note that I haven’t finished updating the JSON code to correctly handle the new event because I’m waiting for feedback on whether this is a good or bad idea (hence the “WIPâ€). It is also not backwards compatible because it changes the publicly visible JobWaiter.jobFailed() method to no longer include a stage that caused the failure. I think this change should definitely stay, because with cancellation (as described above), a failure isn’t necessarily caused by a single stage. Author: Kay Ousterhout Closes #309 from kayousterhout/stage_cancellation and squashes the following commits: 5533ecd [Kay Ousterhout] Fixes in response to Mark's review 320c7c7 [Kay Ousterhout] Notify SparkListeners when stages fail or are cancelled. --- .../scala/org/apache/spark/FutureAction.scala | 2 +- .../apache/spark/scheduler/DAGScheduler.scala | 121 +++++++++++------- .../apache/spark/scheduler/JobLogger.scala | 8 +- .../apache/spark/scheduler/JobResult.scala | 3 +- .../apache/spark/scheduler/JobWaiter.scala | 2 +- .../spark/scheduler/SparkListener.scala | 2 +- .../apache/spark/scheduler/StageInfo.scala | 9 ++ .../spark/ui/jobs/JobProgressListener.scala | 23 +--- .../org/apache/spark/util/JsonProtocol.scala | 11 +- .../spark/scheduler/DAGSchedulerSuite.scala | 45 ++++++- .../apache/spark/util/JsonProtocolSuite.scala | 3 +- 11 files changed, 151 insertions(+), 78 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index f2decd14ef6d9..2eec09cd1c795 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -141,7 +141,7 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: private def awaitResult(): Try[T] = { jobWaiter.awaitResult() match { case JobSucceeded => scala.util.Success(resultFunc) - case JobFailed(e: Exception, _) => scala.util.Failure(e) + case JobFailed(e: Exception) => scala.util.Failure(e) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c96d7435a7ed4..c41d6d75a1d49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -342,22 +342,24 @@ class DAGScheduler( } /** - * Removes job and any stages that are not needed by any other job. Returns the set of ids for - * stages that were removed. The associated tasks for those stages need to be cancelled if we - * got here via job cancellation. + * Removes state for job and any stages that are not needed by any other job. Does not + * handle cancelling tasks or notifying the SparkListener about finished jobs/stages/tasks. + * + * @param job The job whose state to cleanup. + * @param resultStage Specifies the result stage for the job; if set to None, this method + * searches resultStagesToJob to find and cleanup the appropriate result stage. */ - private def removeJobAndIndependentStages(jobId: Int): Set[Int] = { - val registeredStages = jobIdToStageIds(jobId) - val independentStages = new HashSet[Int]() - if (registeredStages.isEmpty) { - logError("No stages registered for job " + jobId) + private def cleanupStateForJobAndIndependentStages(job: ActiveJob, resultStage: Option[Stage]) { + val registeredStages = jobIdToStageIds.get(job.jobId) + if (registeredStages.isEmpty || registeredStages.get.isEmpty) { + logError("No stages registered for job " + job.jobId) } else { - stageIdToJobIds.filterKeys(stageId => registeredStages.contains(stageId)).foreach { + stageIdToJobIds.filterKeys(stageId => registeredStages.get.contains(stageId)).foreach { case (stageId, jobSet) => - if (!jobSet.contains(jobId)) { + if (!jobSet.contains(job.jobId)) { logError( "Job %d not registered for stage %d even though that stage was registered for the job" - .format(jobId, stageId)) + .format(job.jobId, stageId)) } else { def removeStage(stageId: Int) { // data structures based on Stage @@ -394,23 +396,28 @@ class DAGScheduler( .format(stageId, stageIdToStage.size)) } - jobSet -= jobId + jobSet -= job.jobId if (jobSet.isEmpty) { // no other job needs this stage - independentStages += stageId removeStage(stageId) } } } } - independentStages.toSet - } + jobIdToStageIds -= job.jobId + jobIdToActiveJob -= job.jobId + activeJobs -= job - private def jobIdToStageIdsRemove(jobId: Int) { - if (!jobIdToStageIds.contains(jobId)) { - logDebug("Trying to remove unregistered job " + jobId) + if (resultStage.isEmpty) { + // Clean up result stages. + val resultStagesForJob = resultStageToJob.keySet.filter( + stage => resultStageToJob(stage).jobId == job.jobId) + if (resultStagesForJob.size != 1) { + logWarning( + s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + } + resultStageToJob --= resultStagesForJob } else { - removeJobAndIndependentStages(jobId) - jobIdToStageIds -= jobId + resultStageToJob -= resultStage.get } } @@ -460,7 +467,7 @@ class DAGScheduler( val waiter = submitJob(rdd, func, partitions, callSite, allowLocal, resultHandler, properties) waiter.awaitResult() match { case JobSucceeded => {} - case JobFailed(exception: Exception, _) => + case JobFailed(exception: Exception) => logInfo("Failed to run " + callSite) throw exception } @@ -606,7 +613,16 @@ class DAGScheduler( for (job <- activeJobs) { val error = new SparkException("Job cancelled because SparkContext was shut down") job.listener.jobFailed(error) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, -1))) + // Tell the listeners that all of the running stages have ended. Don't bother + // cancelling the stages because if the DAG scheduler is stopped, the entire application + // is in the process of getting stopped. + val stageFailedMessage = "Stage cancelled because SparkContext was shut down" + runningStages.foreach { stage => + val info = stageToInfos(stage) + info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(info)) + } + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } return true } @@ -676,7 +692,7 @@ class DAGScheduler( } } catch { case e: Exception => - jobResult = JobFailed(e, job.finalStage.id) + jobResult = JobFailed(e) job.listener.jobFailed(e) } finally { val s = job.finalStage @@ -826,11 +842,8 @@ class DAGScheduler( job.numFinished += 1 // If the whole job has finished, remove it if (job.numFinished == job.numPartitions) { - jobIdToActiveJob -= stage.jobId - activeJobs -= job - resultStageToJob -= stage markStageAsFinished(stage) - jobIdToStageIdsRemove(job.jobId) + cleanupStateForJobAndIndependentStages(job, Some(stage)) listenerBus.post(SparkListenerJobEnd(job.jobId, JobSucceeded)) } job.listener.taskSucceeded(rt.outputId, event.result) @@ -982,7 +995,7 @@ class DAGScheduler( if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled") + failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled", None) } } @@ -999,7 +1012,8 @@ class DAGScheduler( stageToInfos(failedStage).completionTime = Some(System.currentTimeMillis()) for (resultStage <- dependentStages) { val job = resultStageToJob(resultStage) - failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason") + failJobAndIndependentStages(job, s"Job aborted due to stage failure: $reason", + Some(resultStage)) } if (dependentStages.isEmpty) { logInfo("Ignoring failure of " + failedStage + " because all jobs depending on it are done") @@ -1008,28 +1022,45 @@ class DAGScheduler( /** * Fails a job and all stages that are only used by that job, and cleans up relevant state. + * + * @param resultStage The result stage for the job, if known. Used to cleanup state for the job + * slightly more efficiently than when not specified. */ - private def failJobAndIndependentStages(job: ActiveJob, failureReason: String) { + private def failJobAndIndependentStages(job: ActiveJob, failureReason: String, + resultStage: Option[Stage]) { val error = new SparkException(failureReason) job.listener.jobFailed(error) - // Cancel all tasks in independent stages. - val independentStages = removeJobAndIndependentStages(job.jobId) - independentStages.foreach(taskScheduler.cancelTasks) - - // Clean up remaining state we store for the job. - jobIdToActiveJob -= job.jobId - activeJobs -= job - jobIdToStageIds -= job.jobId - val resultStagesForJob = resultStageToJob.keySet.filter( - stage => resultStageToJob(stage).jobId == job.jobId) - if (resultStagesForJob.size != 1) { - logWarning( - s"${resultStagesForJob.size} result stages for job ${job.jobId} (expect exactly 1)") + // Cancel all independent, running stages. + val stages = jobIdToStageIds(job.jobId) + if (stages.isEmpty) { + logError("No stages registered for job " + job.jobId) } - resultStageToJob --= resultStagesForJob + stages.foreach { stageId => + val jobsForStage = stageIdToJobIds.get(stageId) + if (jobsForStage.isEmpty || !jobsForStage.get.contains(job.jobId)) { + logError( + "Job %d not registered for stage %d even though that stage was registered for the job" + .format(job.jobId, stageId)) + } else if (jobsForStage.get.size == 1) { + if (!stageIdToStage.contains(stageId)) { + logError("Missing Stage for stage with id $stageId") + } else { + // This is the only job that uses this stage, so fail the stage if it is running. + val stage = stageIdToStage(stageId) + if (runningStages.contains(stage)) { + taskScheduler.cancelTasks(stageId) + val stageInfo = stageToInfos(stage) + stageInfo.stageFailed(failureReason) + listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) + } + } + } + } + + cleanupStateForJobAndIndependentStages(job, resultStage) - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error, job.finalStage.id))) + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 5cecf9416b32c..7c5053998f1d6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -191,7 +191,11 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener */ override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { val stageId = stageCompleted.stageInfo.stageId - stageLogInfo(stageId, "STAGE_ID=%d STATUS=COMPLETED".format(stageId)) + if (stageCompleted.stageInfo.failureReason.isEmpty) { + stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=COMPLETED") + } else { + stageLogInfo(stageId, s"STAGE_ID=$stageId STATUS=FAILED") + } } /** @@ -227,7 +231,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener var info = "JOB_ID=" + jobId jobEnd.jobResult match { case JobSucceeded => info += " STATUS=SUCCESS" - case JobFailed(exception, _) => + case JobFailed(exception) => info += " STATUS=FAILED REASON=" exception.getMessage.split("\\s+").foreach(info += _ + "_") case _ => diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 3cf4e3077e4a4..047bd27056120 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -24,5 +24,4 @@ private[spark] sealed trait JobResult private[spark] case object JobSucceeded extends JobResult -// A failed stage ID of -1 means there is not a particular stage that caused the failure -private[spark] case class JobFailed(exception: Exception, failedStageId: Int) extends JobResult +private[spark] case class JobFailed(exception: Exception) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala index 8007b5418741e..e9bfee2248e5b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobWaiter.scala @@ -64,7 +64,7 @@ private[spark] class JobWaiter[T]( override def jobFailed(exception: Exception): Unit = synchronized { _jobFinished = true - jobResult = JobFailed(exception, -1) + jobResult = JobFailed(exception) this.notifyAll() } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index d4eb0ac88d8e8..d42e67742a4f7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -71,7 +71,7 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent */ trait SparkListener { /** - * Called when a stage is completed, with information on the completed stage + * Called when a stage completes successfully or fails, with information on the completed stage. */ def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { } diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 8115a7ed7896d..eec409b182ac6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -26,8 +26,17 @@ private[spark] class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None + /** Time when all tasks in the stage completed or when the stage was cancelled. */ var completionTime: Option[Long] = None + /** If the stage failed, the reason why. */ + var failureReason: Option[String] = None + var emittedTaskSizeWarning = false + + def stageFailed(reason: String) { + failureReason = Some(reason) + completionTime = Some(System.currentTimeMillis) + } } private[spark] diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 048f671c8788f..5167e20ea3d7d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -74,8 +74,13 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage poolToActiveStages(stageIdToPool(stageId)).remove(stageId) activeStages.remove(stageId) - completedStages += stage - trimIfNecessary(completedStages) + if (stage.failureReason.isEmpty) { + completedStages += stage + trimIfNecessary(completedStages) + } else { + failedStages += stage + trimIfNecessary(failedStages) + } } /** If stages is too large, remove and garbage collect old stages */ @@ -215,20 +220,6 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { } } - override def onJobEnd(jobEnd: SparkListenerJobEnd) = synchronized { - jobEnd.jobResult match { - case JobFailed(_, stageId) => - activeStages.get(stageId).foreach { s => - // Remove by stageId, rather than by StageInfo, in case the StageInfo is from storage - activeStages.remove(s.stageId) - poolToActiveStages(stageIdToPool(stageId)).remove(s.stageId) - failedStages += s - trimIfNecessary(failedStages) - } - case _ => - } - } - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { val schedulingModeName = diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 2155a8888c85c..19654892bf661 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -166,12 +166,14 @@ private[spark] object JsonProtocol { val rddInfo = rddInfoToJson(stageInfo.rddInfo) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) + val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing) ("Stage ID" -> stageInfo.stageId) ~ ("Stage Name" -> stageInfo.name) ~ ("Number of Tasks" -> stageInfo.numTasks) ~ ("RDD Info" -> rddInfo) ~ ("Submission Time" -> submissionTime) ~ ("Completion Time" -> completionTime) ~ + ("Failure Reason" -> failureReason) ~ ("Emitted Task Size Warning" -> stageInfo.emittedTaskSizeWarning) } @@ -259,9 +261,7 @@ private[spark] object JsonProtocol { val json = jobResult match { case JobSucceeded => Utils.emptyJson case jobFailed: JobFailed => - val exception = exceptionToJson(jobFailed.exception) - ("Exception" -> exception) ~ - ("Failed Stage ID" -> jobFailed.failedStageId) + JObject("Exception" -> exceptionToJson(jobFailed.exception)) } ("Result" -> result) ~ json } @@ -442,11 +442,13 @@ private[spark] object JsonProtocol { val rddInfo = rddInfoFromJson(json \ "RDD Info") val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) + val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime + stageInfo.failureReason = failureReason stageInfo.emittedTaskSizeWarning = emittedTaskSizeWarning stageInfo } @@ -561,8 +563,7 @@ private[spark] object JsonProtocol { case `jobSucceeded` => JobSucceeded case `jobFailed` => val exception = exceptionFromJson(json \ "Exception") - val failedStageId = (json \ "Failed Stage ID").extract[Int] - new JobFailed(exception, failedStageId) + new JobFailed(exception) } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 2e3026bffba2f..a74724d785ad3 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -64,6 +64,21 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def defaultParallelism() = 2 } + /** Length of time to wait while draining listener events. */ + val WAIT_TIMEOUT_MILLIS = 10000 + val sparkListener = new SparkListener() { + val successfulStages = new HashSet[Int]() + val failedStages = new HashSet[Int]() + override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) { + val stageInfo = stageCompleted.stageInfo + if (stageInfo.failureReason.isEmpty) { + successfulStages += stageInfo.stageId + } else { + failedStages += stageInfo.stageId + } + } + } + var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null @@ -89,13 +104,16 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont /** The list of results that DAGScheduler has collected. */ val results = new HashMap[Int, Any]() var failure: Exception = _ - val listener = new JobListener() { + val jobListener = new JobListener() { override def taskSucceeded(index: Int, result: Any) = results.put(index, result) override def jobFailed(exception: Exception) = { failure = exception } } before { sc = new SparkContext("local", "DAGSchedulerSuite") + sparkListener.successfulStages.clear() + sparkListener.failedStages.clear() + sc.addSparkListener(sparkListener) taskSets.clear() cancelledStages.clear() cacheLocations.clear() @@ -187,7 +205,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont partitions: Array[Int], func: (TaskContext, Iterator[_]) => _ = jobComputeFunc, allowLocal: Boolean = false, - listener: JobListener = listener): Int = { + listener: JobListener = jobListener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) return jobId @@ -231,7 +249,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont override def toString = "DAGSchedulerSuite Local RDD" } val jobId = scheduler.nextJobId.getAndIncrement() - runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, listener)) + runEvent(JobSubmitted(jobId, rdd, jobComputeFunc, Array(0), true, null, jobListener)) assert(results === Map(0 -> 42)) assertDataStructuresEmpty } @@ -262,6 +280,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont submit(makeRdd(1, Nil), Array(0)) failed(taskSets(0), "some failure") assert(failure.getMessage === "Job aborted due to stage failure: some failure") + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) assertDataStructuresEmpty } @@ -270,6 +291,9 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val jobId = submit(rdd, Array(0)) cancel(jobId) assert(failure.getMessage === s"Job $jobId cancelled") + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(0)) + assert(sparkListener.failedStages.size === 1) assertDataStructuresEmpty } @@ -354,6 +378,13 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val stageFailureMessage = "Exception failure in map stage" failed(taskSets(0), stageFailureMessage) assert(failure.getMessage === s"Job aborted due to stage failure: $stageFailureMessage") + + // Listener bus should get told about the map stage failing, but not the reduce stage + // (since the reduce stage hasn't been started yet). + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.failedStages.contains(1)) + assert(sparkListener.failedStages.size === 1) + assertDataStructuresEmpty } @@ -400,6 +431,14 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont failed(taskSets(0), stageFailureMessage) assert(cancelledStages.contains(1)) + + // Make sure the listeners got told about both failed stages. + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + assert(sparkListener.successfulStages.isEmpty) + assert(sparkListener.failedStages.contains(1)) + assert(sparkListener.failedStages.contains(3)) + assert(sparkListener.failedStages.size === 2) + assert(listener1.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assert(listener2.failureMessage === s"Job aborted due to stage failure: $stageFailureMessage") assertDataStructuresEmpty diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 7bab7da8fed68..0342a8aff3c28 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -89,7 +89,7 @@ class JsonProtocolSuite extends FunSuite { // JobResult val exception = new Exception("Out of Memory! Please restock film.") exception.setStackTrace(stackTrace) - val jobFailed = JobFailed(exception, 2) + val jobFailed = JobFailed(exception) testJobResult(JobSucceeded) testJobResult(jobFailed) @@ -294,7 +294,6 @@ class JsonProtocolSuite extends FunSuite { (result1, result2) match { case (JobSucceeded, JobSucceeded) => case (r1: JobFailed, r2: JobFailed) => - assert(r1.failedStageId === r2.failedStageId) assertEquals(r1.exception, r2.exception) case _ => fail("Job results don't match in types!") } From 12c077d5aa0b76a808a55db625c9677a52bd43f9 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Tue, 8 Apr 2014 16:19:22 -0700 Subject: [PATCH 237/397] SPARK-1433: Upgrade Mesos dependency to 0.17.0 Mesos 0.13.0 was released 6 months ago. Upgrade Mesos dependency to 0.17.0 Author: Sandeep Closes #355 from techaddict/mesos_update and squashes the following commits: f1abeee [Sandeep] SPARK-1433: Upgrade Mesos dependency to 0.17.0 Mesos 0.13.0 was released 6 months ago. Upgrade Mesos dependency to 0.17.0 --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 6 ++++-- .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- docs/_config.yml | 2 +- pom.xml | 6 +++--- project/SparkBuild.scala | 2 +- 5 files changed, 10 insertions(+), 8 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 06b041e1fd9a9..c478e685641d7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -194,10 +194,12 @@ private[spark] class CoarseMesosSchedulerBackend( .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", sc.executorMemory)) .build() - d.launchTasks(offer.getId, Collections.singletonList(task), filters) + d.launchTasks(Collections.singletonList(offer.getId), + Collections.singletonList(task), + filters) } else { // Filter it out - d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters) + d.declineOffer(offer.getId, filters) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index dfdcafe19fb93..f878ae338fc95 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -223,7 +223,7 @@ private[spark] class MesosSchedulerBackend( // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? for (i <- 0 until offers.size) { - d.launchTasks(offers(i).getId, mesosTasks(i), filters) + d.launchTasks(Collections.singletonList(offers(i).getId), mesosTasks(i), filters) } } } finally { diff --git a/docs/_config.yml b/docs/_config.yml index d585b8c5ea763..bd5ed6c9220d2 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,6 +7,6 @@ SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.13.0 +MESOS_VERSION: 0.17.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/pom.xml b/pom.xml index c03bb35c99442..11511bcb9da52 100644 --- a/pom.xml +++ b/pom.xml @@ -112,7 +112,7 @@ 2.10.4 2.10 - 0.13.0 + 0.17.0 org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -848,7 +848,7 @@ - + hadoop-provided @@ -893,6 +893,6 @@
    - + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 6b8740d9f21a1..08667aac2cd2d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -332,7 +332,7 @@ object SparkBuild extends Build { "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.13.0", + "org.apache.mesos" % "mesos" % "0.17.0", "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From ce8ec5456169682f27f846e7b8d51e6c4bcf75e3 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 18:15:52 -0700 Subject: [PATCH 238/397] Spark 1271: Co-Group and Group-By should pass Iterable[X] Author: Holden Karau Closes #242 from holdenk/spark-1320-cogroupandgroupshouldpassiterator and squashes the following commits: f289536 [Holden Karau] Fix bad merge, should have been Iterable rather than Iterator 77048f8 [Holden Karau] Fix merge up to master d3fe909 [Holden Karau] use toSeq instead 7a092a3 [Holden Karau] switch resultitr to resultiterable eb06216 [Holden Karau] maybe I should have had a coffee first. use correct import for guava iterables c5075aa [Holden Karau] If guava 14 had iterables 2d06e10 [Holden Karau] Fix Java 8 cogroup tests for the new API 11e730c [Holden Karau] Fix streaming tests 66b583d [Holden Karau] Fix the core test suite to compile 4ed579b [Holden Karau] Refactor from iterator to iterable d052c07 [Holden Karau] Python tests now pass with iterator pandas 3bcd81d [Holden Karau] Revert "Try and make pickling list iterators work" cd1e81c [Holden Karau] Try and make pickling list iterators work c60233a [Holden Karau] Start investigating moving to iterators for python API like the Java/Scala one. tl;dr: We will have to write our own iterator since the default one doesn't pickle well 88a5cef [Holden Karau] Fix cogroup test in JavaAPISuite for streaming a5ee714 [Holden Karau] oops, was checking wrong iterator e687f21 [Holden Karau] Fix groupbykey test in JavaAPISuite of streaming ec8cc3e [Holden Karau] Fix test issues\! 4b0eeb9 [Holden Karau] Switch cast in PairDStreamFunctions fa395c9 [Holden Karau] Revert "Add a join based on the problem in SVD" ec99e32 [Holden Karau] Revert "Revert this but for now put things in list pandas" b692868 [Holden Karau] Revert 7e533f7 [Holden Karau] Fix the bug 8a5153a [Holden Karau] Revert me, but we have some stuff to debug b4e86a9 [Holden Karau] Add a join based on the problem in SVD c4510e2 [Holden Karau] Revert this but for now put things in list pandas b4e0b1d [Holden Karau] Fix style issues 71e8b9f [Holden Karau] I really need to stop calling size on iterators, it is the path of sadness. b1ae51a [Holden Karau] Fix some of the types in the streaming JavaAPI suite. Probably still needs more work 37888ec [Holden Karau] core/tests now pass 249abde [Holden Karau] org.apache.spark.rdd.PairRDDFunctionsSuite passes 6698186 [Holden Karau] Revert "I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy" fe992fe [Holden Karau] hmmm try and fix up basic operation suite 172705c [Holden Karau] Fix Java API suite caafa63 [Holden Karau] I think this might be a bad rabbit hole. Started work to make CoGroupedRDD use iterator and then went crazy 88b3329 [Holden Karau] Fix groupbykey to actually give back an iterator 4991af6 [Holden Karau] Fix some tests be50246 [Holden Karau] Calling size on an iterator is not so good if we want to use it after 687ffbc [Holden Karau] This is the it compiles point of replacing Seq with Iterator and JList with JIterator in the groupby and cogroup signatures --- .../scala/org/apache/spark/bagel/Bagel.scala | 20 ++++--- .../apache/spark/api/java/JavaPairRDD.scala | 36 ++++++------ .../apache/spark/api/java/JavaRDDLike.scala | 6 +- .../apache/spark/rdd/PairRDDFunctions.scala | 39 +++++++------ .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../java/org/apache/spark/JavaAPISuite.java | 20 ++++--- .../scala/org/apache/spark/FailureSuite.scala | 4 +- .../org/apache/spark/PipedRDDSuite.scala | 2 +- .../spark/rdd/PairRDDFunctionsSuite.scala | 12 ++-- .../ExternalAppendOnlyMapSuite.scala | 4 +- .../apache/spark/examples/JavaPageRank.java | 21 ++++--- .../bagel/WikipediaPageRankStandalone.scala | 14 +++-- .../java/org/apache/spark/Java8APISuite.java | 11 ++-- .../org/apache/spark/mllib/linalg/SVD.scala | 6 +- .../spark/mllib/recommendation/ALS.scala | 4 +- .../org/apache/spark/mllib/util/LAUtils.scala | 6 +- python/pyspark/join.py | 5 +- python/pyspark/rdd.py | 10 ++-- python/pyspark/resultiterable.py | 33 +++++++++++ .../streaming/api/java/JavaPairDStream.scala | 42 +++++++------- .../dstream/PairDStreamFunctions.scala | 29 +++++----- .../streaming/dstream/StateDStream.scala | 13 +++-- .../apache/spark/streaming/JavaAPISuite.java | 58 ++++++++++++++++--- .../streaming/BasicOperationsSuite.scala | 4 +- 24 files changed, 252 insertions(+), 153 deletions(-) create mode 100644 python/pyspark/resultiterable.py diff --git a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala index 70c7474a936dc..70a99b33d753c 100644 --- a/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala +++ b/bagel/src/main/scala/org/apache/spark/bagel/Bagel.scala @@ -220,20 +220,23 @@ object Bagel extends Logging { */ private def comp[K: Manifest, V <: Vertex, M <: Message[K], C]( sc: SparkContext, - grouped: RDD[(K, (Seq[C], Seq[V]))], + grouped: RDD[(K, (Iterable[C], Iterable[V]))], compute: (V, Option[C]) => (V, Array[M]), storageLevel: StorageLevel ): (RDD[(K, (V, Array[M]))], Int, Int) = { var numMsgs = sc.accumulator(0) var numActiveVerts = sc.accumulator(0) - val processed = grouped.flatMapValues { - case (_, vs) if vs.size == 0 => None - case (c, vs) => + val processed = grouped.mapValues(x => (x._1.iterator, x._2.iterator)) + .flatMapValues { + case (_, vs) if !vs.hasNext => None + case (c, vs) => { val (newVert, newMsgs) = - compute(vs(0), c match { - case Seq(comb) => Some(comb) - case Seq() => None - }) + compute(vs.next, + c.hasNext match { + case true => Some(c.next) + case false => None + } + ) numMsgs += newMsgs.size if (newVert.active) { @@ -241,6 +244,7 @@ object Bagel extends Logging { } Some((newVert, newMsgs)) + } }.persist(storageLevel) // Force evaluation of processed RDD for accurate performance measurements diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index 9596dbaf75488..e6c5d85917678 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.api.java import java.util.{Comparator, List => JList} +import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ import scala.reflect.ClassTag @@ -250,14 +251,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JList[V]] = + def groupByKey(partitioner: Partitioner): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(partitioner))) /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairRDD[K, JList[V]] = + def groupByKey(numPartitions: Int): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey(numPartitions))) /** @@ -367,7 +368,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): JavaPairRDD[K, JList[V]] = + def groupByKey(): JavaPairRDD[K, JIterable[V]] = fromRDD(groupByResultToJava(rdd.groupByKey())) /** @@ -462,7 +463,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], partitioner: Partitioner) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, partitioner))) /** @@ -470,14 +471,14 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], - partitioner: Partitioner): JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + partitioner: Partitioner): JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, partitioner))) /** * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def cogroup[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other))) /** @@ -485,7 +486,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2))) /** @@ -493,7 +494,7 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * list of values for that key in `this` as well as `other`. */ def cogroup[W](other: JavaPairRDD[K, W], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.cogroup(other, numPartitions))) /** @@ -501,16 +502,16 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2], numPartitions: Int) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.cogroup(other1, other2, numPartitions))) /** Alias for cogroup. */ - def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JList[V], JList[W])] = + def groupWith[W](other: JavaPairRDD[K, W]): JavaPairRDD[K, (JIterable[V], JIterable[W])] = fromRDD(cogroupResultToJava(rdd.groupWith(other))) /** Alias for cogroup. */ def groupWith[W1, W2](other1: JavaPairRDD[K, W1], other2: JavaPairRDD[K, W2]) - : JavaPairRDD[K, (JList[V], JList[W1], JList[W2])] = + : JavaPairRDD[K, (JIterable[V], JIterable[W1], JIterable[W2])] = fromRDD(cogroupResult2ToJava(rdd.groupWith(other1, other2))) /** @@ -695,21 +696,22 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) object JavaPairRDD { private[spark] - def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Seq[T])]): RDD[(K, JList[T])] = { - rddToPairRDDFunctions(rdd).mapValues(seqAsJavaList) + def groupByResultToJava[K: ClassTag, T](rdd: RDD[(K, Iterable[T])]): RDD[(K, JIterable[T])] = { + rddToPairRDDFunctions(rdd).mapValues(asJavaIterable) } private[spark] def cogroupResultToJava[K: ClassTag, V, W]( - rdd: RDD[(K, (Seq[V], Seq[W]))]): RDD[(K, (JList[V], JList[W]))] = { - rddToPairRDDFunctions(rdd).mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2))) + rdd: RDD[(K, (Iterable[V], Iterable[W]))]): RDD[(K, (JIterable[V], JIterable[W]))] = { + rddToPairRDDFunctions(rdd).mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2))) } private[spark] def cogroupResult2ToJava[K: ClassTag, V, W1, W2]( - rdd: RDD[(K, (Seq[V], Seq[W1], Seq[W2]))]): RDD[(K, (JList[V], JList[W1], JList[W2]))] = { + rdd: RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))]) + : RDD[(K, (JIterable[V], JIterable[W1], JIterable[W2]))] = { rddToPairRDDFunctions(rdd) - .mapValues(x => (seqAsJavaList(x._1), seqAsJavaList(x._2), seqAsJavaList(x._3))) + .mapValues(x => (asJavaIterable(x._1), asJavaIterable(x._2), asJavaIterable(x._3))) } def fromRDD[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]): JavaPairRDD[K, V] = { diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 6e8ec8e0c7629..ae577b500ccb4 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -17,7 +17,7 @@ package org.apache.spark.api.java -import java.util.{Comparator, Iterator => JIterator, List => JList} +import java.util.{Comparator, List => JList, Iterator => JIterator} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ @@ -204,7 +204,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K]): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f)(fakeClassTag))) @@ -214,7 +214,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JList[T]] = { + def groupBy[K](f: JFunction[T, K], numPartitions: Int): JavaPairRDD[K, JIterable[T]] = { implicit val ctagK: ClassTag[K] = fakeClassTag implicit val ctagV: ClassTag[JList[T]] = fakeClassTag JavaPairRDD.fromRDD(groupByResultToJava(rdd.groupBy(f, numPartitions)(fakeClassTag[K]))) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index 14386ff5b9127..a92a84b5342d1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -261,7 +261,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Allows controlling the * partitioning of the resulting key-value pair RDD by passing a Partitioner. */ - def groupByKey(partitioner: Partitioner): RDD[(K, Seq[V])] = { + def groupByKey(partitioner: Partitioner): RDD[(K, Iterable[V])] = { // groupByKey shouldn't use map side combine because map side combine does not // reduce the amount of data shuffled and requires all map side data be inserted // into a hash table, leading to more objects in the old gen. @@ -270,14 +270,14 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def mergeCombiners(c1: ArrayBuffer[V], c2: ArrayBuffer[V]) = c1 ++ c2 val bufs = combineByKey[ArrayBuffer[V]]( createCombiner _, mergeValue _, mergeCombiners _, partitioner, mapSideCombine=false) - bufs.asInstanceOf[RDD[(K, Seq[V])]] + bufs.mapValues(_.toIterable) } /** * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with into `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): RDD[(K, Seq[V])] = { + def groupByKey(numPartitions: Int): RDD[(K, Iterable[V])] = { groupByKey(new HashPartitioner(numPartitions)) } @@ -298,7 +298,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def join[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => - for (v <- vs.iterator; w <- ws.iterator) yield (v, w) + for (v <- vs; w <- ws) yield (v, w) } } @@ -311,9 +311,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def leftOuterJoin[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (V, Option[W]))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (ws.isEmpty) { - vs.iterator.map(v => (v, None)) + vs.map(v => (v, None)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (v, Some(w)) + for (v <- vs; w <- ws) yield (v, Some(w)) } } } @@ -328,9 +328,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) : RDD[(K, (Option[V], W))] = { this.cogroup(other, partitioner).flatMapValues { case (vs, ws) => if (vs.isEmpty) { - ws.iterator.map(w => (None, w)) + ws.map(w => (None, w)) } else { - for (v <- vs.iterator; w <- ws.iterator) yield (Some(v), w) + for (v <- vs; w <- ws) yield (Some(v), w) } } } @@ -358,7 +358,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Group the values for each key in the RDD into a single sequence. Hash-partitions the * resulting RDD with the existing partitioner/parallelism level. */ - def groupByKey(): RDD[(K, Seq[V])] = { + def groupByKey(): RDD[(K, Iterable[V])] = { groupByKey(defaultPartitioner(self)) } @@ -453,7 +453,8 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) + : RDD[(K, (Iterable[V], Iterable[W]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } @@ -468,13 +469,15 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) cg.mapValues { case Seq(vs, w1s, w2s) => - (vs.asInstanceOf[Seq[V]], w1s.asInstanceOf[Seq[W1]], w2s.asInstanceOf[Seq[W2]]) + (vs.asInstanceOf[Seq[V]], + w1s.asInstanceOf[Seq[W1]], + w2s.asInstanceOf[Seq[W2]]) } } @@ -482,7 +485,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } @@ -491,7 +494,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } @@ -499,7 +502,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * For each key k in `this` or `other`, return a resulting RDD that contains a tuple with the * list of values for that key in `this` as well as `other`. */ - def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Seq[V], Seq[W]))] = { + def cogroup[W](other: RDD[(K, W)], numPartitions: Int): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, new HashPartitioner(numPartitions)) } @@ -508,18 +511,18 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * tuple with the list of values for that key in `this`, `other1` and `other2`. */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], numPartitions: Int) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, new HashPartitioner(numPartitions)) } /** Alias for cogroup. */ - def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Seq[V], Seq[W]))] = { + def groupWith[W](other: RDD[(K, W)]): RDD[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(self, other)) } /** Alias for cogroup. */ def groupWith[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)]) - : RDD[(K, (Seq[V], Seq[W1], Seq[W2]))] = { + : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { cogroup(other1, other2, defaultPartitioner(self, other1, other2)) } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index bf3c57ad41eb2..74fa2a4fcd401 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -438,20 +438,20 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Seq[T])] = + def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** * Return an RDD of grouped items. */ - def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Seq[T])] = { + def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Iterable[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 762405be2a8f9..ab2fdac553349 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -18,10 +18,12 @@ package org.apache.spark; import java.io.*; +import java.lang.StringBuilder; import java.util.*; import scala.Tuple2; +import com.google.common.collect.Iterables; import com.google.common.collect.Lists; import com.google.common.base.Optional; import com.google.common.base.Charsets; @@ -197,7 +199,7 @@ public void lookup() { new Tuple2("Oranges", "Citrus") )); Assert.assertEquals(2, categories.lookup("Oranges").size()); - Assert.assertEquals(2, categories.groupByKey().lookup("Oranges").get(0).size()); + Assert.assertEquals(2, Iterables.size(categories.groupByKey().lookup("Oranges").get(0))); } @Test @@ -209,15 +211,15 @@ public Boolean call(Integer x) { return x % 2 == 0; } }; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @SuppressWarnings("unchecked") @@ -232,9 +234,9 @@ public void cogroup() { new Tuple2("Oranges", 2), new Tuple2("Apples", 3) )); - JavaPairRDD, List>> cogrouped = categories.cogroup(prices); - Assert.assertEquals("[Fruit, Citrus]", cogrouped.lookup("Oranges").get(0)._1().toString()); - Assert.assertEquals("[2]", cogrouped.lookup("Oranges").get(0)._2().toString()); + JavaPairRDD, Iterable>> cogrouped = categories.cogroup(prices); + Assert.assertEquals("[Fruit, Citrus]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._1())); + Assert.assertEquals("[2]", Iterables.toString(cogrouped.lookup("Oranges").get(0)._2())); cogrouped.collect(); } diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index f3fb64d87a2fd..12dbebcb28644 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -72,7 +72,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { throw new Exception("Intentional task failure") } } - (k, v(0) * v(0)) + (k, v.head * v.head) }.collect() FailureSuiteState.synchronized { assert(FailureSuiteState.tasksRun === 4) @@ -137,5 +137,3 @@ class FailureSuite extends FunSuite with LocalSparkContext { // TODO: Need to add tests with shuffle fetch failures. } - - diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 627e9b5cd9060..867b28cc0d971 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -85,7 +85,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { (f: String => Unit) => { bl.value.map(f(_)); f("\u0001") }, - (i: Tuple2[String, Seq[String]], f: String => Unit) => { + (i: Tuple2[String, Iterable[String]], f: String => Unit) => { for (e <- i._2) { f(e + "_") } diff --git a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala index f9e994b13dfbc..8f3e6bd21b752 100644 --- a/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/PairRDDFunctionsSuite.scala @@ -225,11 +225,12 @@ class PairRDDFunctionsSuite extends FunSuite with SharedSparkContext { val rdd2 = sc.parallelize(Array((1, 'x'), (2, 'y'), (2, 'z'), (4, 'w'))) val joined = rdd1.groupWith(rdd2).collect() assert(joined.size === 4) - assert(joined.toSet === Set( - (1, (ArrayBuffer(1, 2), ArrayBuffer('x'))), - (2, (ArrayBuffer(1), ArrayBuffer('y', 'z'))), - (3, (ArrayBuffer(1), ArrayBuffer())), - (4, (ArrayBuffer(), ArrayBuffer('w'))) + val joinedSet = joined.map(x => (x._1, (x._2._1.toList, x._2._2.toList))).toSet + assert(joinedSet === Set( + (1, (List(1, 2), List('x'))), + (2, (List(1), List('y', 'z'))), + (3, (List(1), List())), + (4, (List(), List('w'))) )) } @@ -447,4 +448,3 @@ class ConfigTestFormat() extends FakeFormat() with Configurable { super.getRecordWriter(p1) } } - diff --git a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala index fce1184d46364..cdebefb67510c 100644 --- a/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/collection/ExternalAppendOnlyMapSuite.scala @@ -174,9 +174,9 @@ class ExternalAppendOnlyMapSuite extends FunSuite with LocalSparkContext { assert(result1.toSet == Set[(Int, Int)]((0, 5), (1, 5))) // groupByKey - val result2 = rdd.groupByKey().collect() + val result2 = rdd.groupByKey().collect().map(x => (x._1, x._2.toList)).toSet assert(result2.toSet == Set[(Int, Seq[Int])] - ((0, ArrayBuffer[Int](1, 1, 1, 1, 1)), (1, ArrayBuffer[Int](1, 1, 1, 1, 1)))) + ((0, List[Int](1, 1, 1, 1, 1)), (1, List[Int](1, 1, 1, 1, 1)))) } test("simple cogroup") { diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index eb70fb547564c..8513ba07e7705 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -17,7 +17,10 @@ package org.apache.spark.examples; + import scala.Tuple2; + +import com.google.common.collect.Iterables; import org.apache.spark.api.java.JavaPairRDD; import org.apache.spark.api.java.JavaRDD; import org.apache.spark.api.java.JavaSparkContext; @@ -26,8 +29,9 @@ import org.apache.spark.api.java.function.PairFlatMapFunction; import org.apache.spark.api.java.function.PairFunction; -import java.util.List; import java.util.ArrayList; +import java.util.List; +import java.util.Iterator; import java.util.regex.Pattern; /** @@ -66,7 +70,7 @@ public static void main(String[] args) throws Exception { JavaRDD lines = ctx.textFile(args[1], 1); // Loads all URLs from input file and initialize their neighbors. - JavaPairRDD> links = lines.mapToPair(new PairFunction() { + JavaPairRDD> links = lines.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { String[] parts = SPACES.split(s); @@ -75,9 +79,9 @@ public Tuple2 call(String s) { }).distinct().groupByKey().cache(); // Loads all URLs with other URL(s) link to from input file and initialize ranks of them to one. - JavaPairRDD ranks = links.mapValues(new Function, Double>() { + JavaPairRDD ranks = links.mapValues(new Function, Double>() { @Override - public Double call(List rs) { + public Double call(Iterable rs) { return 1.0; } }); @@ -86,12 +90,13 @@ public Double call(List rs) { for (int current = 0; current < Integer.parseInt(args[2]); current++) { // Calculates URL contributions to the rank of other URLs. JavaPairRDD contribs = links.join(ranks).values() - .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { + .flatMapToPair(new PairFlatMapFunction, Double>, String, Double>() { @Override - public Iterable> call(Tuple2, Double> s) { + public Iterable> call(Tuple2, Double> s) { + int urlCount = Iterables.size(s._1); List> results = new ArrayList>(); - for (String n : s._1()) { - results.add(new Tuple2(n, s._2() / s._1().size())); + for (String n : s._1) { + results.add(new Tuple2(n, s._2() / urlCount)); } return results; } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 27afa6b642758..7aac6a13597e6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -115,12 +115,16 @@ object WikipediaPageRankStandalone { var ranks = links.mapValues { edges => defaultRank } for (i <- 1 to numIterations) { val contribs = links.groupWith(ranks).flatMap { - case (id, (linksWrapper, rankWrapper)) => - if (linksWrapper.length > 0) { - if (rankWrapper.length > 0) { - linksWrapper(0).map(dest => (dest, rankWrapper(0) / linksWrapper(0).size)) + case (id, (linksWrapperIterable, rankWrapperIterable)) => + val linksWrapper = linksWrapperIterable.iterator + val rankWrapper = rankWrapperIterable.iterator + if (linksWrapper.hasNext) { + val linksWrapperHead = linksWrapper.next + if (rankWrapper.hasNext) { + val rankWrapperHead = rankWrapper.next + linksWrapperHead.map(dest => (dest, rankWrapperHead / linksWrapperHead.size)) } else { - linksWrapper(0).map(dest => (dest, defaultRank / linksWrapper(0).size)) + linksWrapperHead.map(dest => (dest, defaultRank / linksWrapperHead.size)) } } else { Array[(String, Double)]() diff --git a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java index f67251217ed4a..7eb8b45fc3cf0 100644 --- a/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java +++ b/extras/java8-tests/src/test/java/org/apache/spark/Java8APISuite.java @@ -23,6 +23,7 @@ import scala.Tuple2; +import com.google.common.collections.Iterables; import com.google.common.base.Optional; import com.google.common.io.Files; import org.apache.hadoop.io.IntWritable; @@ -85,15 +86,15 @@ public void foreach() { public void groupBy() { JavaRDD rdd = sc.parallelize(Arrays.asList(1, 1, 2, 3, 5, 8, 13)); Function isOdd = x -> x % 2 == 0; - JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); + JavaPairRDD> oddsAndEvens = rdd.groupBy(isOdd); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds oddsAndEvens = rdd.groupBy(isOdd, 1); Assert.assertEquals(2, oddsAndEvens.count()); - Assert.assertEquals(2, oddsAndEvens.lookup(true).get(0).size()); // Evens - Assert.assertEquals(5, oddsAndEvens.lookup(false).get(0).size()); // Odds + Assert.assertEquals(2, Iterables.size(oddsAndEvens.lookup(true).get(0))); // Evens + Assert.assertEquals(5, Iterables.size(oddsAndEvens.lookup(false).get(0))); // Odds } @Test diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala index 3e7cc648d1d37..0d97b7d92f155 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala @@ -69,11 +69,11 @@ class SVD { /** * Compute SVD using the current set parameters - * Returns (U, S, V) such that A = USV^T + * Returns (U, S, V) such that A = USV^T * U is a row-by-row dense matrix * S is a simple double array of singular values * V is a 2d array matrix - * See [[denseSVD]] for more documentation + * See [[denseSVD]] for more documentation */ def compute(matrix: RDD[Array[Double]]): (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { @@ -393,5 +393,3 @@ object SVD { System.exit(0) } } - - diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 0cc9f48769f83..3124fac326d22 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -421,12 +421,12 @@ class ALS private ( * Compute the new feature vectors for a block of the users matrix given the list of factors * it received from each product and its InLinkBlock. */ - private def updateBlock(messages: Seq[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, + private def updateBlock(messages: Iterable[(Int, Array[Array[Double]])], inLinkBlock: InLinkBlock, rank: Int, lambda: Double, alpha: Double, YtY: Option[Broadcast[DoubleMatrix]]) : Array[Array[Double]] = { // Sort the incoming block factor messages by block ID and make them an array - val blockFactors = messages.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] + val blockFactors = messages.toSeq.sortBy(_._1).map(_._2).toArray // Array[Array[Double]] val numBlocks = blockFactors.length val numUsers = inLinkBlock.elementIds.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala index afe081295bfae..87aac347579c7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala @@ -38,8 +38,10 @@ object LAUtils { case (i, cols) => val rowArray = Array.ofDim[Double](n) var j = 0 - while (j < cols.size) { - rowArray(cols(j)._1) = cols(j)._2 + val colsItr = cols.iterator + while (colsItr.hasNext) { + val element = colsItr.next + rowArray(element._1) = element._2 j += 1 } MatrixRow(i, rowArray) diff --git a/python/pyspark/join.py b/python/pyspark/join.py index 5f4294fb1b777..6f94d26ef86a9 100644 --- a/python/pyspark/join.py +++ b/python/pyspark/join.py @@ -31,11 +31,12 @@ OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE. """ +from pyspark.resultiterable import ResultIterable def _do_python_join(rdd, other, numPartitions, dispatch): vs = rdd.map(lambda (k, v): (k, (1, v))) ws = other.map(lambda (k, v): (k, (2, v))) - return vs.union(ws).groupByKey(numPartitions).flatMapValues(dispatch) + return vs.union(ws).groupByKey(numPartitions).flatMapValues(lambda x : dispatch(x.__iter__())) def python_join(rdd, other, numPartitions): @@ -88,5 +89,5 @@ def dispatch(seq): vbuf.append(v) elif n == 2: wbuf.append(v) - return (vbuf, wbuf) + return (ResultIterable(vbuf), ResultIterable(wbuf)) return vs.union(ws).groupByKey(numPartitions).mapValues(dispatch) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index fb27863e07f55..91fc7e637e2c6 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -38,6 +38,7 @@ from pyspark.statcounter import StatCounter from pyspark.rddsampler import RDDSampler from pyspark.storagelevel import StorageLevel +from pyspark.resultiterable import ResultIterable from py4j.java_collections import ListConverter, MapConverter @@ -1118,7 +1119,7 @@ def groupByKey(self, numPartitions=None): Hash-partitions the resulting RDD with into numPartitions partitions. >>> x = sc.parallelize([("a", 1), ("b", 1), ("a", 1)]) - >>> sorted(x.groupByKey().collect()) + >>> map((lambda (x,y): (x, list(y))), sorted(x.groupByKey().collect())) [('a', [1, 1]), ('b', [1])] """ @@ -1133,7 +1134,7 @@ def mergeCombiners(a, b): return a + b return self.combineByKey(createCombiner, mergeValue, mergeCombiners, - numPartitions) + numPartitions).mapValues(lambda x: ResultIterable(x)) # TODO: add tests def flatMapValues(self, f): @@ -1180,7 +1181,7 @@ def cogroup(self, other, numPartitions=None): >>> x = sc.parallelize([("a", 1), ("b", 4)]) >>> y = sc.parallelize([("a", 2)]) - >>> sorted(x.cogroup(y).collect()) + >>> map((lambda (x,y): (x, (list(y[0]), list(y[1])))), sorted(list(x.cogroup(y).collect()))) [('a', ([1], [2])), ('b', ([4], []))] """ return python_cogroup(self, other, numPartitions) @@ -1217,7 +1218,7 @@ def keyBy(self, f): >>> x = sc.parallelize(range(0,3)).keyBy(lambda x: x*x) >>> y = sc.parallelize(zip(range(0,5), range(0,5))) - >>> sorted(x.cogroup(y).collect()) + >>> map((lambda (x,y): (x, (list(y[0]), (list(y[1]))))), sorted(x.cogroup(y).collect())) [(0, ([0], [0])), (1, ([1], [1])), (2, ([], [2])), (3, ([], [3])), (4, ([2], [4]))] """ return self.map(lambda x: (f(x), x)) @@ -1317,7 +1318,6 @@ def getStorageLevel(self): # keys in the pairs. This could be an expensive operation, since those # hashes aren't retained. - class PipelinedRDD(RDD): """ Pipelined maps: diff --git a/python/pyspark/resultiterable.py b/python/pyspark/resultiterable.py new file mode 100644 index 0000000000000..7f418f8d2e29a --- /dev/null +++ b/python/pyspark/resultiterable.py @@ -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. +# + +__all__ = ["ResultIterable"] + +import collections + +class ResultIterable(collections.Iterable): + """ + A special result iterable. This is used because the standard iterator can not be pickled + """ + def __init__(self, data): + self.data = data + self.index = 0 + self.maxindex = len(data) + def __iter__(self): + return iter(self.data) + def __len__(self): + return len(self.data) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index ac451d1913aaa..2ac943d7bf781 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.api.java -import java.lang.{Long => JLong} +import java.lang.{Long => JLong, Iterable => JIterable} import java.util.{List => JList} import scala.collection.JavaConversions._ @@ -115,15 +115,15 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): JavaPairDStream[K, JList[V]] = - dstream.groupByKey().mapValues(seqAsJavaList _) + def groupByKey(): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey().mapValues(asJavaIterable _) /** * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): JavaPairDStream[K, JList[V]] = - dstream.groupByKey(numPartitions).mapValues(seqAsJavaList _) + def groupByKey(numPartitions: Int): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey(numPartitions).mapValues(asJavaIterable _) /** * Return a new DStream by applying `groupByKey` on each RDD of `this` DStream. @@ -131,8 +131,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * single sequence to generate the RDDs of the new DStream. org.apache.spark.Partitioner * is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JList[V]] = - dstream.groupByKey(partitioner).mapValues(seqAsJavaList _) + def groupByKey(partitioner: Partitioner): JavaPairDStream[K, JIterable[V]] = + dstream.groupByKey(partitioner).mapValues(asJavaIterable _) /** * Return a new DStream by applying `reduceByKey` to each RDD. The values for each key are @@ -196,8 +196,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JList[V]] = { - dstream.groupByKeyAndWindow(windowDuration).mapValues(seqAsJavaList _) + def groupByKeyAndWindow(windowDuration: Duration): JavaPairDStream[K, JIterable[V]] = { + dstream.groupByKeyAndWindow(windowDuration).mapValues(asJavaIterable _) } /** @@ -211,8 +211,8 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * DStream's batching interval */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) - : JavaPairDStream[K, JList[V]] = { - dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(seqAsJavaList _) + : JavaPairDStream[K, JIterable[V]] = { + dstream.groupByKeyAndWindow(windowDuration, slideDuration).mapValues(asJavaIterable _) } /** @@ -227,9 +227,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * @param numPartitions Number of partitions of each RDD in the new DStream. */ def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration, numPartitions: Int) - :JavaPairDStream[K, JList[V]] = { + :JavaPairDStream[K, JIterable[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, numPartitions) - .mapValues(seqAsJavaList _) + .mapValues(asJavaIterable _) } /** @@ -247,9 +247,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ):JavaPairDStream[K, JList[V]] = { + ):JavaPairDStream[K, JIterable[V]] = { dstream.groupByKeyAndWindow(windowDuration, slideDuration, partitioner) - .mapValues(seqAsJavaList _) + .mapValues(asJavaIterable _) } /** @@ -518,9 +518,9 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JList[V], JList[W])] = { + def cogroup[W](other: JavaPairDStream[K, W]): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag - dstream.cogroup(other.dstream).mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + dstream.cogroup(other.dstream).mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** @@ -530,10 +530,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], numPartitions: Int - ): JavaPairDStream[K, (JList[V], JList[W])] = { + ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, numPartitions) - .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** @@ -543,10 +543,10 @@ class JavaPairDStream[K, V](val dstream: DStream[(K, V)])( def cogroup[W]( other: JavaPairDStream[K, W], partitioner: Partitioner - ): JavaPairDStream[K, (JList[V], JList[W])] = { + ): JavaPairDStream[K, (JIterable[V], JIterable[W])] = { implicit val cm: ClassTag[W] = fakeClassTag dstream.cogroup(other.dstream, partitioner) - .mapValues(t => (seqAsJavaList(t._1), seqAsJavaList((t._2)))) + .mapValues(t => (asJavaIterable(t._1), asJavaIterable((t._2)))) } /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 2473496949360..354bc132dcdc0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -51,7 +51,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with Spark's default number of partitions. */ - def groupByKey(): DStream[(K, Seq[V])] = { + def groupByKey(): DStream[(K, Iterable[V])] = { groupByKey(defaultPartitioner()) } @@ -59,7 +59,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` to each RDD. Hash partitioning is used to * generate the RDDs with `numPartitions` partitions. */ - def groupByKey(numPartitions: Int): DStream[(K, Seq[V])] = { + def groupByKey(numPartitions: Int): DStream[(K, Iterable[V])] = { groupByKey(defaultPartitioner(numPartitions)) } @@ -67,12 +67,12 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Return a new DStream by applying `groupByKey` on each RDD. The supplied * org.apache.spark.Partitioner is used to control the partitioning of each RDD. */ - def groupByKey(partitioner: Partitioner): DStream[(K, Seq[V])] = { + def groupByKey(partitioner: Partitioner): DStream[(K, Iterable[V])] = { val createCombiner = (v: V) => ArrayBuffer[V](v) val mergeValue = (c: ArrayBuffer[V], v: V) => (c += v) val mergeCombiner = (c1: ArrayBuffer[V], c2: ArrayBuffer[V]) => (c1 ++ c2) combineByKey(createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]] + .asInstanceOf[DStream[(K, Iterable[V])]] } /** @@ -126,7 +126,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * @param windowDuration width of the window; must be a multiple of this DStream's * batching interval */ - def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Seq[V])] = { + def groupByKeyAndWindow(windowDuration: Duration): DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, self.slideDuration, defaultPartitioner()) } @@ -140,7 +140,8 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * the new DStream will generate RDDs); must be a multiple of this * DStream's batching interval */ - def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration): DStream[(K, Seq[V])] = + def groupByKeyAndWindow(windowDuration: Duration, slideDuration: Duration) + : DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner()) } @@ -161,7 +162,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, numPartitions: Int - ): DStream[(K, Seq[V])] = { + ): DStream[(K, Iterable[V])] = { groupByKeyAndWindow(windowDuration, slideDuration, defaultPartitioner(numPartitions)) } @@ -180,14 +181,14 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) windowDuration: Duration, slideDuration: Duration, partitioner: Partitioner - ): DStream[(K, Seq[V])] = { - val createCombiner = (v: Seq[V]) => new ArrayBuffer[V] ++= v - val mergeValue = (buf: ArrayBuffer[V], v: Seq[V]) => buf ++= v + ): DStream[(K, Iterable[V])] = { + val createCombiner = (v: Iterable[V]) => new ArrayBuffer[V] ++= v + val mergeValue = (buf: ArrayBuffer[V], v: Iterable[V]) => buf ++= v val mergeCombiner = (buf1: ArrayBuffer[V], buf2: ArrayBuffer[V]) => buf1 ++= buf2 self.groupByKey(partitioner) .window(windowDuration, slideDuration) .combineByKey[ArrayBuffer[V]](createCombiner, mergeValue, mergeCombiner, partitioner) - .asInstanceOf[DStream[(K, Seq[V])]] + .asInstanceOf[DStream[(K, Iterable[V])]] } /** @@ -438,7 +439,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with Spark's default number * of partitions. */ - def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Seq[V], Seq[W]))] = { + def cogroup[W: ClassTag](other: DStream[(K, W)]): DStream[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner()) } @@ -447,7 +448,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) * Hash partitioning is used to generate the RDDs with `numPartitions` partitions. */ def cogroup[W: ClassTag](other: DStream[(K, W)], numPartitions: Int) - : DStream[(K, (Seq[V], Seq[W]))] = { + : DStream[(K, (Iterable[V], Iterable[W]))] = { cogroup(other, defaultPartitioner(numPartitions)) } @@ -458,7 +459,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) def cogroup[W: ClassTag]( other: DStream[(K, W)], partitioner: Partitioner - ): DStream[(K, (Seq[V], Seq[W]))] = { + ): DStream[(K, (Iterable[V], Iterable[W]))] = { self.transformWith( other, (rdd1: RDD[(K, V)], rdd2: RDD[(K, W)]) => rdd1.cogroup(rdd2, partitioner) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala index 5f7d3ba26c656..7e22268767de7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/StateDStream.scala @@ -56,9 +56,14 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the cogrouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, (Seq[V], Seq[S]))]) => { + val finalFunc = (iterator: Iterator[(K, (Iterable[V], Iterable[S]))]) => { val i = iterator.map(t => { - (t._1, t._2._1, t._2._2.headOption) + val itr = t._2._2.iterator + val headOption = itr.hasNext match { + case true => Some(itr.next()) + case false => None + } + (t._1, t._2._1.toSeq, headOption) }) updateFuncLocal(i) } @@ -90,8 +95,8 @@ class StateDStream[K: ClassTag, V: ClassTag, S: ClassTag]( // first map the grouped tuple to tuples of required type, // and then apply the update function val updateFuncLocal = updateFunc - val finalFunc = (iterator: Iterator[(K, Seq[V])]) => { - updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2, None))) + val finalFunc = (iterator: Iterator[(K, Iterable[V])]) => { + updateFuncLocal(iterator.map(tuple => (tuple._1, tuple._2.toSeq, None))) } val groupedRDD = parentRDD.groupByKey(partitioner) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index e93bf18b6d0b9..13fa64894b773 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -23,6 +23,7 @@ import org.junit.Test; import java.io.*; import java.util.*; +import java.lang.Iterable; import com.google.common.base.Optional; import com.google.common.collect.Lists; @@ -45,6 +46,18 @@ // see http://stackoverflow.com/questions/758570/. public class JavaAPISuite extends LocalJavaStreamingContext implements Serializable { + public void equalIterator(Iterator a, Iterator b) { + while (a.hasNext() && b.hasNext()) { + Assert.assertEquals(a.next(), b.next()); + } + Assert.assertEquals(a.hasNext(), b.hasNext()); + } + + public void equalIterable(Iterable a, Iterable b) { + equalIterator(a.iterator(), b.iterator()); + } + + @SuppressWarnings("unchecked") @Test public void testCount() { @@ -1016,11 +1029,24 @@ public void testPairGroupByKey() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> grouped = pairStream.groupByKey(); + JavaPairDStream> grouped = pairStream.groupByKey(); JavaTestUtils.attachTestOutputStream(grouped); - List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); + List>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected.size(), result.size()); + Iterator>>> resultItr = result.iterator(); + Iterator>>> expectedItr = expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator>> resultElements = resultItr.next().iterator(); + Iterator>> expectedElements = expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2> resultElement = resultElements.next(); + Tuple2> expectedElement = expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterable(expectedElement._2(), resultElement._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); + } } @SuppressWarnings("unchecked") @@ -1128,7 +1154,7 @@ public void testGroupByKeyAndWindow() { JavaDStream> stream = JavaTestUtils.attachTestInputStream(ssc, inputData, 1); JavaPairDStream pairStream = JavaPairDStream.fromJavaDStream(stream); - JavaPairDStream> groupWindowed = + JavaPairDStream> groupWindowed = pairStream.groupByKeyAndWindow(new Duration(2000), new Duration(1000)); JavaTestUtils.attachTestOutputStream(groupWindowed); List>>> result = JavaTestUtils.runStreams(ssc, 3, 3); @@ -1471,11 +1497,25 @@ public void testCoGroup() { ssc, stringStringKVStream2, 1); JavaPairDStream pairStream2 = JavaPairDStream.fromJavaDStream(stream2); - JavaPairDStream, List>> grouped = pairStream1.cogroup(pairStream2); + JavaPairDStream, Iterable>> grouped = pairStream1.cogroup(pairStream2); JavaTestUtils.attachTestOutputStream(grouped); - List, List>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); - - Assert.assertEquals(expected, result); + List, Iterable>>>> result = JavaTestUtils.runStreams(ssc, 2, 2); + + Assert.assertEquals(expected.size(), result.size()); + Iterator, Iterable>>>> resultItr = result.iterator(); + Iterator, List>>>> expectedItr = expected.iterator(); + while (resultItr.hasNext() && expectedItr.hasNext()) { + Iterator, Iterable>>> resultElements = resultItr.next().iterator(); + Iterator, List>>> expectedElements = expectedItr.next().iterator(); + while (resultElements.hasNext() && expectedElements.hasNext()) { + Tuple2, Iterable>> resultElement = resultElements.next(); + Tuple2, List>> expectedElement = expectedElements.next(); + Assert.assertEquals(expectedElement._1(), resultElement._1()); + equalIterable(expectedElement._2()._1(), resultElement._2()._1()); + equalIterable(expectedElement._2()._2(), resultElement._2()._2()); + } + Assert.assertEquals(resultElements.hasNext(), expectedElements.hasNext()); + } } @SuppressWarnings("unchecked") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index bb73dbf29b649..8aec27e39478a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -117,7 +117,7 @@ class BasicOperationsSuite extends TestSuiteBase { test("groupByKey") { testOperation( Seq( Seq("a", "a", "b"), Seq("", ""), Seq() ), - (s: DStream[String]) => s.map(x => (x, 1)).groupByKey(), + (s: DStream[String]) => s.map(x => (x, 1)).groupByKey().mapValues(_.toSeq), Seq( Seq(("a", Seq(1, 1)), ("b", Seq(1))), Seq(("", Seq(1, 1))), Seq() ), true ) @@ -251,7 +251,7 @@ class BasicOperationsSuite extends TestSuiteBase { Seq( ) ) val operation = (s1: DStream[String], s2: DStream[String]) => { - s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x"))) + s1.map(x => (x,1)).cogroup(s2.map(x => (x, "x"))).mapValues(x => (x._1.toSeq, x._2.toSeq)) } testOperation(inputData1, inputData2, operation, outputData, true) } From b9e0c937dfa1ca93b63d0b39d5f156b16c2fdc0a Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 8 Apr 2014 20:37:01 -0700 Subject: [PATCH 239/397] [SPARK-1434] [MLLIB] change labelParser from anonymous function to trait This is a patch to address @mateiz 's comment in https://github.com/apache/spark/pull/245 MLUtils#loadLibSVMData uses an anonymous function for the label parser. Java users won't like it. So I make a trait for LabelParser and provide two implementations: binary and multiclass. Author: Xiangrui Meng Closes #345 from mengxr/label-parser and squashes the following commits: ac44409 [Xiangrui Meng] use singleton objects for label parsers 3b1a7c6 [Xiangrui Meng] add tests for label parsers c2e571c [Xiangrui Meng] rename LabelParser.apply to LabelParser.parse use extends for singleton 11c94e0 [Xiangrui Meng] add return types 7f8eb36 [Xiangrui Meng] change labelParser from annoymous function to trait --- .../spark/mllib/util/LabelParsers.scala | 49 +++++++++++++++++++ .../org/apache/spark/mllib/util/MLUtils.scala | 28 ++--------- .../spark/mllib/util/LabelParsersSuite.scala | 41 ++++++++++++++++ .../spark/mllib/util/MLUtilsSuite.scala | 4 +- 4 files changed, 97 insertions(+), 25 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala new file mode 100644 index 0000000000000..f7966d3ebb613 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LabelParsers.scala @@ -0,0 +1,49 @@ +/* + * 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.mllib.util + +/** Trait for label parsers. */ +trait LabelParser extends Serializable { + /** Parses a string label into a double label. */ + def parse(labelString: String): Double +} + +/** + * Label parser for binary labels, which outputs 1.0 (positive) if the value is greater than 0.5, + * or 0.0 (negative) otherwise. So it works with +1/-1 labeling and +1/0 labeling. + */ +object BinaryLabelParser extends LabelParser { + /** Gets the default instance of BinaryLabelParser. */ + def getInstance(): LabelParser = this + + /** + * Parses the input label into positive (1.0) if the value is greater than 0.5, + * or negative (0.0) otherwise. + */ + override def parse(labelString: String): Double = if (labelString.toDouble > 0.5) 1.0 else 0.0 +} + +/** + * Label parser for multiclass labels, which converts the input label to double. + */ +object MulticlassLabelParser extends LabelParser { + /** Gets the default instance of MulticlassLabelParser. */ + def getInstance(): LabelParser = this + + override def parse(labelString: String): Double = labelString.toDouble +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index cb85e433bfc73..83d1bd3fd57fe 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -38,17 +38,6 @@ object MLUtils { eps } - /** - * Multiclass label parser, which parses a string into double. - */ - val multiclassLabelParser: String => Double = _.toDouble - - /** - * Binary label parser, which outputs 1.0 (positive) if the value is greater than 0.5, - * or 0.0 (negative) otherwise. - */ - val binaryLabelParser: String => Double = label => if (label.toDouble > 0.5) 1.0 else 0.0 - /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint]. * The LIBSVM format is a text-based format used by LIBSVM and LIBLINEAR. @@ -69,7 +58,7 @@ object MLUtils { def loadLibSVMData( sc: SparkContext, path: String, - labelParser: String => Double, + labelParser: LabelParser, numFeatures: Int, minSplits: Int): RDD[LabeledPoint] = { val parsed = sc.textFile(path, minSplits) @@ -89,7 +78,7 @@ object MLUtils { }.reduce(math.max) } parsed.map { items => - val label = labelParser(items.head) + val label = labelParser.parse(items.head) val (indices, values) = items.tail.map { item => val indexAndValue = item.split(':') val index = indexAndValue(0).toInt - 1 @@ -107,14 +96,7 @@ object MLUtils { * with number of features determined automatically and the default number of partitions. */ def loadLibSVMData(sc: SparkContext, path: String): RDD[LabeledPoint] = - loadLibSVMData(sc, path, binaryLabelParser, -1, sc.defaultMinSplits) - - /** - * Loads binary labeled data in the LIBSVM format into an RDD[LabeledPoint], - * with number of features specified explicitly and the default number of partitions. - */ - def loadLibSVMData(sc: SparkContext, path: String, numFeatures: Int): RDD[LabeledPoint] = - loadLibSVMData(sc, path, binaryLabelParser, numFeatures, sc.defaultMinSplits) + loadLibSVMData(sc, path, BinaryLabelParser, -1, sc.defaultMinSplits) /** * Loads labeled data in the LIBSVM format into an RDD[LabeledPoint], @@ -124,7 +106,7 @@ object MLUtils { def loadLibSVMData( sc: SparkContext, path: String, - labelParser: String => Double): RDD[LabeledPoint] = + labelParser: LabelParser): RDD[LabeledPoint] = loadLibSVMData(sc, path, labelParser, -1, sc.defaultMinSplits) /** @@ -135,7 +117,7 @@ object MLUtils { def loadLibSVMData( sc: SparkContext, path: String, - labelParser: String => Double, + labelParser: LabelParser, numFeatures: Int): RDD[LabeledPoint] = loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinSplits) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala new file mode 100644 index 0000000000000..ac85677f2f014 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/LabelParsersSuite.scala @@ -0,0 +1,41 @@ +/* + * 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.mllib.util + +import org.scalatest.FunSuite + +class LabelParsersSuite extends FunSuite { + test("binary label parser") { + for (parser <- Seq(BinaryLabelParser, BinaryLabelParser.getInstance())) { + assert(parser.parse("+1") === 1.0) + assert(parser.parse("1") === 1.0) + assert(parser.parse("0") === 0.0) + assert(parser.parse("-1") === 0.0) + } + } + + test("multiclass label parser") { + for (parser <- Seq(MulticlassLabelParser, MulticlassLabelParser.getInstance())) { + assert(parser.parse("0") == 0.0) + assert(parser.parse("+1") === 1.0) + assert(parser.parse("1") === 1.0) + assert(parser.parse("2") === 2.0) + assert(parser.parse("3") === 3.0) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index 27d41c7869aa0..e451c350b8d88 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -80,7 +80,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { Files.write(lines, file, Charsets.US_ASCII) val path = tempDir.toURI.toString - val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, 6).collect() + val pointsWithNumFeatures = MLUtils.loadLibSVMData(sc, path, BinaryLabelParser, 6).collect() val pointsWithoutNumFeatures = MLUtils.loadLibSVMData(sc, path).collect() for (points <- Seq(pointsWithNumFeatures, pointsWithoutNumFeatures)) { @@ -93,7 +93,7 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { assert(points(2).features === Vectors.sparse(6, Seq((1, 4.0), (3, 5.0), (5, 6.0)))) } - val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MLUtils.multiclassLabelParser).collect() + val multiclassPoints = MLUtils.loadLibSVMData(sc, path, MulticlassLabelParser).collect() assert(multiclassPoints.length === 3) assert(multiclassPoints(0).label === 1.0) assert(multiclassPoints(1).label === -1.0) From fa0524fd02eedd0bbf1edc750dc3997a86ea25f5 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Tue, 8 Apr 2014 22:29:21 -0700 Subject: [PATCH 240/397] Spark-939: allow user jars to take precedence over spark jars I still need to do a small bit of re-factoring [mostly the one Java file I'll switch it back to a Scala file and use it in both the close loaders], but comments on other things I should do would be great. Author: Holden Karau Closes #217 from holdenk/spark-939-allow-user-jars-to-take-precedence-over-spark-jars and squashes the following commits: cf0cac9 [Holden Karau] Fix the executorclassloader 1955232 [Holden Karau] Fix long line in TestUtils 8f89965 [Holden Karau] Fix tests for new class name 7546549 [Holden Karau] CR feedback, merge some of the testutils methods down, rename the classloader 644719f [Holden Karau] User the class generator for the repl class loader tests too f0b7114 [Holden Karau] Fix the core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala tests 204b199 [Holden Karau] Fix the generated classes 9f68f10 [Holden Karau] Start rewriting the ExecutorURLClassLoaderSuite to not use the hard coded classes 858aba2 [Holden Karau] Remove a bunch of test junk 261aaee [Holden Karau] simplify executorurlclassloader a bit 7a7bf5f [Holden Karau] CR feedback d4ae848 [Holden Karau] rewrite component into scala aa95083 [Holden Karau] CR feedback 7752594 [Holden Karau] re-add https comment a0ef85a [Holden Karau] Fix style issues 125ea7f [Holden Karau] Easier to just remove those files, we don't need them bb8d179 [Holden Karau] Fix issues with the repl class loader 241b03d [Holden Karau] fix my rat excludes a343350 [Holden Karau] Update rat-excludes and remove a useless file d90d217 [Holden Karau] Fix fall back with custom class loader and add a test for it 4919bf9 [Holden Karau] Fix parent calling class loader issue 8a67302 [Holden Karau] Test are good 9e2d236 [Holden Karau] It works comrade 691ee00 [Holden Karau] It works ish dc4fe44 [Holden Karau] Does not depend on being in my home directory 47046ff [Holden Karau] Remove bad import' 22d83cb [Holden Karau] Add a test suite for the executor url class loader suite 7ef4628 [Holden Karau] Clean up 792d961 [Holden Karau] Almost works 16aecd1 [Holden Karau] Doesn't quite work 8d2241e [Holden Karau] Adda FakeClass for testing ClassLoader precedence options 648b559 [Holden Karau] Both class loaders compile. Now for testing e1d9f71 [Holden Karau] One loader workers. --- .rat-excludes | 2 +- .../scala/org/apache/spark/TestUtils.scala | 20 +++-- .../org/apache/spark/executor/Executor.scala | 17 +++-- .../executor/ExecutorURLClassLoader.scala | 45 ++++++++++- .../apache/spark/util/ParentClassLoader.scala | 32 ++++++++ .../ExecutorURLClassLoaderSuite.scala | 67 ++++++++++++++++ docs/configuration.md | 9 +++ project/SparkBuild.scala | 1 + .../spark/repl/ExecutorClassLoader.scala | 39 +++++++--- .../spark/repl/ExecutorClassLoaderSuite.scala | 76 +++++++++++++++++++ 10 files changed, 287 insertions(+), 21 deletions(-) rename core/src/{test => main}/scala/org/apache/spark/TestUtils.scala (84%) create mode 100644 core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala create mode 100644 core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala create mode 100644 repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala diff --git a/.rat-excludes b/.rat-excludes index 85bfad60fcadc..a2b5665a0be26 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -39,4 +39,4 @@ work .*\.q golden test.out/* -.*iml +.*iml \ No newline at end of file diff --git a/core/src/test/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala similarity index 84% rename from core/src/test/scala/org/apache/spark/TestUtils.scala rename to core/src/main/scala/org/apache/spark/TestUtils.scala index 1611d09652d40..4597595a838e3 100644 --- a/core/src/test/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -26,7 +26,14 @@ import scala.collection.JavaConversions._ import javax.tools.{JavaFileObject, SimpleJavaFileObject, ToolProvider} import com.google.common.io.Files -object TestUtils { +/** + * Utilities for tests. Included in main codebase since it's used by multiple + * projects. + * + * TODO: See if we can move this to the test codebase by specifying + * test dependencies between projects. + */ +private[spark] object TestUtils { /** * Create a jar that defines classes with the given names. @@ -34,13 +41,14 @@ object TestUtils { * Note: if this is used during class loader tests, class names should be unique * in order to avoid interference between tests. */ - def createJarWithClasses(classNames: Seq[String]): URL = { + def createJarWithClasses(classNames: Seq[String], value: String = ""): URL = { val tempDir = Files.createTempDir() - val files = for (name <- classNames) yield createCompiledClass(name, tempDir) + val files = for (name <- classNames) yield createCompiledClass(name, tempDir, value) val jarFile = new File(tempDir, "testJar-%s.jar".format(System.currentTimeMillis())) createJar(files, jarFile) } + /** * Create a jar file that contains this set of files. All files will be located at the root * of the jar. @@ -80,9 +88,11 @@ object TestUtils { } /** Creates a compiled class with the given name. Class file will be placed in destDir. */ - def createCompiledClass(className: String, destDir: File): File = { + def createCompiledClass(className: String, destDir: File, value: String = ""): File = { val compiler = ToolProvider.getSystemJavaCompiler - val sourceFile = new JavaSourceFromString(className, s"public class $className {}") + val sourceFile = new JavaSourceFromString(className, + "public class " + className + " { @Override public String toString() { " + + "return \"" + value + "\";}}") // Calling this outputs a class file in pwd. It's easier to just rename the file than // build a custom FileManager that controls the output location. diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index aecb069e4202b..c12bd922d40e4 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -291,15 +291,19 @@ private[spark] class Executor( * Create a ClassLoader for use in tasks, adding any JARs specified by the user or any classes * created by the interpreter to the search path */ - private def createClassLoader(): ExecutorURLClassLoader = { - val loader = Thread.currentThread().getContextClassLoader + private def createClassLoader(): MutableURLClassLoader = { + val loader = this.getClass.getClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. val urls = currentJars.keySet.map { uri => new File(uri.split("/").last).toURI.toURL }.toArray - new ExecutorURLClassLoader(urls, loader) + val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false) + userClassPathFirst match { + case true => new ChildExecutorURLClassLoader(urls, loader) + case false => new ExecutorURLClassLoader(urls, loader) + } } /** @@ -310,11 +314,14 @@ private[spark] class Executor( val classUri = conf.get("spark.repl.class.uri", null) if (classUri != null) { logInfo("Using REPL class URI: " + classUri) + val userClassPathFirst: java.lang.Boolean = + conf.getBoolean("spark.files.userClassPathFirst", false) try { val klass = Class.forName("org.apache.spark.repl.ExecutorClassLoader") .asInstanceOf[Class[_ <: ClassLoader]] - val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader]) - constructor.newInstance(classUri, parent) + val constructor = klass.getConstructor(classOf[String], classOf[ClassLoader], + classOf[Boolean]) + constructor.newInstance(classUri, parent, userClassPathFirst) } catch { case _: ClassNotFoundException => logError("Could not find org.apache.spark.repl.ExecutorClassLoader on classpath!") diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala index f9bfe8ed2f5ba..208e77073fd03 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala @@ -19,13 +19,56 @@ package org.apache.spark.executor import java.net.{URLClassLoader, URL} +import org.apache.spark.util.ParentClassLoader + /** * The addURL method in URLClassLoader is protected. We subclass it to make this accessible. + * We also make changes so user classes can come before the default classes. */ + +private[spark] trait MutableURLClassLoader extends ClassLoader { + def addURL(url: URL) + def getURLs: Array[URL] +} + +private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) + extends MutableURLClassLoader { + + private object userClassLoader extends URLClassLoader(urls, null){ + override def addURL(url: URL) { + super.addURL(url) + } + override def findClass(name: String): Class[_] = { + super.findClass(name) + } + } + + private val parentClassLoader = new ParentClassLoader(parent) + + override def findClass(name: String): Class[_] = { + try { + userClassLoader.findClass(name) + } catch { + case e: ClassNotFoundException => { + parentClassLoader.loadClass(name) + } + } + } + + def addURL(url: URL) { + userClassLoader.addURL(url) + } + + def getURLs() = { + userClassLoader.getURLs() + } +} + private[spark] class ExecutorURLClassLoader(urls: Array[URL], parent: ClassLoader) - extends URLClassLoader(urls, parent) { + extends URLClassLoader(urls, parent) with MutableURLClassLoader { override def addURL(url: URL) { super.addURL(url) } } + diff --git a/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.scala new file mode 100644 index 0000000000000..3abc12681fe9a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/util/ParentClassLoader.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.util + +/** + * A class loader which makes findClass accesible to the child + */ +private[spark] class ParentClassLoader(parent: ClassLoader) extends ClassLoader(parent) { + + override def findClass(name: String) = { + super.findClass(name) + } + + override def loadClass(name: String): Class[_] = { + super.loadClass(name) + } +} diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala new file mode 100644 index 0000000000000..c40cfc0696fce --- /dev/null +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala @@ -0,0 +1,67 @@ +/* + * 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.executor + +import java.io.File +import java.net.URLClassLoader + +import org.scalatest.FunSuite + +import org.apache.spark.TestUtils + +class ExecutorURLClassLoaderSuite extends FunSuite { + + val childClassNames = List("FakeClass1", "FakeClass2") + val parentClassNames = List("FakeClass1", "FakeClass2", "FakeClass3") + val urls = List(TestUtils.createJarWithClasses(childClassNames, "1")).toArray + val urls2 = List(TestUtils.createJarWithClasses(parentClassNames, "2")).toArray + + test("child first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass2").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "1") + } + + test("parent first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass1").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fall back") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + val fakeClass = classLoader.loadClass("FakeClass3").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fail") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ChildExecutorURLClassLoader(urls, parentLoader) + intercept[java.lang.ClassNotFoundException] { + classLoader.loadClass("FakeClassDoesNotExist").newInstance() + } + } + + +} diff --git a/docs/configuration.md b/docs/configuration.md index 57bda20edcdf1..9c602402f0635 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -596,6 +596,15 @@ Apart from these, the following properties are also available, and may be useful the driver. + + spark.files.userClassPathFirst + false + + (Experimental) Whether to give user-added jars precedence over Spark's own jars when + loading classes in Executors. This feature can be used to mitigate conflicts between + Spark's dependencies and user dependencies. It is currently an experimental feature. + + spark.authenticate false diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 08667aac2cd2d..694f90a83ab67 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -195,6 +195,7 @@ object SparkBuild extends Build { concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), resolvers ++= Seq( + // HTTPS is unavailable for Maven Central "Maven Repository" at "http://repo.maven.apache.org/maven2", "Apache Repository" at "https://repository.apache.org/content/repositories/releases", "JBoss Repository" at "https://repository.jboss.org/nexus/content/repositories/releases/", diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index bf73800388ebf..a30dcfdcecf27 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -26,21 +26,23 @@ import org.apache.hadoop.fs.{FileSystem, Path} import org.apache.spark.SparkEnv import org.apache.spark.util.Utils - +import org.apache.spark.util.ParentClassLoader import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm._ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ - /** * A ClassLoader that reads classes from a Hadoop FileSystem or HTTP URI, - * used to load classes defined by the interpreter when the REPL is used - */ -class ExecutorClassLoader(classUri: String, parent: ClassLoader) -extends ClassLoader(parent) { + * used to load classes defined by the interpreter when the REPL is used. + * Allows the user to specify if user class path should be first + */ +class ExecutorClassLoader(classUri: String, parent: ClassLoader, + userClassPathFirst: Boolean) extends ClassLoader { val uri = new URI(classUri) val directory = uri.getPath + val parentLoader = new ParentClassLoader(parent) + // Hadoop FileSystem object for our URI, if it isn't using HTTP var fileSystem: FileSystem = { if (uri.getScheme() == "http") { @@ -49,8 +51,27 @@ extends ClassLoader(parent) { FileSystem.get(uri, new Configuration()) } } - + override def findClass(name: String): Class[_] = { + userClassPathFirst match { + case true => findClassLocally(name).getOrElse(parentLoader.loadClass(name)) + case false => { + try { + parentLoader.loadClass(name) + } catch { + case e: ClassNotFoundException => { + val classOption = findClassLocally(name) + classOption match { + case None => throw new ClassNotFoundException(name, e) + case Some(a) => a + } + } + } + } + } + } + + def findClassLocally(name: String): Option[Class[_]] = { try { val pathInDirectory = name.replace('.', '/') + ".class" val inputStream = { @@ -68,9 +89,9 @@ extends ClassLoader(parent) { } val bytes = readAndTransformClass(name, inputStream) inputStream.close() - return defineClass(name, bytes, 0, bytes.length) + Some(defineClass(name, bytes, 0, bytes.length)) } catch { - case e: Exception => throw new ClassNotFoundException(name, e) + case e: Exception => None } } diff --git a/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala new file mode 100644 index 0000000000000..336df988a1b7f --- /dev/null +++ b/repl/src/test/scala/org/apache/spark/repl/ExecutorClassLoaderSuite.scala @@ -0,0 +1,76 @@ +/* + * 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.repl + +import java.io.File +import java.net.URLClassLoader + +import org.scalatest.BeforeAndAfterAll +import org.scalatest.FunSuite + +import com.google.common.io.Files + +import org.apache.spark.TestUtils + +class ExecutorClassLoaderSuite extends FunSuite with BeforeAndAfterAll { + + val childClassNames = List("ReplFakeClass1", "ReplFakeClass2") + val parentClassNames = List("ReplFakeClass1", "ReplFakeClass2", "ReplFakeClass3") + val tempDir1 = Files.createTempDir() + val tempDir2 = Files.createTempDir() + val url1 = "file://" + tempDir1 + val urls2 = List(tempDir2.toURI.toURL).toArray + + override def beforeAll() { + childClassNames.foreach(TestUtils.createCompiledClass(_, tempDir1, "1")) + parentClassNames.foreach(TestUtils.createCompiledClass(_, tempDir2, "2")) + } + + test("child first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, true) + val fakeClass = classLoader.loadClass("ReplFakeClass2").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "1") + } + + test("parent first") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, false) + val fakeClass = classLoader.loadClass("ReplFakeClass1").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fall back") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, true) + val fakeClass = classLoader.loadClass("ReplFakeClass3").newInstance() + val fakeClassVersion = fakeClass.toString + assert(fakeClassVersion === "2") + } + + test("child first can fail") { + val parentLoader = new URLClassLoader(urls2, null) + val classLoader = new ExecutorClassLoader(url1, parentLoader, true) + intercept[java.lang.ClassNotFoundException] { + classLoader.loadClass("ReplFakeClassDoesNotExist").newInstance() + } + } + +} From 9689b663a2a4947ad60795321c770052f3c637f1 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 8 Apr 2014 23:01:15 -0700 Subject: [PATCH 241/397] [SPARK-1390] Refactoring of matrices backed by RDDs This is to refactor interfaces for matrices backed by RDDs. It would be better if we have a clear separation of local matrices and those backed by RDDs. Right now, we have 1. `org.apache.spark.mllib.linalg.SparseMatrix`, which is a wrapper over an RDD of matrix entries, i.e., coordinate list format. 2. `org.apache.spark.mllib.linalg.TallSkinnyDenseMatrix`, which is a wrapper over RDD[Array[Double]], i.e. row-oriented format. We will see naming collision when we introduce local `SparseMatrix`, and the name `TallSkinnyDenseMatrix` is not exact if we switch to `RDD[Vector]` from `RDD[Array[Double]]`. It would be better to have "RDD" in the class name to suggest that operations may trigger jobs. The proposed names are (all under `org.apache.spark.mllib.linalg.rdd`): 1. `RDDMatrix`: trait for matrices backed by one or more RDDs 2. `CoordinateRDDMatrix`: wrapper of `RDD[(Long, Long, Double)]` 3. `RowRDDMatrix`: wrapper of `RDD[Vector]` whose rows do not have special ordering 4. `IndexedRowRDDMatrix`: wrapper of `RDD[(Long, Vector)]` whose rows are associated with indices The current code also introduces local matrices. Author: Xiangrui Meng Closes #296 from mengxr/mat and squashes the following commits: 24d8294 [Xiangrui Meng] fix for groupBy returning Iterable bfc2b26 [Xiangrui Meng] merge master 8e4f1f5 [Xiangrui Meng] Merge branch 'master' into mat 0135193 [Xiangrui Meng] address Reza's comments 03cd7e1 [Xiangrui Meng] add pca/gram to IndexedRowMatrix add toBreeze to DistributedMatrix for test simplify tests b177ff1 [Xiangrui Meng] address Matei's comments be119fe [Xiangrui Meng] rename m/n to numRows/numCols for local matrix add tests for matrices b881506 [Xiangrui Meng] rename SparkPCA/SVD to TallSkinnyPCA/SVD e7d0d4a [Xiangrui Meng] move IndexedRDDMatrixRow to IndexedRowRDDMatrix 0d1491c [Xiangrui Meng] fix test errors a85262a [Xiangrui Meng] rename RDDMatrixRow to IndexedRDDMatrixRow b8b6ac3 [Xiangrui Meng] Remove old code 4cf679c [Xiangrui Meng] port pca to RowRDDMatrix, and add multiply and covariance 7836e2f [Xiangrui Meng] initial refactoring of matrices backed by RDDs --- .../spark/examples/mllib/SparkPCA.scala | 51 --- .../spark/examples/mllib/SparkSVD.scala | 59 --- .../spark/examples/mllib/TallSkinnyPCA.scala | 64 +++ .../spark/examples/mllib/TallSkinnySVD.scala | 64 +++ .../apache/spark/mllib/linalg/Matrices.scala | 101 +++++ .../apache/spark/mllib/linalg/MatrixSVD.scala | 29 -- .../org/apache/spark/mllib/linalg/PCA.scala | 120 ------ .../org/apache/spark/mllib/linalg/SVD.scala | 395 ------------------ ...scala => SingularValueDecomposition.scala} | 9 +- .../mllib/linalg/TallSkinnyMatrixSVD.scala | 31 -- .../linalg/distributed/CoordinateMatrix.scala | 112 +++++ .../DistributedMatrix.scala} | 23 +- .../linalg/distributed/IndexedRowMatrix.scala | 148 +++++++ .../mllib/linalg/distributed/RowMatrix.scala | 344 +++++++++++++++ .../org/apache/spark/mllib/util/LAUtils.scala | 67 --- .../linalg/BreezeMatrixConversionSuite.scala} | 29 +- .../spark/mllib/linalg/MatricesSuite.scala} | 27 +- .../apache/spark/mllib/linalg/PCASuite.scala | 124 ------ .../apache/spark/mllib/linalg/SVDSuite.scala | 194 --------- .../distributed/CoordinateMatrixSuite.scala | 98 +++++ .../distributed/IndexedRowMatrixSuite.scala | 120 ++++++ .../linalg/distributed/RowMatrixSuite.scala | 173 ++++++++ 22 files changed, 1280 insertions(+), 1102 deletions(-) delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala delete mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala create mode 100644 examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{MatrixRow.scala => SingularValueDecomposition.scala} (81%) delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala rename mllib/src/main/scala/org/apache/spark/mllib/linalg/{SparseMatrix.scala => distributed/DistributedMatrix.scala} (60%) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala delete mode 100644 mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala rename mllib/src/{main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala => test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala} (51%) rename mllib/src/{main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala => test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala} (58%) delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala delete mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala deleted file mode 100644 index d4e08c5e12d81..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkPCA.scala +++ /dev/null @@ -1,51 +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.examples.mllib - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.PCA -import org.apache.spark.mllib.linalg.MatrixEntry -import org.apache.spark.mllib.linalg.SparseMatrix -import org.apache.spark.mllib.util._ - - -/** - * Compute PCA of an example matrix. - */ -object SparkPCA { - def main(args: Array[String]) { - if (args.length != 3) { - System.err.println("Usage: SparkPCA m n") - System.exit(1) - } - val sc = new SparkContext(args(0), "PCA", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - - val m = args(2).toInt - val n = args(3).toInt - - // Make example matrix - val data = Array.tabulate(m, n) { (a, b) => - (a + 2).toDouble * (b + 1) / (1 + a + b) } - - // recover top principal component - val coeffs = new PCA().setK(1).compute(sc.makeRDD(data)) - - println("top principal component = " + coeffs.mkString(", ")) - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala deleted file mode 100644 index 2933cec497b37..0000000000000 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/SparkSVD.scala +++ /dev/null @@ -1,59 +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.examples.mllib - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.SVD -import org.apache.spark.mllib.linalg.MatrixEntry -import org.apache.spark.mllib.linalg.SparseMatrix - -/** - * Compute SVD of an example matrix - * Input file should be comma separated, 1 indexed of the form - * i,j,value - * Where i is the column, j the row, and value is the matrix entry - * - * For example input file, see: - * mllib/data/als/test.data (example is 4 x 4) - */ -object SparkSVD { - def main(args: Array[String]) { - if (args.length != 4) { - System.err.println("Usage: SparkSVD m n") - System.exit(1) - } - val sc = new SparkContext(args(0), "SVD", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - - // Load and parse the data file - val data = sc.textFile(args(1)).map { line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt - 1, parts(1).toInt - 1, parts(2).toDouble) - } - val m = args(2).toInt - val n = args(3).toInt - - // recover largest singular vector - val decomposed = new SVD().setK(1).compute(SparseMatrix(data, m, n)) - val u = decomposed.U.data - val s = decomposed.S.data - val v = decomposed.V.data - - println("singular values = " + s.collect().mkString) - } -} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala new file mode 100644 index 0000000000000..a177435e606ab --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -0,0 +1,64 @@ +/* + * 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.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.Vectors + +/** + * Compute the principal components of a tall-and-skinny matrix, whose rows are observations. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 + * }}} + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). + */ +object TallSkinnyPCA { + def main(args: Array[String]) { + if (args.length != 2) { + System.err.println("Usage: TallSkinnyPCA ") + System.exit(1) + } + + val conf = new SparkConf() + .setMaster(args(0)) + .setAppName("TallSkinnyPCA") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(SparkContext.jarOfClass(this.getClass)) + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(args(1)).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + } + val mat = new RowMatrix(rows) + + // Compute principal components. + val pc = mat.computePrincipalComponents(mat.numCols().toInt) + + println("Principal components are:\n" + pc) + + sc.stop() + } +} diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala new file mode 100644 index 0000000000000..49d09692c8e4a --- /dev/null +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -0,0 +1,64 @@ +/* + * 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.examples.mllib + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.linalg.Vectors + +/** + * Compute the singular value decomposition (SVD) of a tall-and-skinny matrix. + * + * The input matrix must be stored in row-oriented dense format, one line per row with its entries + * separated by space. For example, + * {{{ + * 0.5 1.0 + * 2.0 3.0 + * 4.0 5.0 + * }}} + * represents a 3-by-2 matrix, whose first row is (0.5, 1.0). + */ +object TallSkinnySVD { + def main(args: Array[String]) { + if (args.length != 2) { + System.err.println("Usage: TallSkinnySVD ") + System.exit(1) + } + + val conf = new SparkConf() + .setMaster(args(0)) + .setAppName("TallSkinnySVD") + .setSparkHome(System.getenv("SPARK_HOME")) + .setJars(SparkContext.jarOfClass(this.getClass)) + val sc = new SparkContext(conf) + + // Load and parse the data file. + val rows = sc.textFile(args(1)).map { line => + val values = line.split(' ').map(_.toDouble) + Vectors.dense(values) + } + val mat = new RowMatrix(rows) + + // Compute SVD. + val svd = mat.computeSVD(mat.numCols().toInt) + + println("Singular values are " + svd.s) + + sc.stop() + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala new file mode 100644 index 0000000000000..b11ba5d30fbd3 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Matrices.scala @@ -0,0 +1,101 @@ +/* + * 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.mllib.linalg + +import breeze.linalg.{Matrix => BM, DenseMatrix => BDM} + +/** + * Trait for a local matrix. + */ +trait Matrix extends Serializable { + + /** Number of rows. */ + def numRows: Int + + /** Number of columns. */ + def numCols: Int + + /** Converts to a dense array in column major. */ + def toArray: Array[Double] + + /** Converts to a breeze matrix. */ + private[mllib] def toBreeze: BM[Double] + + /** Gets the (i, j)-th element. */ + private[mllib] def apply(i: Int, j: Int): Double = toBreeze(i, j) + + override def toString: String = toBreeze.toString() +} + +/** + * Column-majored dense matrix. + * The entry values are stored in a single array of doubles with columns listed in sequence. + * For example, the following matrix + * {{{ + * 1.0 2.0 + * 3.0 4.0 + * 5.0 6.0 + * }}} + * is stored as `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]`. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major + */ +class DenseMatrix(val numRows: Int, val numCols: Int, val values: Array[Double]) extends Matrix { + + require(values.length == numRows * numCols) + + override def toArray: Array[Double] = values + + private[mllib] override def toBreeze: BM[Double] = new BDM[Double](numRows, numCols, values) +} + +/** + * Factory methods for [[org.apache.spark.mllib.linalg.Matrix]]. + */ +object Matrices { + + /** + * Creates a column-majored dense matrix. + * + * @param numRows number of rows + * @param numCols number of columns + * @param values matrix entries in column major + */ + def dense(numRows: Int, numCols: Int, values: Array[Double]): Matrix = { + new DenseMatrix(numRows, numCols, values) + } + + /** + * Creates a Matrix instance from a breeze matrix. + * @param breeze a breeze matrix + * @return a Matrix instance + */ + private[mllib] def fromBreeze(breeze: BM[Double]): Matrix = { + breeze match { + case dm: BDM[Double] => + require(dm.majorStride == dm.rows, + "Do not support stride size different from the number of rows.") + new DenseMatrix(dm.rows, dm.cols, dm.data) + case _ => + throw new UnsupportedOperationException( + s"Do not support conversion from type ${breeze.getClass.getName}.") + } + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala deleted file mode 100644 index 319f82b449096..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixSVD.scala +++ /dev/null @@ -1,29 +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.mllib.linalg - -/** - * Class that represents the SV decomposition of a matrix - * - * @param U such that A = USV^T - * @param S such that A = USV^T - * @param V such that A = USV^T - */ -case class MatrixSVD(val U: SparseMatrix, - val S: SparseMatrix, - val V: SparseMatrix) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala deleted file mode 100644 index fe5b3f6c7e463..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/PCA.scala +++ /dev/null @@ -1,120 +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.mllib.linalg - -import org.apache.spark.rdd.RDD - - -import org.jblas.DoubleMatrix - - -/** - * Class used to obtain principal components - */ -class PCA { - private var k = 1 - - /** - * Set the number of top-k principle components to return - */ - def setK(k: Int): PCA = { - this.k = k - this - } - - /** - * Compute PCA using the current set parameters - */ - def compute(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { - computePCA(matrix) - } - - /** - * Compute PCA using the parameters currently set - * See computePCA() for more details - */ - def compute(matrix: RDD[Array[Double]]): Array[Array[Double]] = { - computePCA(matrix) - } - - /** - * Computes the top k principal component coefficients for the m-by-n data matrix X. - * Rows of X correspond to observations and columns correspond to variables. - * The coefficient matrix is n-by-k. Each column of coeff contains coefficients - * for one principal component, and the columns are in descending - * order of component variance. - * This function centers the data and uses the - * singular value decomposition (SVD) algorithm. - * - * @param matrix dense matrix to perform PCA on - * @return An nxk matrix with principal components in columns. Columns are inner arrays - */ - private def computePCA(matrix: TallSkinnyDenseMatrix): Array[Array[Double]] = { - val m = matrix.m - val n = matrix.n - - if (m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a well-formed matrix: m=$m n=$n") - } - - computePCA(matrix.rows.map(_.data)) - } - - /** - * Computes the top k principal component coefficients for the m-by-n data matrix X. - * Rows of X correspond to observations and columns correspond to variables. - * The coefficient matrix is n-by-k. Each column of coeff contains coefficients - * for one principal component, and the columns are in descending - * order of component variance. - * This function centers the data and uses the - * singular value decomposition (SVD) algorithm. - * - * @param matrix dense matrix to perform pca on - * @return An nxk matrix of principal components - */ - private def computePCA(matrix: RDD[Array[Double]]): Array[Array[Double]] = { - val n = matrix.first.size - - // compute column sums and normalize matrix - val colSumsTemp = matrix.map((_, 1)).fold((Array.ofDim[Double](n), 0)) { - (a, b) => - val am = new DoubleMatrix(a._1) - val bm = new DoubleMatrix(b._1) - am.addi(bm) - (a._1, a._2 + b._2) - } - - val m = colSumsTemp._2 - val colSums = colSumsTemp._1.map(x => x / m) - - val data = matrix.map { - x => - val row = Array.ofDim[Double](n) - var i = 0 - while (i < n) { - row(i) = x(i) - colSums(i) - i += 1 - } - row - } - - val (u, s, v) = new SVD().setK(k).compute(data) - v - } -} - diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala deleted file mode 100644 index 0d97b7d92f155..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SVD.scala +++ /dev/null @@ -1,395 +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.mllib.linalg - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.jblas.{DoubleMatrix, Singular, MatrixFunctions} - -/** - * Class used to obtain singular value decompositions - */ -class SVD { - private var k = 1 - private var computeU = true - - // All singular values smaller than rCond * sigma(0) - // are treated as zero, where sigma(0) is the largest singular value. - private var rCond = 1e-9 - - /** - * Set the number of top-k singular vectors to return - */ - def setK(k: Int): SVD = { - this.k = k - this - } - - /** - * Sets the reciprocal condition number (rCond). All singular values - * smaller than rCond * sigma(0) are treated as zero, - * where sigma(0) is the largest singular value. - */ - def setReciprocalConditionNumber(smallS: Double): SVD = { - this.rCond = smallS - this - } - - /** - * Should U be computed? - */ - def setComputeU(compU: Boolean): SVD = { - this.computeU = compU - this - } - - /** - * Compute SVD using the current set parameters - */ - def compute(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { - denseSVD(matrix) - } - - /** - * Compute SVD using the current set parameters - * Returns (U, S, V) such that A = USV^T - * U is a row-by-row dense matrix - * S is a simple double array of singular values - * V is a 2d array matrix - * See [[denseSVD]] for more documentation - */ - def compute(matrix: RDD[Array[Double]]): - (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { - denseSVD(matrix) - } - - /** - * See full paramter definition of sparseSVD for more description. - * - * @param matrix sparse matrix to factorize - * @return Three sparse matrices: U, S, V such that A = USV^T - */ - def compute(matrix: SparseMatrix): MatrixSVD = { - sparseSVD(matrix) - } - - /** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require n^2 doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * @param matrix dense matrix to factorize - * @return See [[TallSkinnyMatrixSVD]] for the output matrices and arrays - */ - private def denseSVD(matrix: TallSkinnyDenseMatrix): TallSkinnyMatrixSVD = { - val m = matrix.m - val n = matrix.n - - if (m < n || m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a tall and skinny matrix m=$m n=$n") - } - - if (k < 1 || k > n) { - throw new IllegalArgumentException("Request up to n singular values n=$n k=$k") - } - - val rowIndices = matrix.rows.map(_.i) - - // compute SVD - val (u, sigma, v) = denseSVD(matrix.rows.map(_.data)) - - if (computeU) { - // prep u for returning - val retU = TallSkinnyDenseMatrix( - u.zip(rowIndices).map { - case (row, i) => MatrixRow(i, row) - }, - m, - k) - - TallSkinnyMatrixSVD(retU, sigma, v) - } else { - TallSkinnyMatrixSVD(null, sigma, v) - } - } - - /** - * Singular Value Decomposition for Tall and Skinny matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require n^2 doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * The return values are as lean as possible: an RDD of rows for U, - * a simple array for sigma, and a dense 2d matrix array for V - * - * @param matrix dense matrix to factorize - * @return Three matrices: U, S, V such that A = USV^T - */ - private def denseSVD(matrix: RDD[Array[Double]]): - (RDD[Array[Double]], Array[Double], Array[Array[Double]]) = { - val n = matrix.first.size - - if (k < 1 || k > n) { - throw new IllegalArgumentException( - "Request up to n singular values k=$k n=$n") - } - - // Compute A^T A - val fullata = matrix.mapPartitions { - iter => - val localATA = Array.ofDim[Double](n, n) - while (iter.hasNext) { - val row = iter.next() - var i = 0 - while (i < n) { - var j = 0 - while (j < n) { - localATA(i)(j) += row(i) * row(j) - j += 1 - } - i += 1 - } - } - Iterator(localATA) - }.fold(Array.ofDim[Double](n, n)) { - (a, b) => - var i = 0 - while (i < n) { - var j = 0 - while (j < n) { - a(i)(j) += b(i)(j) - j += 1 - } - i += 1 - } - a - } - - // Construct jblas A^T A locally - val ata = new DoubleMatrix(fullata) - - // Since A^T A is small, we can compute its SVD directly - val svd = Singular.sparseSVD(ata) - val V = svd(0) - val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x / svd(1).get(0) > rCond) - - val sk = Math.min(k, sigmas.size) - val sigma = sigmas.take(sk) - - // prepare V for returning - val retV = Array.tabulate(n, sk)((i, j) => V.get(i, j)) - - if (computeU) { - // Compute U as U = A V S^-1 - // Compute VS^-1 - val vsinv = new DoubleMatrix(Array.tabulate(n, sk)((i, j) => V.get(i, j) / sigma(j))) - val retU = matrix.map { - x => - val v = new DoubleMatrix(Array(x)) - v.mmul(vsinv).data - } - (retU, sigma, retV) - } else { - (null, sigma, retV) - } - } - - /** - * Singular Value Decomposition for Tall and Skinny sparse matrices. - * Given an m x n matrix A, this will compute matrices U, S, V such that - * A = U * S * V' - * - * There is no restriction on m, but we require O(n^2) doubles to fit in memory. - * Further, n should be less than m. - * - * The decomposition is computed by first computing A'A = V S^2 V', - * computing svd locally on that (since n x n is small), - * from which we recover S and V. - * Then we compute U via easy matrix multiplication - * as U = A * V * S^-1 - * - * Only the k largest singular values and associated vectors are found. - * If there are k such values, then the dimensions of the return will be: - * - * S is k x k and diagonal, holding the singular values on diagonal - * U is m x k and satisfies U'U = eye(k) - * V is n x k and satisfies V'V = eye(k) - * - * All input and output is expected in sparse matrix format, 0-indexed - * as tuples of the form ((i,j),value) all in RDDs using the - * SparseMatrix class - * - * @param matrix sparse matrix to factorize - * @return Three sparse matrices: U, S, V such that A = USV^T - */ - private def sparseSVD(matrix: SparseMatrix): MatrixSVD = { - val data = matrix.data - val m = matrix.m - val n = matrix.n - - if (m < n || m <= 0 || n <= 0) { - throw new IllegalArgumentException("Expecting a tall and skinny matrix") - } - - if (k < 1 || k > n) { - throw new IllegalArgumentException("Must request up to n singular values") - } - - // Compute A^T A, assuming rows are sparse enough to fit in memory - val rows = data.map(entry => - (entry.i, (entry.j, entry.mval))).groupByKey() - val emits = rows.flatMap { - case (rowind, cols) => - cols.flatMap { - case (colind1, mval1) => - cols.map { - case (colind2, mval2) => - ((colind1, colind2), mval1 * mval2) - } - } - }.reduceByKey(_ + _) - - // Construct jblas A^T A locally - val ata = DoubleMatrix.zeros(n, n) - for (entry <- emits.collect()) { - ata.put(entry._1._1, entry._1._2, entry._2) - } - - // Since A^T A is small, we can compute its SVD directly - val svd = Singular.sparseSVD(ata) - val V = svd(0) - // This will be updated to rcond - val sigmas = MatrixFunctions.sqrt(svd(1)).toArray.filter(x => x > 1e-9) - - if (sigmas.size < k) { - throw new Exception("Not enough singular values to return k=" + k + " s=" + sigmas.size) - } - - val sigma = sigmas.take(k) - - val sc = data.sparkContext - - // prepare V for returning - val retVdata = sc.makeRDD( - Array.tabulate(V.rows, sigma.length) { - (i, j) => - MatrixEntry(i, j, V.get(i, j)) - }.flatten) - val retV = SparseMatrix(retVdata, V.rows, sigma.length) - - val retSdata = sc.makeRDD(Array.tabulate(sigma.length) { - x => MatrixEntry(x, x, sigma(x)) - }) - - val retS = SparseMatrix(retSdata, sigma.length, sigma.length) - - // Compute U as U = A V S^-1 - // turn V S^-1 into an RDD as a sparse matrix - val vsirdd = sc.makeRDD(Array.tabulate(V.rows, sigma.length) { - (i, j) => ((i, j), V.get(i, j) / sigma(j)) - }.flatten) - - if (computeU) { - // Multiply A by VS^-1 - val aCols = data.map(entry => (entry.j, (entry.i, entry.mval))) - val bRows = vsirdd.map(entry => (entry._1._1, (entry._1._2, entry._2))) - val retUdata = aCols.join(bRows).map { - case (key, ((rowInd, rowVal), (colInd, colVal))) => - ((rowInd, colInd), rowVal * colVal) - }.reduceByKey(_ + _).map { - case ((row, col), mval) => MatrixEntry(row, col, mval) - } - - val retU = SparseMatrix(retUdata, m, sigma.length) - MatrixSVD(retU, retS, retV) - } else { - MatrixSVD(null, retS, retV) - } - } -} - -/** - * Top-level methods for calling sparse Singular Value Decomposition - * NOTE: All matrices are 0-indexed - */ -object SVD { - def main(args: Array[String]) { - if (args.length < 8) { - println("Usage: SVD " + - " ") - System.exit(1) - } - - val (master, inputFile, m, n, k, output_u, output_s, output_v) = - (args(0), args(1), args(2).toInt, args(3).toInt, - args(4).toInt, args(5), args(6), args(7)) - - val sc = new SparkContext(master, "SVD") - - val rawData = sc.textFile(inputFile) - val data = rawData.map { - line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble) - } - - val decomposed = new SVD().setK(k).compute(SparseMatrix(data, m, n)) - val u = decomposed.U.data - val s = decomposed.S.data - val v = decomposed.V.data - - println("Computed " + s.collect().length + " singular values and vectors") - u.saveAsTextFile(output_u) - s.saveAsTextFile(output_s) - v.saveAsTextFile(output_v) - System.exit(0) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala similarity index 81% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala index 2608a67bfe260..46b105457430c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixRow.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/SingularValueDecomposition.scala @@ -17,10 +17,5 @@ package org.apache.spark.mllib.linalg -/** - * Class that represents a row of a dense matrix - * - * @param i row index (0 indexing used) - * @param data entries of the row - */ -case class MatrixRow(val i: Int, val data: Array[Double]) +/** Represents singular value decomposition (SVD) factors. */ +case class SingularValueDecomposition[UType, VType](U: UType, s: Vector, V: VType) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala deleted file mode 100644 index b3a450e92394e..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyMatrixSVD.scala +++ /dev/null @@ -1,31 +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.mllib.linalg - -/** - * Class that represents the singular value decomposition of a matrix - * - * @param U such that A = USV^T is a TallSkinnyDenseMatrix - * @param S such that A = USV^T is a simple double array - * @param V such that A = USV^T, V is a 2d array matrix that holds - * singular vectors in columns. Columns are inner arrays - * i.e. V(i)(j) is standard math notation V_{ij} - */ -case class TallSkinnyMatrixSVD(val U: TallSkinnyDenseMatrix, - val S: Array[Double], - val V: Array[Array[Double]]) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala new file mode 100644 index 0000000000000..9194f657494b2 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -0,0 +1,112 @@ +/* + * 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.mllib.linalg.distributed + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.linalg.Vectors + +/** + * Represents an entry in an distributed matrix. + * @param i row index + * @param j column index + * @param value value of the entry + */ +case class MatrixEntry(i: Long, j: Long, value: Double) + +/** + * Represents a matrix in coordinate format. + * + * @param entries matrix entries + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the max row index plus one. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the max column index plus one. + */ +class CoordinateMatrix( + val entries: RDD[MatrixEntry], + private var nRows: Long, + private var nCols: Long) extends DistributedMatrix { + + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(entries: RDD[MatrixEntry]) = this(entries, 0L, 0L) + + /** Gets or computes the number of columns. */ + override def numCols(): Long = { + if (nCols <= 0L) { + computeSize() + } + nCols + } + + /** Gets or computes the number of rows. */ + override def numRows(): Long = { + if (nRows <= 0L) { + computeSize() + } + nRows + } + + /** Converts to IndexedRowMatrix. The number of columns must be within the integer range. */ + def toIndexedRowMatrix(): IndexedRowMatrix = { + val nl = numCols() + if (nl > Int.MaxValue) { + sys.error(s"Cannot convert to a row-oriented format because the number of columns $nl is " + + "too large.") + } + val n = nl.toInt + val indexedRows = entries.map(entry => (entry.i, (entry.j.toInt, entry.value))) + .groupByKey() + .map { case (i, vectorEntries) => + IndexedRow(i, Vectors.sparse(n, vectorEntries.toSeq)) + } + new IndexedRowMatrix(indexedRows, numRows(), n) + } + + /** + * Converts to RowMatrix, dropping row indices after grouping by row index. + * The number of columns must be within the integer range. + */ + def toRowMatrix(): RowMatrix = { + toIndexedRowMatrix().toRowMatrix() + } + + /** Determines the size by computing the max row/column index. */ + private def computeSize() { + // Reduce will throw an exception if `entries` is empty. + val (m1, n1) = entries.map(entry => (entry.i, entry.j)).reduce { case ((i1, j1), (i2, j2)) => + (math.max(i1, i2), math.max(j1, j2)) + } + // There may be empty columns at the very right and empty rows at the very bottom. + nRows = math.max(nRows, m1 + 1L) + nCols = math.max(nCols, n1 + 1L) + } + + /** Collects data and assembles a local matrix. */ + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + entries.collect().foreach { case MatrixEntry(i, j, value) => + mat(i.toInt, j.toInt) = value + } + mat + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala similarity index 60% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala rename to mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index cbd1a2a5a4bd8..13f72a3c724ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/SparseMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -15,16 +15,23 @@ * limitations under the License. */ -package org.apache.spark.mllib.linalg +package org.apache.spark.mllib.linalg.distributed -import org.apache.spark.rdd.RDD +import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.mllib.linalg.Matrix /** - * Class that represents a sparse matrix - * - * @param data RDD of nonzero entries - * @param m number of rows - * @param n numner of columns + * Represents a distributively stored matrix backed by one or more RDDs. */ -case class SparseMatrix(val data: RDD[MatrixEntry], val m: Int, val n: Int) +trait DistributedMatrix extends Serializable { + + /** Gets or computes the number of rows. */ + def numRows(): Long + + /** Gets or computes the number of columns. */ + def numCols(): Long + + /** Collects data and assembles a local dense breeze matrix (for test only). */ + private[mllib] def toBreeze(): BDM[Double] +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala new file mode 100644 index 0000000000000..e110f070bd7c1 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -0,0 +1,148 @@ +/* + * 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.mllib.linalg.distributed + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg._ +import org.apache.spark.mllib.linalg.SingularValueDecomposition + +/** Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ +case class IndexedRow(index: Long, vector: Vector) + +/** + * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with + * indexed rows. + * + * @param rows indexed rows of this matrix + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the max row index plus one. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the size of the first row. + */ +class IndexedRowMatrix( + val rows: RDD[IndexedRow], + private var nRows: Long, + private var nCols: Int) extends DistributedMatrix { + + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(rows: RDD[IndexedRow]) = this(rows, 0L, 0) + + override def numCols(): Long = { + if (nCols <= 0) { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().vector.size + } + nCols + } + + override def numRows(): Long = { + if (nRows <= 0L) { + // Reduce will throw an exception if `rows` is empty. + nRows = rows.map(_.index).reduce(math.max) + 1L + } + nRows + } + + /** + * Drops row indices and converts this matrix to a + * [[org.apache.spark.mllib.linalg.distributed.RowMatrix]]. + */ + def toRowMatrix(): RowMatrix = { + new RowMatrix(rows.map(_.vector), 0L, nCols) + } + + /** + * Computes the singular value decomposition of this matrix. + * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. + * + * There is no restriction on m, but we require `n^2` doubles to fit in memory. + * Further, n should be less than m. + + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), from which we recover S and V. + * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). + * Note that this approach requires `O(n^3)` time on the master node. + * + * At most k largest non-zero singular values and associated vectors are returned. + * If there are k such values, then the dimensions of the return will be: + * + * U is an [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]] of size m x k that + * satisfies U'U = eye(k), + * s is a Vector of size k, holding the singular values in descending order, + * and V is a local Matrix of size n x k that satisfies V'V = eye(k). + * + * @param k number of singular values to keep. We might return less than k if there are + * numerically zero singular values. See rCond. + * @param computeU whether to compute U + * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) + * are treated as zero, where sigma(0) is the largest singular value. + * @return SingularValueDecomposition(U, s, V) + */ + def computeSVD( + k: Int, + computeU: Boolean = false, + rCond: Double = 1e-9): SingularValueDecomposition[IndexedRowMatrix, Matrix] = { + val indices = rows.map(_.index) + val svd = toRowMatrix().computeSVD(k, computeU, rCond) + val U = if (computeU) { + val indexedRows = indices.zip(svd.U.rows).map { case (i, v) => + IndexedRow(i, v) + } + new IndexedRowMatrix(indexedRows, nRows, nCols) + } else { + null + } + SingularValueDecomposition(U, svd.s, svd.V) + } + + /** + * Multiply this matrix by a local matrix on the right. + * + * @param B a local matrix whose number of rows must match the number of columns of this matrix + * @return an IndexedRowMatrix representing the product, which preserves partitioning + */ + def multiply(B: Matrix): IndexedRowMatrix = { + val mat = toRowMatrix().multiply(B) + val indexedRows = rows.map(_.index).zip(mat.rows).map { case (i, v) => + IndexedRow(i, v) + } + new IndexedRowMatrix(indexedRows, nRows, nCols) + } + + /** + * Computes the Gramian matrix `A^T A`. + */ + def computeGramianMatrix(): Matrix = { + toRowMatrix().computeGramianMatrix() + } + + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + rows.collect().foreach { case IndexedRow(rowIndex, vector) => + val i = rowIndex.toInt + vector.toBreeze.activeIterator.foreach { case (j, v) => + mat(i, j) = v + } + } + mat + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala new file mode 100644 index 0000000000000..f59811f18a68f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -0,0 +1,344 @@ +/* + * 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.mllib.linalg.distributed + +import java.util + +import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} +import breeze.numerics.{sqrt => brzSqrt} +import com.github.fommil.netlib.BLAS.{getInstance => blas} + +import org.apache.spark.mllib.linalg._ +import org.apache.spark.rdd.RDD +import org.apache.spark.Logging + +/** + * Represents a row-oriented distributed Matrix with no meaningful row indices. + * + * @param rows rows stored as an RDD[Vector] + * @param nRows number of rows. A non-positive value means unknown, and then the number of rows will + * be determined by the number of records in the RDD `rows`. + * @param nCols number of columns. A non-positive value means unknown, and then the number of + * columns will be determined by the size of the first row. + */ +class RowMatrix( + val rows: RDD[Vector], + private var nRows: Long, + private var nCols: Int) extends DistributedMatrix with Logging { + + /** Alternative constructor leaving matrix dimensions to be determined automatically. */ + def this(rows: RDD[Vector]) = this(rows, 0L, 0) + + /** Gets or computes the number of columns. */ + override def numCols(): Long = { + if (nCols <= 0) { + // Calling `first` will throw an exception if `rows` is empty. + nCols = rows.first().size + } + nCols + } + + /** Gets or computes the number of rows. */ + override def numRows(): Long = { + if (nRows <= 0L) { + nRows = rows.count() + if (nRows == 0L) { + sys.error("Cannot determine the number of rows because it is not specified in the " + + "constructor and the rows RDD is empty.") + } + } + nRows + } + + /** + * Computes the Gramian matrix `A^T A`. + */ + def computeGramianMatrix(): Matrix = { + val n = numCols().toInt + val nt: Int = n * (n + 1) / 2 + + // Compute the upper triangular part of the gram matrix. + val GU = rows.aggregate(new BDV[Double](new Array[Double](nt)))( + seqOp = (U, v) => { + RowMatrix.dspr(1.0, v, U.data) + U + }, + combOp = (U1, U2) => U1 += U2 + ) + + RowMatrix.triuToFull(n, GU.data) + } + + /** + * Computes the singular value decomposition of this matrix. + * Denote this matrix by A (m x n), this will compute matrices U, S, V such that A = U * S * V'. + * + * There is no restriction on m, but we require `n^2` doubles to fit in memory. + * Further, n should be less than m. + + * The decomposition is computed by first computing A'A = V S^2 V', + * computing svd locally on that (since n x n is small), from which we recover S and V. + * Then we compute U via easy matrix multiplication as U = A * (V * S^-1). + * Note that this approach requires `O(n^3)` time on the master node. + * + * At most k largest non-zero singular values and associated vectors are returned. + * If there are k such values, then the dimensions of the return will be: + * + * U is a RowMatrix of size m x k that satisfies U'U = eye(k), + * s is a Vector of size k, holding the singular values in descending order, + * and V is a Matrix of size n x k that satisfies V'V = eye(k). + * + * @param k number of singular values to keep. We might return less than k if there are + * numerically zero singular values. See rCond. + * @param computeU whether to compute U + * @param rCond the reciprocal condition number. All singular values smaller than rCond * sigma(0) + * are treated as zero, where sigma(0) is the largest singular value. + * @return SingularValueDecomposition(U, s, V) + */ + def computeSVD( + k: Int, + computeU: Boolean = false, + rCond: Double = 1e-9): SingularValueDecomposition[RowMatrix, Matrix] = { + val n = numCols().toInt + require(k > 0 && k <= n, s"Request up to n singular values k=$k n=$n.") + + val G = computeGramianMatrix() + + // TODO: Use sparse SVD instead. + val (u: BDM[Double], sigmaSquares: BDV[Double], v: BDM[Double]) = + brzSvd(G.toBreeze.asInstanceOf[BDM[Double]]) + val sigmas: BDV[Double] = brzSqrt(sigmaSquares) + + // Determine effective rank. + val sigma0 = sigmas(0) + val threshold = rCond * sigma0 + var i = 0 + while (i < k && sigmas(i) >= threshold) { + i += 1 + } + val sk = i + + if (sk < k) { + logWarning(s"Requested $k singular values but only found $sk nonzeros.") + } + + val s = Vectors.dense(util.Arrays.copyOfRange(sigmas.data, 0, sk)) + val V = Matrices.dense(n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) + + if (computeU) { + // N = Vk * Sk^{-1} + val N = new BDM[Double](n, sk, util.Arrays.copyOfRange(u.data, 0, n * sk)) + var i = 0 + var j = 0 + while (j < sk) { + i = 0 + val sigma = sigmas(j) + while (i < n) { + N(i, j) /= sigma + i += 1 + } + j += 1 + } + val U = this.multiply(Matrices.fromBreeze(N)) + SingularValueDecomposition(U, s, V) + } else { + SingularValueDecomposition(null, s, V) + } + } + + /** + * Computes the covariance matrix, treating each row as an observation. + * @return a local dense matrix of size n x n + */ + def computeCovariance(): Matrix = { + val n = numCols().toInt + + if (n > 10000) { + val mem = n * n * java.lang.Double.SIZE / java.lang.Byte.SIZE + logWarning(s"The number of columns $n is greater than 10000! " + + s"We need at least $mem bytes of memory.") + } + + val (m, mean) = rows.aggregate[(Long, BDV[Double])]((0L, BDV.zeros[Double](n)))( + seqOp = (s: (Long, BDV[Double]), v: Vector) => (s._1 + 1L, s._2 += v.toBreeze), + combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => (s1._1 + s2._1, s1._2 += s2._2) + ) + + // Update _m if it is not set, or verify its value. + if (nRows <= 0L) { + nRows = m + } else { + require(nRows == m, + s"The number of rows $m is different from what specified or previously computed: ${nRows}.") + } + + mean :/= m.toDouble + + // We use the formula Cov(X, Y) = E[X * Y] - E[X] E[Y], which is not accurate if E[X * Y] is + // large but Cov(X, Y) is small, but it is good for sparse computation. + // TODO: find a fast and stable way for sparse data. + + val G = computeGramianMatrix().toBreeze.asInstanceOf[BDM[Double]] + + var i = 0 + var j = 0 + val m1 = m - 1.0 + var alpha = 0.0 + while (i < n) { + alpha = m / m1 * mean(i) + j = 0 + while (j < n) { + G(i, j) = G(i, j) / m1 - alpha * mean(j) + j += 1 + } + i += 1 + } + + Matrices.fromBreeze(G) + } + + /** + * Computes the top k principal components. + * Rows correspond to observations and columns correspond to variables. + * The principal components are stored a local matrix of size n-by-k. + * Each column corresponds for one principal component, + * and the columns are in descending order of component variance. + * + * @param k number of top principal components. + * @return a matrix of size n-by-k, whose columns are principal components + */ + def computePrincipalComponents(k: Int): Matrix = { + val n = numCols().toInt + require(k > 0 && k <= n, s"k = $k out of range (0, n = $n]") + + val Cov = computeCovariance().toBreeze.asInstanceOf[BDM[Double]] + + val (u: BDM[Double], _, _) = brzSvd(Cov) + + if (k == n) { + Matrices.dense(n, k, u.data) + } else { + Matrices.dense(n, k, util.Arrays.copyOfRange(u.data, 0, n * k)) + } + } + + /** + * Multiply this matrix by a local matrix on the right. + * + * @param B a local matrix whose number of rows must match the number of columns of this matrix + * @return a [[org.apache.spark.mllib.linalg.distributed.RowMatrix]] representing the product, + * which preserves partitioning + */ + def multiply(B: Matrix): RowMatrix = { + val n = numCols().toInt + require(n == B.numRows, s"Dimension mismatch: $n vs ${B.numRows}") + + require(B.isInstanceOf[DenseMatrix], + s"Only support dense matrix at this time but found ${B.getClass.getName}.") + + val Bb = rows.context.broadcast(B) + val AB = rows.mapPartitions({ iter => + val Bi = Bb.value.toBreeze.asInstanceOf[BDM[Double]] + iter.map(v => Vectors.fromBreeze(Bi.t * v.toBreeze)) + }, preservesPartitioning = true) + + new RowMatrix(AB, nRows, B.numCols) + } + + private[mllib] override def toBreeze(): BDM[Double] = { + val m = numRows().toInt + val n = numCols().toInt + val mat = BDM.zeros[Double](m, n) + var i = 0 + rows.collect().foreach { v => + v.toBreeze.activeIterator.foreach { case (j, v) => + mat(i, j) = v + } + i += 1 + } + mat + } +} + +object RowMatrix { + + /** + * Adds alpha * x * x.t to a matrix in-place. This is the same as BLAS's DSPR. + * + * @param U the upper triangular part of the matrix packed in an array (column major) + */ + private def dspr(alpha: Double, v: Vector, U: Array[Double]): Unit = { + // TODO: Find a better home (breeze?) for this method. + val n = v.size + v match { + case dv: DenseVector => + blas.dspr("U", n, 1.0, dv.values, 1, U) + case sv: SparseVector => + val indices = sv.indices + val values = sv.values + val nnz = indices.length + var colStartIdx = 0 + var prevCol = 0 + var col = 0 + var j = 0 + var i = 0 + var av = 0.0 + while (j < nnz) { + col = indices(j) + // Skip empty columns. + colStartIdx += (col - prevCol) * (col + prevCol + 1) / 2 + col = indices(j) + av = alpha * values(j) + i = 0 + while (i <= j) { + U(colStartIdx + indices(i)) += av * values(i) + i += 1 + } + j += 1 + prevCol = col + } + } + } + + /** + * Fills a full square matrix from its upper triangular part. + */ + private def triuToFull(n: Int, U: Array[Double]): Matrix = { + val G = new BDM[Double](n, n) + + var row = 0 + var col = 0 + var idx = 0 + var value = 0.0 + while (col < n) { + row = 0 + while (row < col) { + value = U(idx) + G(row, col) = value + G(col, row) = value + idx += 1 + row += 1 + } + G(col, col) = U(idx) + idx += 1 + col +=1 + } + + Matrices.dense(n, n, G.data) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala deleted file mode 100644 index 87aac347579c7..0000000000000 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LAUtils.scala +++ /dev/null @@ -1,67 +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.mllib.util - -import org.apache.spark.SparkContext._ - -import org.apache.spark.mllib.linalg._ - -/** - * Helper methods for linear algebra - */ -object LAUtils { - /** - * Convert a SparseMatrix into a TallSkinnyDenseMatrix - * - * @param sp Sparse matrix to be converted - * @return dense version of the input - */ - def sparseToTallSkinnyDense(sp: SparseMatrix): TallSkinnyDenseMatrix = { - val m = sp.m - val n = sp.n - val rows = sp.data.map(x => (x.i, (x.j, x.mval))).groupByKey().map { - case (i, cols) => - val rowArray = Array.ofDim[Double](n) - var j = 0 - val colsItr = cols.iterator - while (colsItr.hasNext) { - val element = colsItr.next - rowArray(element._1) = element._2 - j += 1 - } - MatrixRow(i, rowArray) - } - TallSkinnyDenseMatrix(rows, m, n) - } - - /** - * Convert a TallSkinnyDenseMatrix to a SparseMatrix - * - * @param a matrix to be converted - * @return sparse version of the input - */ - def denseToSparse(a: TallSkinnyDenseMatrix): SparseMatrix = { - val m = a.m - val n = a.n - val data = a.rows.flatMap { - mrow => Array.tabulate(n)(j => MatrixEntry(mrow.i, j, mrow.data(j))) - .filter(x => x.mval != 0) - } - SparseMatrix(data, m, n) - } -} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala similarity index 51% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala rename to mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala index 416996fcbe760..82d49c76ed02b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/MatrixEntry.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/BreezeMatrixConversionSuite.scala @@ -17,11 +17,24 @@ package org.apache.spark.mllib.linalg -/** - * Class that represents an entry in a sparse matrix of doubles. - * - * @param i row index (0 indexing used) - * @param j column index (0 indexing used) - * @param mval value of entry in matrix - */ -case class MatrixEntry(val i: Int, val j: Int, val mval: Double) +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM} + +class BreezeMatrixConversionSuite extends FunSuite { + test("dense matrix to breeze") { + val mat = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val breeze = mat.toBreeze.asInstanceOf[BDM[Double]] + assert(breeze.rows === mat.numRows) + assert(breeze.cols === mat.numCols) + assert(breeze.data.eq(mat.asInstanceOf[DenseMatrix].values), "should not copy data") + } + + test("dense breeze matrix to matrix") { + val breeze = new BDM[Double](3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val mat = Matrices.fromBreeze(breeze).asInstanceOf[DenseMatrix] + assert(mat.numRows === breeze.rows) + assert(mat.numCols === breeze.cols) + assert(mat.values.eq(breeze.data), "should not copy data") + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala similarity index 58% rename from mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala rename to mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala index e4ef3c58e8680..9c66b4db9f16b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/TallSkinnyDenseMatrix.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/MatricesSuite.scala @@ -17,14 +17,23 @@ package org.apache.spark.mllib.linalg -import org.apache.spark.rdd.RDD +import org.scalatest.FunSuite +class MatricesSuite extends FunSuite { + test("dense matrix construction") { + val m = 3 + val n = 2 + val values = Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0) + val mat = Matrices.dense(m, n, values).asInstanceOf[DenseMatrix] + assert(mat.numRows === m) + assert(mat.numCols === n) + assert(mat.values.eq(values), "should not copy data") + assert(mat.toArray.eq(values), "toArray should not copy data") + } -/** - * Class that represents a dense matrix - * - * @param rows RDD of rows - * @param m number of rows - * @param n number of columns - */ -case class TallSkinnyDenseMatrix(val rows: RDD[MatrixRow], val m: Int, val n: Int) + test("dense matrix construction with wrong dimension") { + intercept[RuntimeException] { + Matrices.dense(3, 2, Array(0.0, 1.0, 2.0)) + } + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala deleted file mode 100644 index 5e5086b1bf73e..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/PCASuite.scala +++ /dev/null @@ -1,124 +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.mllib.linalg - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.apache.spark.mllib.util._ - -import org.jblas._ - -class PCASuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-3 - - // Return jblas matrix from sparse matrix RDD - def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { - val data = matrix.data - val ret = DoubleMatrix.zeros(matrix.m, matrix.n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) - ret - } - - def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { - assert(a.rows == b.rows && a.columns == b.columns, - "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") - for (i <- 0 until a.columns) { - val aCol = a.getColumn(i) - val bCol = b.getColumn(i) - val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) - assert(diff < EPSILON, "matrix mismatch: " + diff) - } - } - - test("full rank matrix pca") { - val m = 5 - val n = 3 - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), - (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), - (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3)), 3) - - val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) - } - - test("sparse matrix full rank matrix pca") { - val m = 5 - val n = 3 - // the entry that gets dropped is zero to test sparse support - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten.drop(1) - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), (0,2,0.7054), - (1,0,-0.1448), (1,1,0.7483), (1,2,0.6474), - (2,0,0.9553), (2,1,-0.0649), (2,2,0.2886)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) - - val coeffs = new DoubleMatrix(new PCA().setK(n).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,n)), coeffs) - } - - test("truncated matrix pca") { - val m = 5 - val n = 3 - val dataArr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, Math.sin(a + b + a * b)) }.flatten - - val data = sc.makeRDD(dataArr, 3) - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val realPCAArray = Array((0,0,-0.2579), (0,1,-0.6602), - (1,0,-0.1448), (1,1,0.7483), - (2,0,0.9553), (2,1,-0.0649)) - val realPCA = sc.makeRDD(realPCAArray.map(x => MatrixEntry(x._1, x._2, x._3))) - - val k = 2 - val coeffs = new DoubleMatrix(new PCA().setK(k).compute(a)) - - assertMatrixApproximatelyEquals(getDenseMatrix(SparseMatrix(realPCA,n,k)), coeffs) - } -} - - diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala deleted file mode 100644 index 20e2b0f84be06..0000000000000 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/SVDSuite.scala +++ /dev/null @@ -1,194 +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.mllib.linalg - -import scala.util.Random - -import org.scalatest.BeforeAndAfterAll -import org.scalatest.FunSuite - -import org.jblas.{DoubleMatrix, Singular, MatrixFunctions} - -import org.apache.spark.SparkContext -import org.apache.spark.SparkContext._ -import org.apache.spark.rdd.RDD - -import org.apache.spark.mllib.util._ - -import org.jblas._ - -class SVDSuite extends FunSuite with BeforeAndAfterAll { - @transient private var sc: SparkContext = _ - - override def beforeAll() { - sc = new SparkContext("local", "test") - } - - override def afterAll() { - sc.stop() - System.clearProperty("spark.driver.port") - } - - val EPSILON = 1e-4 - - // Return jblas matrix from sparse matrix RDD - def getDenseMatrix(matrix: SparseMatrix) : DoubleMatrix = { - val data = matrix.data - val m = matrix.m - val n = matrix.n - val ret = DoubleMatrix.zeros(m, n) - matrix.data.collect().map(x => ret.put(x.i, x.j, x.mval)) - ret - } - - def assertMatrixApproximatelyEquals(a: DoubleMatrix, b: DoubleMatrix) { - assert(a.rows == b.rows && a.columns == b.columns, - "dimension mismatch: $a.rows vs $b.rows and $a.columns vs $b.columns") - for (i <- 0 until a.columns) { - val aCol = a.getColumn(i) - val bCol = b.getColumn(i) - val diff = Math.min(aCol.sub(bCol).norm1, aCol.add(bCol).norm1) - assert(diff < EPSILON, "matrix mismatch: " + diff) - } - } - - test("full rank matrix svd") { - val m = 10 - val n = 3 - val datarr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten - val data = sc.makeRDD(datarr, 3) - - val a = SparseMatrix(data, m, n) - - val decomposed = new SVD().setK(n).compute(a) - val u = decomposed.U - val v = decomposed.V - val s = decomposed.S - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) - assertMatrixApproximatelyEquals(retv, svd(2)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("dense full rank matrix svd") { - val m = 10 - val n = 3 - val datarr = Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1) / (1 + a + b)) }.flatten - val data = sc.makeRDD(datarr, 3) - - val a = LAUtils.sparseToTallSkinnyDense(SparseMatrix(data, m, n)) - - val decomposed = new SVD().setK(n).setComputeU(true).compute(a) - val u = LAUtils.denseToSparse(decomposed.U) - val v = decomposed.V - val s = decomposed.S - - val denseA = getDenseMatrix(LAUtils.denseToSparse(a)) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = DoubleMatrix.diag(new DoubleMatrix(s)) - val retv = new DoubleMatrix(v) - - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1))) - assertMatrixApproximatelyEquals(retv, svd(2)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("rank one matrix svd") { - val m = 10 - val n = 3 - val data = sc.makeRDD(Array.tabulate(m, n){ (a,b) => - MatrixEntry(a, b, 1.0) }.flatten ) - val k = 1 - - val a = SparseMatrix(data, m, n) - - val decomposed = new SVD().setK(k).compute(a) - val u = decomposed.U - val s = decomposed.S - val v = decomposed.V - val retrank = s.data.collect().length - - assert(retrank == 1, "rank returned not one") - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) - - // check multiplication guarantee - assertMatrixApproximatelyEquals(retu.mmul(rets).mmul(retv.transpose), denseA) - } - - test("truncated with k") { - val m = 10 - val n = 3 - val data = sc.makeRDD(Array.tabulate(m,n){ (a, b) => - MatrixEntry(a, b, (a + 2).toDouble * (b + 1)/(1 + a + b)) }.flatten ) - val a = SparseMatrix(data, m, n) - - val k = 1 // only one svalue above this - - val decomposed = new SVD().setK(k).compute(a) - val u = decomposed.U - val s = decomposed.S - val v = decomposed.V - val retrank = s.data.collect().length - - val denseA = getDenseMatrix(a) - val svd = Singular.sparseSVD(denseA) - - val retu = getDenseMatrix(u) - val rets = getDenseMatrix(s) - val retv = getDenseMatrix(v) - - assert(retrank == 1, "rank returned not one") - - // check individual decomposition - assertMatrixApproximatelyEquals(retu, svd(0).getColumn(0)) - assertMatrixApproximatelyEquals(rets, DoubleMatrix.diag(svd(1).getRow(0))) - assertMatrixApproximatelyEquals(retv, svd(2).getColumn(0)) - } -} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala new file mode 100644 index 0000000000000..cd45438fb628f --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrixSuite.scala @@ -0,0 +1,98 @@ +/* + * 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.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseMatrix => BDM} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.Vectors + +class CoordinateMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 5 + val n = 4 + var mat: CoordinateMatrix = _ + + override def beforeAll() { + super.beforeAll() + val entries = sc.parallelize(Seq( + (0, 0, 1.0), + (0, 1, 2.0), + (1, 1, 3.0), + (1, 2, 4.0), + (2, 2, 5.0), + (2, 3, 6.0), + (3, 0, 7.0), + (3, 3, 8.0), + (4, 1, 9.0)), 3).map { case (i, j, value) => + MatrixEntry(i, j, value) + } + mat = new CoordinateMatrix(entries) + } + + test("size") { + assert(mat.numRows() === m) + assert(mat.numCols() === n) + } + + test("empty entries") { + val entries = sc.parallelize(Seq[MatrixEntry](), 1) + val emptyMat = new CoordinateMatrix(entries) + intercept[RuntimeException] { + emptyMat.numCols() + } + intercept[RuntimeException] { + emptyMat.numRows() + } + } + + test("toBreeze") { + val expected = BDM( + (1.0, 2.0, 0.0, 0.0), + (0.0, 3.0, 4.0, 0.0), + (0.0, 0.0, 5.0, 6.0), + (7.0, 0.0, 0.0, 8.0), + (0.0, 9.0, 0.0, 0.0)) + assert(mat.toBreeze() === expected) + } + + test("toIndexedRowMatrix") { + val indexedRowMatrix = mat.toIndexedRowMatrix() + val expected = BDM( + (1.0, 2.0, 0.0, 0.0), + (0.0, 3.0, 4.0, 0.0), + (0.0, 0.0, 5.0, 6.0), + (7.0, 0.0, 0.0, 8.0), + (0.0, 9.0, 0.0, 0.0)) + assert(indexedRowMatrix.toBreeze() === expected) + } + + test("toRowMatrix") { + val rowMatrix = mat.toRowMatrix() + val rows = rowMatrix.rows.collect().toSet + val expected = Set( + Vectors.dense(1.0, 2.0, 0.0, 0.0), + Vectors.dense(0.0, 3.0, 4.0, 0.0), + Vectors.dense(0.0, 0.0, 5.0, 6.0), + Vectors.dense(7.0, 0.0, 0.0, 8.0), + Vectors.dense(0.0, 9.0, 0.0, 0.0)) + assert(rows === expected) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala new file mode 100644 index 0000000000000..f7c46f23b746d --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrixSuite.scala @@ -0,0 +1,120 @@ +/* + * 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.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{diag => brzDiag, DenseMatrix => BDM, DenseVector => BDV} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.linalg.{Matrices, Vectors} + +class IndexedRowMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 4 + val n = 3 + val data = Seq( + (0L, Vectors.dense(0.0, 1.0, 2.0)), + (1L, Vectors.dense(3.0, 4.0, 5.0)), + (3L, Vectors.dense(9.0, 0.0, 1.0)) + ).map(x => IndexedRow(x._1, x._2)) + var indexedRows: RDD[IndexedRow] = _ + + override def beforeAll() { + super.beforeAll() + indexedRows = sc.parallelize(data, 2) + } + + test("size") { + val mat1 = new IndexedRowMatrix(indexedRows) + assert(mat1.numRows() === m) + assert(mat1.numCols() === n) + + val mat2 = new IndexedRowMatrix(indexedRows, 5, 0) + assert(mat2.numRows() === 5) + assert(mat2.numCols() === n) + } + + test("empty rows") { + val rows = sc.parallelize(Seq[IndexedRow](), 1) + val mat = new IndexedRowMatrix(rows) + intercept[RuntimeException] { + mat.numRows() + } + intercept[RuntimeException] { + mat.numCols() + } + } + + test("toBreeze") { + val mat = new IndexedRowMatrix(indexedRows) + val expected = BDM( + (0.0, 1.0, 2.0), + (3.0, 4.0, 5.0), + (0.0, 0.0, 0.0), + (9.0, 0.0, 1.0)) + assert(mat.toBreeze() === expected) + } + + test("toRowMatrix") { + val idxRowMat = new IndexedRowMatrix(indexedRows) + val rowMat = idxRowMat.toRowMatrix() + assert(rowMat.numCols() === n) + assert(rowMat.numRows() === 3, "should drop empty rows") + assert(rowMat.rows.collect().toSeq === data.map(_.vector).toSeq) + } + + test("multiply a local matrix") { + val A = new IndexedRowMatrix(indexedRows) + val B = Matrices.dense(3, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + val C = A.multiply(B) + val localA = A.toBreeze() + val localC = C.toBreeze() + val expected = localA * B.toBreeze.asInstanceOf[BDM[Double]] + assert(localC === expected) + } + + test("gram") { + val A = new IndexedRowMatrix(indexedRows) + val G = A.computeGramianMatrix() + val expected = BDM( + (90.0, 12.0, 24.0), + (12.0, 17.0, 22.0), + (24.0, 22.0, 30.0)) + assert(G.toBreeze === expected) + } + + test("svd") { + val A = new IndexedRowMatrix(indexedRows) + val svd = A.computeSVD(n, computeU = true) + assert(svd.U.isInstanceOf[IndexedRowMatrix]) + val localA = A.toBreeze() + val U = svd.U.toBreeze() + val s = svd.s.toBreeze.asInstanceOf[BDV[Double]] + val V = svd.V.toBreeze.asInstanceOf[BDM[Double]] + assert(closeToZero(U.t * U - BDM.eye[Double](n))) + assert(closeToZero(V.t * V - BDM.eye[Double](n))) + assert(closeToZero(U * brzDiag(s) * V.t - localA)) + } + + def closeToZero(G: BDM[Double]): Boolean = { + G.valuesIterator.map(math.abs).sum < 1e-6 + } +} + diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala new file mode 100644 index 0000000000000..71ee8e8a4f6fd --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -0,0 +1,173 @@ +/* + * 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.mllib.linalg.distributed + +import org.scalatest.FunSuite + +import breeze.linalg.{DenseVector => BDV, DenseMatrix => BDM, norm => brzNorm, svd => brzSvd} + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.linalg.{Matrices, Vectors, Vector} + +class RowMatrixSuite extends FunSuite with LocalSparkContext { + + val m = 4 + val n = 3 + val arr = Array(0.0, 3.0, 6.0, 9.0, 1.0, 4.0, 7.0, 0.0, 2.0, 5.0, 8.0, 1.0) + val denseData = Seq( + Vectors.dense(0.0, 1.0, 2.0), + Vectors.dense(3.0, 4.0, 5.0), + Vectors.dense(6.0, 7.0, 8.0), + Vectors.dense(9.0, 0.0, 1.0) + ) + val sparseData = Seq( + Vectors.sparse(3, Seq((1, 1.0), (2, 2.0))), + Vectors.sparse(3, Seq((0, 3.0), (1, 4.0), (2, 5.0))), + Vectors.sparse(3, Seq((0, 6.0), (1, 7.0), (2, 8.0))), + Vectors.sparse(3, Seq((0, 9.0), (2, 1.0))) + ) + + val principalComponents = BDM( + (0.0, 1.0, 0.0), + (math.sqrt(2.0) / 2.0, 0.0, math.sqrt(2.0) / 2.0), + (math.sqrt(2.0) / 2.0, 0.0, - math.sqrt(2.0) / 2.0)) + + var denseMat: RowMatrix = _ + var sparseMat: RowMatrix = _ + + override def beforeAll() { + super.beforeAll() + denseMat = new RowMatrix(sc.parallelize(denseData, 2)) + sparseMat = new RowMatrix(sc.parallelize(sparseData, 2)) + } + + test("size") { + assert(denseMat.numRows() === m) + assert(denseMat.numCols() === n) + assert(sparseMat.numRows() === m) + assert(sparseMat.numCols() === n) + } + + test("empty rows") { + val rows = sc.parallelize(Seq[Vector](), 1) + val emptyMat = new RowMatrix(rows) + intercept[RuntimeException] { + emptyMat.numCols() + } + intercept[RuntimeException] { + emptyMat.numRows() + } + } + + test("toBreeze") { + val expected = BDM( + (0.0, 1.0, 2.0), + (3.0, 4.0, 5.0), + (6.0, 7.0, 8.0), + (9.0, 0.0, 1.0)) + for (mat <- Seq(denseMat, sparseMat)) { + assert(mat.toBreeze() === expected) + } + } + + test("gram") { + val expected = + Matrices.dense(n, n, Array(126.0, 54.0, 72.0, 54.0, 66.0, 78.0, 72.0, 78.0, 94.0)) + for (mat <- Seq(denseMat, sparseMat)) { + val G = mat.computeGramianMatrix() + assert(G.toBreeze === expected.toBreeze) + } + } + + test("svd of a full-rank matrix") { + for (mat <- Seq(denseMat, sparseMat)) { + val localMat = mat.toBreeze() + val (localU, localSigma, localVt) = brzSvd(localMat) + val localV: BDM[Double] = localVt.t.toDenseMatrix + for (k <- 1 to n) { + val svd = mat.computeSVD(k, computeU = true) + val U = svd.U + val s = svd.s + val V = svd.V + assert(U.numRows() === m) + assert(U.numCols() === k) + assert(s.size === k) + assert(V.numRows === n) + assert(V.numCols === k) + assertColumnEqualUpToSign(U.toBreeze(), localU, k) + assertColumnEqualUpToSign(V.toBreeze.asInstanceOf[BDM[Double]], localV, k) + assert(closeToZero(s.toBreeze.asInstanceOf[BDV[Double]] - localSigma(0 until k))) + } + val svdWithoutU = mat.computeSVD(n) + assert(svdWithoutU.U === null) + } + } + + test("svd of a low-rank matrix") { + val rows = sc.parallelize(Array.fill(4)(Vectors.dense(1.0, 1.0)), 2) + val mat = new RowMatrix(rows, 4, 2) + val svd = mat.computeSVD(2, computeU = true) + assert(svd.s.size === 1, "should not return zero singular values") + assert(svd.U.numRows() === 4) + assert(svd.U.numCols() === 1) + assert(svd.V.numRows === 2) + assert(svd.V.numCols === 1) + } + + def closeToZero(G: BDM[Double]): Boolean = { + G.valuesIterator.map(math.abs).sum < 1e-6 + } + + def closeToZero(v: BDV[Double]): Boolean = { + brzNorm(v, 1.0) < 1e-6 + } + + def assertColumnEqualUpToSign(A: BDM[Double], B: BDM[Double], k: Int) { + assert(A.rows === B.rows) + for (j <- 0 until k) { + val aj = A(::, j) + val bj = B(::, j) + assert(closeToZero(aj - bj) || closeToZero(aj + bj), + s"The $j-th columns mismatch: $aj and $bj") + } + } + + test("pca") { + for (mat <- Seq(denseMat, sparseMat); k <- 1 to n) { + val pc = denseMat.computePrincipalComponents(k) + assert(pc.numRows === n) + assert(pc.numCols === k) + assertColumnEqualUpToSign(pc.toBreeze.asInstanceOf[BDM[Double]], principalComponents, k) + } + } + + test("multiply a local matrix") { + val B = Matrices.dense(n, 2, Array(0.0, 1.0, 2.0, 3.0, 4.0, 5.0)) + for (mat <- Seq(denseMat, sparseMat)) { + val AB = mat.multiply(B) + assert(AB.numRows() === m) + assert(AB.numCols() === 2) + assert(AB.rows.collect().toSeq === Seq( + Vectors.dense(5.0, 14.0), + Vectors.dense(14.0, 50.0), + Vectors.dense(23.0, 86.0), + Vectors.dense(2.0, 32.0) + )) + } + } +} From 87bd1f9ef7d547ee54a8a83214b45462e0751efb Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 9 Apr 2014 01:14:46 -0700 Subject: [PATCH 242/397] SPARK-1093: Annotate developer and experimental API's This patch marks some existing classes as private[spark] and adds two types of API annotations: - `EXPERIMENTAL API` = experimental user-facing module - `DEVELOPER API - UNSTABLE` = developer-facing API that might change There is some discussion of the different mechanisms for doing this here: https://issues.apache.org/jira/browse/SPARK-1081 I was pretty aggressive with marking things private. Keep in mind that if we want to open something up in the future we can, but we can never reduce visibility. A few notes here: - In the past we've been inconsistent with the visiblity of the X-RDD classes. This patch marks them private whenever there is an existing function in RDD that can directly creat them (e.g. CoalescedRDD and rdd.coalesce()). One trade-off here is users can't subclass them. - Noted that compression and serialization formats don't have to be wire compatible across versions. - Compression codecs and serialization formats are semi-private as users typically don't instantiate them directly. - Metrics sources are made private - user only interacts with them through Spark's reflection Author: Patrick Wendell Author: Andrew Or Closes #274 from pwendell/private-apis and squashes the following commits: 44179e4 [Patrick Wendell] Merge remote-tracking branch 'apache-github/master' into private-apis 042c803 [Patrick Wendell] spark.annotations -> spark.annotation bfe7b52 [Patrick Wendell] Adding experimental for approximate counts 8d0c873 [Patrick Wendell] Warning in SparkEnv 99b223a [Patrick Wendell] Cleaning up annotations e849f64 [Patrick Wendell] Merge pull request #2 from andrewor14/annotations 982a473 [Andrew Or] Generalize jQuery matching for non Spark-core API docs a01c076 [Patrick Wendell] Merge pull request #1 from andrewor14/annotations c1bcb41 [Andrew Or] DeveloperAPI -> DeveloperApi 0d48908 [Andrew Or] Comments and new lines (minor) f3954e0 [Andrew Or] Add identifier tags in comments to work around scaladocs bug 99192ef [Andrew Or] Dynamically add badges based on annotations 824011b [Andrew Or] Add support for injecting arbitrary JavaScript to API docs 037755c [Patrick Wendell] Some changes after working with andrew or f7d124f [Patrick Wendell] Small fixes c318b24 [Patrick Wendell] Use CSS styles e4c76b9 [Patrick Wendell] Logging f390b13 [Patrick Wendell] Better visibility for workaround constructors d6b0afd [Patrick Wendell] Small chang to existing constructor 403ba52 [Patrick Wendell] Style fix 870a7ba [Patrick Wendell] Work around for SI-8479 7fb13b2 [Patrick Wendell] Changes to UnionRDD and EmptyRDD 4a9e90c [Patrick Wendell] EXPERIMENTAL API --> EXPERIMENTAL c581dce [Patrick Wendell] Changes after building against Shark. 8452309 [Patrick Wendell] Style fixes 1ed27d2 [Patrick Wendell] Formatting and coloring of badges cd7a465 [Patrick Wendell] Code review feedback 2f706f1 [Patrick Wendell] Don't use floats 542a736 [Patrick Wendell] Small fixes cf23ec6 [Patrick Wendell] Marking GraphX as alpha d86818e [Patrick Wendell] Another naming change 5a76ed6 [Patrick Wendell] More visiblity clean-up 42c1f09 [Patrick Wendell] Using better labels 9d48cbf [Patrick Wendell] Initial pass --- .../scala/org/apache/spark/Aggregator.scala | 3 + .../scala/org/apache/spark/Dependency.scala | 11 +++ .../scala/org/apache/spark/FutureAction.scala | 7 ++ .../apache/spark/InterruptibleIterator.scala | 2 +- .../main/scala/org/apache/spark/Logging.scala | 7 ++ .../apache/spark/SerializableWritable.scala | 3 + .../scala/org/apache/spark/SparkContext.scala | 86 ++++++++++++++++--- .../scala/org/apache/spark/SparkEnv.scala | 8 +- .../scala/org/apache/spark/TaskContext.scala | 6 ++ .../org/apache/spark/TaskEndReason.scala | 31 +++++-- .../spark/annotation/AlphaComponent.java | 26 ++++++ .../apache/spark/annotation/DeveloperApi.java | 30 +++++++ .../apache/spark/annotation/Experimental.java | 31 +++++++ .../apache/spark/api/java/JavaDoubleRDD.scala | 19 +++- .../apache/spark/api/java/JavaPairRDD.scala | 11 ++- .../apache/spark/api/java/JavaRDDLike.scala | 9 +- .../spark/api/java/JavaSparkContext.scala | 2 +- .../org/apache/spark/api/java/JavaUtils.scala | 2 +- .../spark/broadcast/BroadcastFactory.scala | 7 +- .../spark/executor/ExecutorSource.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 20 ++++- .../apache/spark/io/CompressionCodec.scala | 19 ++++ .../spark/metrics/sink/ConsoleSink.scala | 2 +- .../apache/spark/metrics/sink/CsvSink.scala | 2 +- .../spark/metrics/sink/GraphiteSink.scala | 2 +- .../apache/spark/metrics/sink/JmxSink.scala | 2 +- .../spark/metrics/sink/MetricsServlet.scala | 2 +- .../org/apache/spark/metrics/sink/Sink.scala | 2 +- .../spark/metrics/source/JvmSource.scala | 2 +- .../apache/spark/metrics/source/Source.scala | 2 +- .../main/scala/org/apache/spark/package.scala | 9 ++ .../apache/spark/partial/BoundedDouble.scala | 6 +- .../apache/spark/partial/PartialResult.scala | 3 + .../apache/spark/rdd/AsyncRDDActions.scala | 3 + .../org/apache/spark/rdd/CoGroupedRDD.scala | 8 +- .../org/apache/spark/rdd/CoalescedRDD.scala | 4 +- .../apache/spark/rdd/DoubleRDDFunctions.scala | 13 ++- .../scala/org/apache/spark/rdd/EmptyRDD.scala | 4 +- .../org/apache/spark/rdd/HadoopRDD.scala | 6 ++ .../scala/org/apache/spark/rdd/JdbcRDD.scala | 2 +- .../org/apache/spark/rdd/NewHadoopRDD.scala | 6 ++ .../apache/spark/rdd/PairRDDFunctions.scala | 5 +- .../spark/rdd/PartitionPruningRDD.scala | 9 +- .../spark/rdd/PartitionwiseSampledRDD.scala | 2 +- .../scala/org/apache/spark/rdd/PipedRDD.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 29 ++++++- .../org/apache/spark/rdd/SampledRDD.scala | 2 +- .../org/apache/spark/rdd/ShuffledRDD.scala | 3 + .../scala/org/apache/spark/rdd/UnionRDD.scala | 2 + .../spark/rdd/ZippedPartitionsRDD.scala | 8 +- .../org/apache/spark/rdd/ZippedRDD.scala | 2 +- .../spark/scheduler/InputFormatInfo.scala | 3 + .../apache/spark/scheduler/JobLogger.scala | 4 +- .../apache/spark/scheduler/JobResult.scala | 10 ++- .../spark/scheduler/SparkListener.scala | 20 ++++- .../apache/spark/scheduler/SplitInfo.scala | 3 + .../apache/spark/scheduler/StageInfo.scala | 4 +- .../org/apache/spark/scheduler/TaskInfo.scala | 15 ++-- .../apache/spark/scheduler/TaskLocality.scala | 5 +- .../spark/serializer/JavaSerializer.scala | 7 ++ .../spark/serializer/KryoSerializer.scala | 4 + .../apache/spark/serializer/Serializer.scala | 15 +++- .../apache/spark/storage/BlockManager.scala | 8 +- .../apache/spark/storage/StorageUtils.scala | 2 + .../spark/util/BoundedPriorityQueue.scala | 2 +- .../apache/spark/util/CollectionsUtil.scala | 2 +- .../org/apache/spark/util/Distribution.scala | 4 +- .../org/apache/spark/util/FileLogger.scala | 2 +- .../org/apache/spark/util/MutablePair.scala | 4 + .../spark/util/TimeStampedHashSet.scala | 2 +- .../scala/org/apache/spark/util/Vector.scala | 1 + .../spark/util/collection/AppendOnlyMap.scala | 9 +- .../collection/ExternalAppendOnlyMap.scala | 6 +- .../spark/util/collection/OpenHashMap.scala | 5 +- .../spark/util/random/Pseudorandom.scala | 4 + .../spark/util/random/RandomSampler.scala | 8 ++ docs/_plugins/copy_api_dirs.rb | 24 +++--- docs/css/api-docs.css | 18 ++++ docs/js/api-docs.js | 26 ++++++ .../org/apache/spark/graphx/package.scala | 5 +- .../spark/sql/catalyst/expressions/Row.scala | 2 +- .../org/apache/spark/sql/SQLContext.scala | 9 +- .../org/apache/spark/sql/SchemaRDD.scala | 21 ++--- .../spark/tools}/StoragePerfTester.scala | 4 +- 84 files changed, 614 insertions(+), 130 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java create mode 100644 core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java create mode 100644 core/src/main/scala/org/apache/spark/annotation/Experimental.java create mode 100644 docs/css/api-docs.css create mode 100644 docs/js/api-docs.js rename {core/src/main/scala/org/apache/spark/storage => tools/src/main/scala/org/apache/spark/tools}/StoragePerfTester.scala (97%) diff --git a/core/src/main/scala/org/apache/spark/Aggregator.scala b/core/src/main/scala/org/apache/spark/Aggregator.scala index ceead59b79ed6..59fdf659c9e11 100644 --- a/core/src/main/scala/org/apache/spark/Aggregator.scala +++ b/core/src/main/scala/org/apache/spark/Aggregator.scala @@ -17,15 +17,18 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{AppendOnlyMap, ExternalAppendOnlyMap} /** + * :: DeveloperApi :: * A set of functions used to aggregate data. * * @param createCombiner function to create the initial value of the aggregation. * @param mergeValue function to merge a new value into the aggregation result. * @param mergeCombiners function to merge outputs from multiple mergeValue function. */ +@DeveloperApi case class Aggregator[K, V, C] ( createCombiner: V => C, mergeValue: (C, V) => C, diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala index 1cd629c15bd46..2c31cc20211ff 100644 --- a/core/src/main/scala/org/apache/spark/Dependency.scala +++ b/core/src/main/scala/org/apache/spark/Dependency.scala @@ -17,19 +17,24 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.serializer.Serializer /** + * :: DeveloperApi :: * Base class for dependencies. */ +@DeveloperApi abstract class Dependency[T](val rdd: RDD[T]) extends Serializable /** + * :: DeveloperApi :: * Base class for dependencies where each partition of the parent RDD is used by at most one * partition of the child RDD. Narrow dependencies allow for pipelined execution. */ +@DeveloperApi abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** * Get the parent partitions for a child partition. @@ -41,6 +46,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { /** + * :: DeveloperApi :: * Represents a dependency on the output of a shuffle stage. * @param rdd the parent RDD * @param partitioner partitioner used to partition the shuffle output @@ -48,6 +54,7 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) { * the default serializer, as specified by `spark.serializer` config option, will * be used. */ +@DeveloperApi class ShuffleDependency[K, V]( @transient rdd: RDD[_ <: Product2[K, V]], val partitioner: Partitioner, @@ -61,20 +68,24 @@ class ShuffleDependency[K, V]( /** + * :: DeveloperApi :: * Represents a one-to-one dependency between partitions of the parent and child RDDs. */ +@DeveloperApi class OneToOneDependency[T](rdd: RDD[T]) extends NarrowDependency[T](rdd) { override def getParents(partitionId: Int) = List(partitionId) } /** + * :: DeveloperApi :: * Represents a one-to-one dependency between ranges of partitions in the parent and child RDDs. * @param rdd the parent RDD * @param inStart the start of the range in the parent RDD * @param outStart the start of the range in the child RDD * @param length the length of the range */ +@DeveloperApi class RangeDependency[T](rdd: RDD[T], inStart: Int, outStart: Int, length: Int) extends NarrowDependency[T](rdd) { diff --git a/core/src/main/scala/org/apache/spark/FutureAction.scala b/core/src/main/scala/org/apache/spark/FutureAction.scala index 2eec09cd1c795..1e4dec86a0530 100644 --- a/core/src/main/scala/org/apache/spark/FutureAction.scala +++ b/core/src/main/scala/org/apache/spark/FutureAction.scala @@ -21,13 +21,16 @@ import scala.concurrent._ import scala.concurrent.duration.Duration import scala.util.Try +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.{JobFailed, JobSucceeded, JobWaiter} /** + * :: Experimental :: * A future for the result of an action to support cancellation. This is an extension of the * Scala Future interface to support cancellation. */ +@Experimental trait FutureAction[T] extends Future[T] { // Note that we redefine methods of the Future trait here explicitly so we can specify a different // documentation (with reference to the word "action"). @@ -84,9 +87,11 @@ trait FutureAction[T] extends Future[T] { /** + * :: Experimental :: * A [[FutureAction]] holding the result of an action that triggers a single job. Examples include * count, collect, reduce. */ +@Experimental class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: => T) extends FutureAction[T] { @@ -148,10 +153,12 @@ class SimpleFutureAction[T] private[spark](jobWaiter: JobWaiter[_], resultFunc: /** + * :: Experimental :: * A [[FutureAction]] for actions that could trigger multiple Spark jobs. Examples include take, * takeSample. Cancellation works by setting the cancelled flag to true and interrupting the * action thread if it is being blocked by a job. */ +@Experimental class ComplexFutureAction[T] extends FutureAction[T] { // Pointer to the thread that is executing the action. It is set when the action is run. diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index 9b1601d5b95fa..fd1802ba2f984 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -21,7 +21,7 @@ package org.apache.spark * An iterator that wraps around an existing iterator to provide task killing functionality. * It works by checking the interrupted flag in [[TaskContext]]. */ -class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) +private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { def hasNext: Boolean = !context.interrupted && delegate.hasNext diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 7423082e34f47..e5e15617acb10 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -21,11 +21,18 @@ import org.apache.log4j.{LogManager, PropertyConfigurator} import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. + * + * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. + * This will likely be changed or removed in future releases. */ +@DeveloperApi trait Logging { // Make the log field transient so that objects with Logging can // be serialized and used on another machine diff --git a/core/src/main/scala/org/apache/spark/SerializableWritable.scala b/core/src/main/scala/org/apache/spark/SerializableWritable.scala index dff665cae6cb6..e50b9ac2291f9 100644 --- a/core/src/main/scala/org/apache/spark/SerializableWritable.scala +++ b/core/src/main/scala/org/apache/spark/SerializableWritable.scala @@ -23,6 +23,9 @@ import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.ObjectWritable import org.apache.hadoop.io.Writable +import org.apache.spark.annotation.DeveloperApi + +@DeveloperApi class SerializableWritable[T <: Writable](@transient var t: T) extends Serializable { def value = t override def toString = t.toString diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d7124616d3bfb..f7750514ae13d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -34,6 +34,7 @@ import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat, Job => NewHad import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat => NewFileInputFormat} import org.apache.mesos.MesosNativeLibrary +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.{LocalSparkCluster, SparkHadoopUtil} import org.apache.spark.input.WholeTextFileInputFormat @@ -48,22 +49,35 @@ import org.apache.spark.ui.SparkUI import org.apache.spark.util.{ClosureCleaner, MetadataCleaner, MetadataCleanerType, TimeStampedWeakValueHashMap, Utils} /** + * :: DeveloperApi :: * Main entry point for Spark functionality. A SparkContext represents the connection to a Spark * cluster, and can be used to create RDDs, accumulators and broadcast variables on that cluster. * * @param config a Spark Config object describing the application configuration. Any settings in * this config overrides the default configs as well as system properties. - * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Can - * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] - * from a list of input files or InputFormats for the application. */ -class SparkContext( - config: SparkConf, - // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, - // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It - // contains a map from hostname to a list of input format splits on the host. - val preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) - extends Logging { + +@DeveloperApi +class SparkContext(config: SparkConf) extends Logging { + + // This is used only by YARN for now, but should be relevant to other cluster types (Mesos, + // etc) too. This is typically generated from InputFormatInfo.computePreferredLocations. It + // contains a map from hostname to a list of input format splits on the host. + private[spark] var preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map() + + /** + * :: DeveloperApi :: + * Alternative constructor for setting preferred locations where Spark will create executors. + * + * @param preferredNodeLocationData used in YARN mode to select nodes to launch containers on. Ca + * be generated using [[org.apache.spark.scheduler.InputFormatInfo.computePreferredLocations]] + * from a list of input files or InputFormats for the application. + */ + @DeveloperApi + def this(config: SparkConf, preferredNodeLocationData: Map[String, Set[SplitInfo]]) = { + this(config) + this.preferredNodeLocationData = preferredNodeLocationData + } /** * Alternative constructor that allows setting common Spark properties directly @@ -93,10 +107,45 @@ class SparkContext( environment: Map[String, String] = Map(), preferredNodeLocationData: Map[String, Set[SplitInfo]] = Map()) = { - this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment), - preferredNodeLocationData) + this(SparkContext.updatedConf(new SparkConf(), master, appName, sparkHome, jars, environment)) + this.preferredNodeLocationData = preferredNodeLocationData } + // NOTE: The below constructors could be consolidated using default arguments. Due to + // Scala bug SI-8479, however, this causes the compile step to fail when generating docs. + // Until we have a good workaround for that bug the constructors remain broken out. + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + */ + private[spark] def this(master: String, appName: String) = + this(master, appName, null, Nil, Map(), Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + */ + private[spark] def this(master: String, appName: String, sparkHome: String) = + this(master, appName, sparkHome, Nil, Map(), Map()) + + /** + * Alternative constructor that allows setting common Spark properties directly + * + * @param master Cluster URL to connect to (e.g. mesos://host:port, spark://host:port, local[4]). + * @param appName A name for your application, to display on the cluster web UI. + * @param sparkHome Location where Spark is installed on cluster nodes. + * @param jars Collection of JARs to send to the cluster. These can be paths on the local file + * system or HDFS, HTTP, HTTPS, or FTP URLs. + */ + private[spark] def this(master: String, appName: String, sparkHome: String, jars: Seq[String]) = + this(master, appName, sparkHome, jars, Map(), Map()) + private[spark] val conf = config.clone() /** @@ -189,7 +238,7 @@ class SparkContext( jars.foreach(addJar) } - def warnSparkMem(value: String): String = { + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") value @@ -653,6 +702,9 @@ class SparkContext( def union[T: ClassTag](first: RDD[T], rest: RDD[T]*): RDD[T] = new UnionRDD(this, Seq(first) ++ rest) + /** Get an RDD that has no partitions or elements. */ + def emptyRDD[T: ClassTag] = new EmptyRDD[T](this) + // Methods for creating shared variables /** @@ -716,6 +768,11 @@ class SparkContext( postEnvironmentUpdate() } + /** + * :: DeveloperApi :: + * Register a listener to receive up-calls from events that happen during execution. + */ + @DeveloperApi def addSparkListener(listener: SparkListener) { listenerBus.addListener(listener) } @@ -1021,8 +1078,10 @@ class SparkContext( } /** + * :: DeveloperApi :: * Run a job that can return approximate results. */ + @DeveloperApi def runApproximateJob[T, U, R]( rdd: RDD[T], func: (TaskContext, Iterator[T]) => U, @@ -1040,6 +1099,7 @@ class SparkContext( /** * Submit a job for execution and return a FutureJob holding the result. */ + @Experimental def submitJob[T, U, R]( rdd: RDD[T], processPartition: Iterator[T] => U, diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 9ea123f174b95..915315ed74436 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -25,6 +25,7 @@ import scala.util.Properties import akka.actor._ import com.google.common.collect.MapMaker +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.python.PythonWorkerFactory import org.apache.spark.broadcast.BroadcastManager import org.apache.spark.metrics.MetricsSystem @@ -35,13 +36,18 @@ import org.apache.spark.storage._ import org.apache.spark.util.{AkkaUtils, Utils} /** + * :: DeveloperApi :: * Holds all the runtime environment objects for a running Spark instance (either master or worker), * including the serializer, Akka actor system, block manager, map output tracker, etc. Currently * Spark code finds the SparkEnv through a thread-local variable, so each thread that accesses these * objects needs to have the right SparkEnv set. You can get the current environment with * SparkEnv.get (e.g. after creating a SparkContext) and set it with SparkEnv.set. + * + * NOTE: This is not intended for external use. This is exposed for Shark and may be made private + * in a future release. */ -class SparkEnv private[spark] ( +@DeveloperApi +class SparkEnv ( val executorId: String, val actorSystem: ActorSystem, val serializer: Serializer, diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index be53ca2968cfb..dc5a19ecd738e 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -19,8 +19,14 @@ package org.apache.spark import scala.collection.mutable.ArrayBuffer +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics +/** + * :: DeveloperApi :: + * Contextual information about a task which can be read or mutated during execution. + */ +@DeveloperApi class TaskContext( val stageId: Int, val partitionId: Int, diff --git a/core/src/main/scala/org/apache/spark/TaskEndReason.scala b/core/src/main/scala/org/apache/spark/TaskEndReason.scala index f1a753b6ab8a9..a3074916d13e7 100644 --- a/core/src/main/scala/org/apache/spark/TaskEndReason.scala +++ b/core/src/main/scala/org/apache/spark/TaskEndReason.scala @@ -17,29 +17,35 @@ package org.apache.spark +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId /** + * :: DeveloperApi :: * Various possible reasons why a task ended. The low-level TaskScheduler is supposed to retry * tasks several times for "ephemeral" failures, and only report back failures that require some * old stages to be resubmitted, such as shuffle map fetch failures. */ -private[spark] sealed trait TaskEndReason +@DeveloperApi +sealed trait TaskEndReason -private[spark] case object Success extends TaskEndReason +@DeveloperApi +case object Success extends TaskEndReason -private[spark] +@DeveloperApi case object Resubmitted extends TaskEndReason // Task was finished earlier but we've now lost it -private[spark] case class FetchFailed( +@DeveloperApi +case class FetchFailed( bmAddress: BlockManagerId, shuffleId: Int, mapId: Int, reduceId: Int) extends TaskEndReason -private[spark] case class ExceptionFailure( +@DeveloperApi +case class ExceptionFailure( className: String, description: String, stackTrace: Array[StackTraceElement], @@ -47,21 +53,28 @@ private[spark] case class ExceptionFailure( extends TaskEndReason /** + * :: DeveloperApi :: * The task finished successfully, but the result was lost from the executor's block manager before * it was fetched. */ -private[spark] case object TaskResultLost extends TaskEndReason +@DeveloperApi +case object TaskResultLost extends TaskEndReason -private[spark] case object TaskKilled extends TaskEndReason +@DeveloperApi +case object TaskKilled extends TaskEndReason /** + * :: DeveloperApi :: * The task failed because the executor that it was running on was lost. This may happen because * the task crashed the JVM. */ -private[spark] case object ExecutorLostFailure extends TaskEndReason +@DeveloperApi +case object ExecutorLostFailure extends TaskEndReason /** + * :: DeveloperApi :: * We don't know why the task ended -- for example, because of a ClassNotFound exception when * deserializing the task result. */ -private[spark] case object UnknownReason extends TaskEndReason +@DeveloperApi +case object UnknownReason extends TaskEndReason diff --git a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java new file mode 100644 index 0000000000000..af01fb7cfbd04 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java @@ -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.annotation; + +import java.lang.annotation.*; + +/** A new component of Spark which may have unstable API's. */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface AlphaComponent {} diff --git a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java new file mode 100644 index 0000000000000..5d546e7a63985 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java @@ -0,0 +1,30 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.annotation; + +import java.lang.annotation.*; + +/** + * A lower-level, unstable API intended for developers. + * + * Developer API's might change or be removed in minor versions of Spark. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface DeveloperApi {} diff --git a/core/src/main/scala/org/apache/spark/annotation/Experimental.java b/core/src/main/scala/org/apache/spark/annotation/Experimental.java new file mode 100644 index 0000000000000..306b1418d8d0a --- /dev/null +++ b/core/src/main/scala/org/apache/spark/annotation/Experimental.java @@ -0,0 +1,31 @@ +/* + * 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.annotation; + +import java.lang.annotation.*; + +/** + * An experimental user-facing API. + * + * Experimental API's might change or be removed in minor versions of Spark, or be adopted as + * first-class Spark API's. + */ +@Retention(RetentionPolicy.RUNTIME) +@Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, + ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) +public @interface Experimental {} diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index f816bb43a5b44..537f410b0ca26 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.Partitioner import org.apache.spark.SparkContext.doubleRDDToDoubleRDDFunctions +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD @@ -184,14 +185,26 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja def meanApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.meanApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.meanApprox(timeout) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: JDouble): PartialResult[BoundedDouble] = srdd.sumApprox(timeout, confidence) - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long): PartialResult[BoundedDouble] = srdd.sumApprox(timeout) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index e6c5d85917678..a41c7dbda2afc 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -27,11 +27,12 @@ import com.google.common.base.Optional import org.apache.hadoop.conf.Configuration import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.mapred.{JobConf, OutputFormat} -import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat, Job} +import org.apache.hadoop.mapreduce.{OutputFormat => NewOutputFormat} import org.apache.spark.{HashPartitioner, Partitioner} import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext.rddToPairRDDFunctions +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, PairFunction} import org.apache.spark.partial.{BoundedDouble, PartialResult} @@ -201,16 +202,20 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) def countByKey(): java.util.Map[K, Long] = mapAsJavaMap(rdd.countByKey()) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long): PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout).map(mapAsJavaMap) /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[java.util.Map[K, BoundedDouble]] = rdd.countByKeyApprox(timeout, confidence).map(mapAsJavaMap) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index ae577b500ccb4..725c423a53e35 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -27,6 +27,7 @@ import com.google.common.base.Optional import org.apache.hadoop.io.compress.CompressionCodec import org.apache.spark.{Partition, SparkContext, TaskContext} +import org.apache.spark.annotation.Experimental import org.apache.spark.api.java.JavaPairRDD._ import org.apache.spark.api.java.JavaSparkContext.fakeClassTag import org.apache.spark.api.java.function.{Function => JFunction, Function2 => JFunction2, _} @@ -343,16 +344,20 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def count(): Long = rdd.count() /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long, confidence: Double): PartialResult[BoundedDouble] = rdd.countApprox(timeout, confidence) /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long): PartialResult[BoundedDouble] = rdd.countApprox(timeout) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index a2855d4db1d2e..1e8242a2cbbce 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -89,7 +89,7 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork */ def this(master: String, appName: String, sparkHome: String, jars: Array[String], environment: JMap[String, String]) = - this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment)) + this(new SparkContext(master, appName, sparkHome, jars.toSeq, environment, Map())) private[spark] val env = sc.env diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala index ecbf18849ad48..22810cb1c662d 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaUtils.scala @@ -19,7 +19,7 @@ package org.apache.spark.api.java import com.google.common.base.Optional -object JavaUtils { +private[spark] object JavaUtils { def optionToOptional[T](option: Option[T]): Optional[T] = option match { case Some(value) => Optional.of(value) diff --git a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala index c7f7c59cfb449..8c8ce9b1691ac 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/BroadcastFactory.scala @@ -16,16 +16,19 @@ */ package org.apache.spark.broadcast -import org.apache.spark.SecurityManager +import org.apache.spark.SecurityManager import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi /** - * An interface for all the broadcast implementations in Spark (to allow + * :: DeveloperApi :: + * An interface for all the broadcast implementations in Spark (to allow * multiple broadcast implementations). SparkContext uses a user-specified * BroadcastFactory implementation to instantiate a particular broadcast for the * entire Spark job. */ +@DeveloperApi trait BroadcastFactory { def initialize(isDriver: Boolean, conf: SparkConf, securityMgr: SecurityManager): Unit def newBroadcast[T](value: T, isLocal: Boolean, id: Long): Broadcast[T] diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala index 127f5e90f3e1a..0ed52cfe9df61 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorSource.scala @@ -24,7 +24,7 @@ import org.apache.hadoop.fs.FileSystem import org.apache.spark.metrics.source.Source -class ExecutorSource(val executor: Executor, executorId: String) extends Source { +private[spark] class ExecutorSource(val executor: Executor, executorId: String) extends Source { private def fileStats(scheme: String) : Option[FileSystem.Statistics] = FileSystem.getAllStatistics().filter(s => s.getScheme.equals(scheme)).headOption diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index 88625e79a5c68..e4f02a4be0b97 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -17,8 +17,14 @@ package org.apache.spark.executor +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.{BlockId, BlockStatus} +/** + * :: DeveloperApi :: + * Metrics tracked during the execution of a task. + */ +@DeveloperApi class TaskMetrics extends Serializable { /** * Host's name the task runs on @@ -77,11 +83,16 @@ class TaskMetrics extends Serializable { var updatedBlocks: Option[Seq[(BlockId, BlockStatus)]] = None } -object TaskMetrics { - private[spark] def empty(): TaskMetrics = new TaskMetrics +private[spark] object TaskMetrics { + def empty(): TaskMetrics = new TaskMetrics } +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data read in a given task. + */ +@DeveloperApi class ShuffleReadMetrics extends Serializable { /** * Absolute time when this task finished reading shuffle data @@ -116,6 +127,11 @@ class ShuffleReadMetrics extends Serializable { var remoteBytesRead: Long = _ } +/** + * :: DeveloperApi :: + * Metrics pertaining to shuffle data written in a given task. + */ +@DeveloperApi class ShuffleWriteMetrics extends Serializable { /** * Number of bytes written for the shuffle by this task diff --git a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala index 059e58824c39b..e1a5ee316bb69 100644 --- a/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala +++ b/core/src/main/scala/org/apache/spark/io/CompressionCodec.scala @@ -23,11 +23,18 @@ import com.ning.compress.lzf.{LZFInputStream, LZFOutputStream} import org.xerial.snappy.{SnappyInputStream, SnappyOutputStream} import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * CompressionCodec allows the customization of choosing different compression implementations * to be used in block storage. + * + * Note: The wire protocol for a codec is not guaranteed compatible across versions of Spark. + * This is intended for use as an internal compression utility within a single + * Spark application. */ +@DeveloperApi trait CompressionCodec { def compressedOutputStream(s: OutputStream): OutputStream @@ -52,8 +59,14 @@ private[spark] object CompressionCodec { /** + * :: DeveloperApi :: * LZF implementation of [[org.apache.spark.io.CompressionCodec]]. + * + * Note: The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. */ +@DeveloperApi class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { @@ -65,9 +78,15 @@ class LZFCompressionCodec(conf: SparkConf) extends CompressionCodec { /** + * :: DeveloperApi :: * Snappy implementation of [[org.apache.spark.io.CompressionCodec]]. * Block size can be configured by spark.io.compression.snappy.block.size. + * + * Note: The wire protocol for this codec is not guaranteed to be compatible across versions + * of Spark. This is intended for use as an internal compression utility within a single Spark + * application. */ +@DeveloperApi class SnappyCompressionCodec(conf: SparkConf) extends CompressionCodec { override def compressedOutputStream(s: OutputStream): OutputStream = { diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala index 64eac73605388..05852f1f98993 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/ConsoleSink.scala @@ -25,7 +25,7 @@ import com.codahale.metrics.{ConsoleReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class ConsoleSink(val property: Properties, val registry: MetricRegistry, +private[spark] class ConsoleSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CONSOLE_DEFAULT_PERIOD = 10 val CONSOLE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 544848d4150b6..42c1200926fea 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -26,7 +26,7 @@ import com.codahale.metrics.{CsvReporter, MetricRegistry} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class CsvSink(val property: Properties, val registry: MetricRegistry, +private[spark] class CsvSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val CSV_KEY_PERIOD = "period" val CSV_KEY_UNIT = "unit" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala index 7f0a2fd16fa99..aeb4ad44a0647 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/GraphiteSink.scala @@ -27,7 +27,7 @@ import com.codahale.metrics.graphite.{Graphite, GraphiteReporter} import org.apache.spark.SecurityManager import org.apache.spark.metrics.MetricsSystem -class GraphiteSink(val property: Properties, val registry: MetricRegistry, +private[spark] class GraphiteSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val GRAPHITE_DEFAULT_PERIOD = 10 val GRAPHITE_DEFAULT_UNIT = "SECONDS" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala index 3b5edd5c376f0..ed27234b4e760 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/JmxSink.scala @@ -22,7 +22,7 @@ import java.util.Properties import com.codahale.metrics.{JmxReporter, MetricRegistry} import org.apache.spark.SecurityManager -class JmxSink(val property: Properties, val registry: MetricRegistry, +private[spark] class JmxSink(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val reporter: JmxReporter = JmxReporter.forRegistry(registry).build() diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala index 854b52c510e3d..571539ba5e467 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/MetricsServlet.scala @@ -30,7 +30,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.SecurityManager import org.apache.spark.ui.JettyUtils._ -class MetricsServlet(val property: Properties, val registry: MetricRegistry, +private[spark] class MetricsServlet(val property: Properties, val registry: MetricRegistry, securityMgr: SecurityManager) extends Sink { val SERVLET_KEY_PATH = "path" val SERVLET_KEY_SAMPLE = "sample" diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala index 3a739aa563eae..6f2b5a06027ea 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/Sink.scala @@ -17,7 +17,7 @@ package org.apache.spark.metrics.sink -trait Sink { +private[spark] trait Sink { def start: Unit def stop: Unit } diff --git a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala index 75cb2b8973aa1..f865f9648a91e 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/JvmSource.scala @@ -20,7 +20,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry import com.codahale.metrics.jvm.{GarbageCollectorMetricSet, MemoryUsageGaugeSet} -class JvmSource extends Source { +private[spark] class JvmSource extends Source { val sourceName = "jvm" val metricRegistry = new MetricRegistry() diff --git a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala index 3fee55cc6dcd5..1dda2cd83b2a9 100644 --- a/core/src/main/scala/org/apache/spark/metrics/source/Source.scala +++ b/core/src/main/scala/org/apache/spark/metrics/source/Source.scala @@ -19,7 +19,7 @@ package org.apache.spark.metrics.source import com.codahale.metrics.MetricRegistry -trait Source { +private[spark] trait Source { def sourceName: String def metricRegistry: MetricRegistry } diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 2625a7f6a575a..59bbb1171f239 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -32,7 +32,16 @@ package org.apache * * Java programmers should reference the [[spark.api.java]] package * for Spark programming APIs in Java. + * + * Classes and methods marked with + * Experimental are user-facing features which have not been officially adopted by the + * Spark project. These are subject to change or removal in minor releases. + * + * Classes and methods marked with + * Developer API are intended for advanced users want to extend Spark through lower + * level interfaces. These are subject to changes or removal in minor releases. */ + package object spark { // For package docs only } diff --git a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala index 5f4450859cc9b..aed0353344427 100644 --- a/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala +++ b/core/src/main/scala/org/apache/spark/partial/BoundedDouble.scala @@ -17,9 +17,13 @@ package org.apache.spark.partial +import org.apache.spark.annotation.Experimental + /** - * A Double with error bars on it. + * :: Experimental :: + * A Double value with error bars and associated confidence. */ +@Experimental class BoundedDouble(val mean: Double, val confidence: Double, val low: Double, val high: Double) { override def toString(): String = "[%.3f, %.3f]".format(low, high) } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index 812368e04ac0d..eade07fbcbe37 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -17,6 +17,9 @@ package org.apache.spark.partial +import org.apache.spark.annotation.Experimental + +@Experimental class PartialResult[R](initialVal: R, isFinal: Boolean) { private var finalValue: Option[R] = if (isFinal) Some(initialVal) else None private var failure: Option[Exception] = None diff --git a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala index d1c74a5063510..aed951a40b40c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/AsyncRDDActions.scala @@ -24,11 +24,14 @@ import scala.concurrent.ExecutionContext.Implicits.global import scala.reflect.ClassTag import org.apache.spark.{ComplexFutureAction, FutureAction, Logging} +import org.apache.spark.annotation.Experimental /** + * :: Experimental :: * A set of asynchronous RDD actions available through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ +@Experimental class AsyncRDDActions[T: ClassTag](self: RDD[T]) extends Serializable with Logging { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index 9aa454a5c8b88..c6e79557f08a1 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.collection.{ExternalAppendOnlyMap, AppendOnlyMap} import org.apache.spark.serializer.Serializer @@ -51,12 +52,17 @@ private[spark] class CoGroupPartition(idx: Int, val deps: Array[CoGroupSplitDep] } /** + * :: DeveloperApi :: * A RDD that cogroups its parents. For each key k in parent RDDs, the resulting RDD contains a * tuple with the list of values for that key. * + * Note: This is an internal API. We recommend users use RDD.coGroup(...) instead of + * instantiating this directly. + * @param rdds parent RDDs. - * @param part partitioner used to partition the shuffle output. + * @param part partitioner used to partition the shuffle output */ +@DeveloperApi class CoGroupedRDD[K](@transient var rdds: Seq[RDD[_ <: Product2[K, _]]], part: Partitioner) extends RDD[(K, Seq[Seq[_]])](rdds.head.context, Nil) { diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 4e82b51313bf0..44401a663440c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -32,7 +32,7 @@ import org.apache.spark._ * @param parentsIndices list of indices in the parent that have been coalesced into this partition * @param preferredLocation the preferred location for this partition */ -case class CoalescedRDDPartition( +private[spark] case class CoalescedRDDPartition( index: Int, @transient rdd: RDD[_], parentsIndices: Array[Int], @@ -70,7 +70,7 @@ case class CoalescedRDDPartition( * @param maxPartitions number of desired partitions in the coalesced RDD * @param balanceSlack used to trade-off balance and locality. 1.0 is all locality, 0 is all balance */ -class CoalescedRDD[T: ClassTag]( +private[spark] class CoalescedRDD[T: ClassTag]( @transient var prev: RDD[T], maxPartitions: Int, balanceSlack: Double = 0.10) diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index a7b6b3b5146ce..2306c9736b334 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.rdd +import org.apache.spark.annotation.Experimental import org.apache.spark.{TaskContext, Logging} import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.MeanEvaluator @@ -63,14 +64,22 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { */ def sampleVariance(): Double = stats().sampleVariance - /** (Experimental) Approximate operation to return the mean within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the mean within a timeout. + */ + @Experimental def meanApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new MeanEvaluator(self.partitions.size, confidence) self.context.runApproximateJob(self, processPartition, evaluator, timeout) } - /** (Experimental) Approximate operation to return the sum within a timeout. */ + /** + * :: Experimental :: + * Approximate operation to return the sum within a timeout. + */ + @Experimental def sumApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val processPartition = (ctx: TaskContext, ns: Iterator[Double]) => StatCounter(ns) val evaluator = new SumEvaluator(self.partitions.size, confidence) diff --git a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala index a84e5f9fd8ef8..a2d7e344cf1b2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/EmptyRDD.scala @@ -22,9 +22,9 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, SparkContext, TaskContext} /** - * An RDD that is empty, i.e. has no element in it. + * An RDD that has no partitions and no elements. */ -class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { +private[spark] class EmptyRDD[T: ClassTag](sc: SparkContext) extends RDD[T](sc, Nil) { override def getPartitions: Array[Partition] = Array.empty diff --git a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala index 3af008bd72378..6811e1abb8b70 100644 --- a/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/HadoopRDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TaskID import org.apache.hadoop.util.ReflectionUtils import org.apache.spark._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.NextIterator @@ -70,9 +71,13 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp } /** + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the older MapReduce API (`org.apache.hadoop.mapred`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.hadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param broadcastedConf A general Hadoop Configuration, or a subclass of it. If the enclosed * variabe references an instance of JobConf, then that JobConf will be used for the Hadoop job. @@ -84,6 +89,7 @@ private[spark] class HadoopPartition(rddId: Int, idx: Int, @transient s: InputSp * @param valueClass Class of the value associated with the inputFormatClass. * @param minSplits Minimum number of Hadoop Splits (HadoopRDD partitions) to generate. */ +@DeveloperApi class HadoopRDD[K, V]( sc: SparkContext, broadcastedConf: Broadcast[SerializableWritable[Configuration]], diff --git a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala index 1b503743ac117..a76a070b5b863 100644 --- a/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/JdbcRDD.scala @@ -27,7 +27,7 @@ import org.apache.spark.util.NextIterator private[spark] class JdbcPartition(idx: Int, val lower: Long, val upper: Long) extends Partition { override def index = idx } - +// TODO: Expose a jdbcRDD function in SparkContext and mark this as semi-private /** * An RDD that executes an SQL query on a JDBC connection and reads results. * For usage example, see test case JdbcRDDSuite. diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 461a749eac48b..2d8dfa5a1645a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} +import org.apache.spark.annotation.DeveloperApi private[spark] class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) @@ -36,15 +37,20 @@ class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputS } /** + * :: DeveloperApi :: * An RDD that provides core functionality for reading data stored in Hadoop (e.g., files in HDFS, * sources in HBase, or S3), using the new MapReduce API (`org.apache.hadoop.mapreduce`). * + * Note: Instantiating this class directly is not recommended, please use + * [[org.apache.spark.SparkContext.newAPIHadoopRDD()]] + * * @param sc The SparkContext to associate the RDD with. * @param inputFormatClass Storage format of the data to be read. * @param keyClass Class of the key associated with the inputFormatClass. * @param valueClass Class of the value associated with the inputFormatClass. * @param conf The Hadoop configuration. */ +@DeveloperApi class NewHadoopRDD[K, V]( sc : SparkContext, inputFormatClass: Class[_ <: InputFormat[K, V]], diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index a92a84b5342d1..343e4325c0ef0 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -39,6 +39,7 @@ RecordWriter => NewRecordWriter, SparkHadoopMapReduceUtil} import org.apache.hadoop.mapreduce.lib.output.{FileOutputFormat => NewFileOutputFormat} import org.apache.spark._ +import org.apache.spark.annotation.Experimental import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.SparkHadoopWriter import org.apache.spark.Partitioner.defaultPartitioner @@ -201,9 +202,11 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def countByKey(): Map[K, Long] = self.map(_._1).countByValue() /** - * (Experimental) Approximate version of countByKey that can return a partial result if it does + * :: Experimental :: + * Approximate version of countByKey that can return a partial result if it does * not finish within a timeout. */ + @Experimental def countByKeyApprox(timeout: Long, confidence: Double = 0.95) : PartialResult[Map[K, BoundedDouble]] = { self.map(_._1).countByValueApprox(timeout, confidence) diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala index b0440ca7f32cf..f781a8d776f2a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionPruningRDD.scala @@ -20,8 +20,10 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{NarrowDependency, Partition, TaskContext} +import org.apache.spark.annotation.DeveloperApi -class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends Partition { +private[spark] class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) + extends Partition { override val index = idx } @@ -30,7 +32,7 @@ class PartitionPruningRDDPartition(idx: Int, val parentSplit: Partition) extends * Represents a dependency between the PartitionPruningRDD and its parent. In this * case, the child RDD contains a subset of partitions of the parents'. */ -class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) +private[spark] class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boolean) extends NarrowDependency[T](rdd) { @transient @@ -45,11 +47,13 @@ class PruneDependency[T](rdd: RDD[T], @transient partitionFilterFunc: Int => Boo /** + * :: DeveloperApi :: * A RDD used to prune RDD partitions/partitions so we can avoid launching tasks on * all partitions. An example use case: If we know the RDD is partitioned by range, * and the execution DAG has a filter on the key, we can avoid launching tasks * on partitions that don't have the range covering the key. */ +@DeveloperApi class PartitionPruningRDD[T: ClassTag]( @transient prev: RDD[T], @transient partitionFilterFunc: Int => Boolean) @@ -63,6 +67,7 @@ class PartitionPruningRDD[T: ClassTag]( } +@DeveloperApi object PartitionPruningRDD { /** diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index ce4c0d382baab..b4e3bb5d75e17 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -42,7 +42,7 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * @tparam T input RDD item type * @tparam U sampled RDD item type */ -class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( +private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], @transient seed: Long = System.nanoTime) diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index 41ae0fec823e7..e441d4a40ccd2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -37,7 +37,7 @@ import org.apache.spark.util.Utils * An RDD that pipes the contents of each parent partition through an external command * (printing them one per line) and returns the output as a collection of strings. */ -class PipedRDD[T: ClassTag]( +private[spark] class PipedRDD[T: ClassTag]( prev: RDD[T], command: Seq[String], envVars: Map[String, String], diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 74fa2a4fcd401..3437b2cac19c2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -35,6 +35,7 @@ import org.apache.hadoop.mapred.TextOutputFormat import org.apache.spark._ import org.apache.spark.Partitioner._ import org.apache.spark.SparkContext._ +import org.apache.spark.annotation.{DeveloperApi, Experimental} import org.apache.spark.api.java.JavaRDD import org.apache.spark.partial.BoundedDouble import org.apache.spark.partial.CountEvaluator @@ -86,22 +87,34 @@ abstract class RDD[T: ClassTag]( // Methods that should be implemented by subclasses of RDD // ======================================================================= - /** Implemented by subclasses to compute a given partition. */ + /** + * :: DeveloperApi :: + * Implemented by subclasses to compute a given partition. + */ + @DeveloperApi def compute(split: Partition, context: TaskContext): Iterator[T] /** + * :: DeveloperApi :: * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ + @DeveloperApi protected def getPartitions: Array[Partition] /** + * :: DeveloperApi :: * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ + @DeveloperApi protected def getDependencies: Seq[Dependency[_]] = deps - /** Optionally overridden by subclasses to specify placement preferences. */ + /** + * :: DeveloperApi :: + * Optionally overridden by subclasses to specify placement preferences. + */ + @DeveloperApi protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ @@ -518,9 +531,11 @@ abstract class RDD[T: ClassTag]( } /** + * :: DeveloperApi :: * Return a new RDD by applying a function to each partition of this RDD. This is a variant of * mapPartitions that also passes the TaskContext into the closure. */ + @DeveloperApi def mapPartitionsWithContext[U: ClassTag]( f: (TaskContext, Iterator[T]) => Iterator[U], preservesPartitioning: Boolean = false): RDD[U] = { @@ -792,9 +807,11 @@ abstract class RDD[T: ClassTag]( def count(): Long = sc.runJob(this, Utils.getIteratorSize _).sum /** - * (Experimental) Approximate version of count() that returns a potentially incomplete result + * :: Experimental :: + * Approximate version of count() that returns a potentially incomplete result * within a timeout, even if not all tasks have finished. */ + @Experimental def countApprox(timeout: Long, confidence: Double = 0.95): PartialResult[BoundedDouble] = { val countElements: (TaskContext, Iterator[T]) => Long = { (ctx, iter) => var result = 0L @@ -838,8 +855,10 @@ abstract class RDD[T: ClassTag]( } /** - * (Experimental) Approximate version of countByValue(). + * :: Experimental :: + * Approximate version of countByValue(). */ + @Experimental def countByValueApprox( timeout: Long, confidence: Double = 0.95 @@ -860,6 +879,7 @@ abstract class RDD[T: ClassTag]( } /** + * :: Experimental :: * Return approximate number of distinct elements in the RDD. * * The accuracy of approximation can be controlled through the relative standard deviation @@ -867,6 +887,7 @@ abstract class RDD[T: ClassTag]( * more accurate counts but increase the memory footprint and vise versa. The default value of * relativeSD is 0.05. */ + @Experimental def countApproxDistinct(relativeSD: Double = 0.05): Long = { val zeroCounter = new SerializableHyperLogLog(new HyperLogLog(relativeSD)) aggregate(zeroCounter)(_.add(_), _.merge(_)).value.cardinality() diff --git a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala index 4ceea557f569c..b097c30f8c231 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SampledRDD.scala @@ -33,7 +33,7 @@ class SampledRDDPartition(val prev: Partition, val seed: Int) extends Partition } @deprecated("Replaced by PartitionwiseSampledRDD", "1.0.0") -class SampledRDD[T: ClassTag]( +private[spark] class SampledRDD[T: ClassTag]( prev: RDD[T], withReplacement: Boolean, frac: Double, diff --git a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala index 02660ea6a45c5..802b0bdfb2d59 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ShuffledRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, Partitioner, ShuffleDependency, SparkEnv, TaskContext} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { @@ -28,12 +29,14 @@ private[spark] class ShuffledRDDPartition(val idx: Int) extends Partition { } /** + * :: DeveloperApi :: * The resulting RDD from a shuffle (e.g. repartitioning of data). * @param prev the parent RDD. * @param part the partitioner used to partition the RDD * @tparam K the key class. * @tparam V the value class. */ +@DeveloperApi class ShuffledRDD[K, V, P <: Product2[K, V] : ClassTag]( @transient var prev: RDD[P], part: Partitioner) diff --git a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala index a447030752096..21c6e07d69f90 100644 --- a/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/UnionRDD.scala @@ -23,6 +23,7 @@ import scala.collection.mutable.ArrayBuffer import scala.reflect.ClassTag import org.apache.spark.{Dependency, Partition, RangeDependency, SparkContext, TaskContext} +import org.apache.spark.annotation.DeveloperApi private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitIndex: Int) extends Partition { @@ -43,6 +44,7 @@ private[spark] class UnionPartition[T: ClassTag](idx: Int, rdd: RDD[T], splitInd } } +@DeveloperApi class UnionRDD[T: ClassTag]( sc: SparkContext, @transient var rdds: Seq[RDD[T]]) diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala index b56643444aa40..f3d30f6c9b32f 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedPartitionsRDD.scala @@ -41,7 +41,7 @@ private[spark] class ZippedPartitionsPartition( } } -abstract class ZippedPartitionsBaseRDD[V: ClassTag]( +private[spark] abstract class ZippedPartitionsBaseRDD[V: ClassTag]( sc: SparkContext, var rdds: Seq[RDD[_]], preservesPartitioning: Boolean = false) @@ -74,7 +74,7 @@ abstract class ZippedPartitionsBaseRDD[V: ClassTag]( } } -class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( +private[spark] class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B]) => Iterator[V], var rdd1: RDD[A], @@ -94,7 +94,7 @@ class ZippedPartitionsRDD2[A: ClassTag, B: ClassTag, V: ClassTag]( } } -class ZippedPartitionsRDD3 +private[spark] class ZippedPartitionsRDD3 [A: ClassTag, B: ClassTag, C: ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C]) => Iterator[V], @@ -119,7 +119,7 @@ class ZippedPartitionsRDD3 } } -class ZippedPartitionsRDD4 +private[spark] class ZippedPartitionsRDD4 [A: ClassTag, B: ClassTag, C: ClassTag, D:ClassTag, V: ClassTag]( sc: SparkContext, f: (Iterator[A], Iterator[B], Iterator[C], Iterator[D]) => Iterator[V], diff --git a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala index 2119e76f0e032..b8110ffc42f2d 100644 --- a/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/ZippedRDD.scala @@ -44,7 +44,7 @@ private[spark] class ZippedPartition[T: ClassTag, U: ClassTag]( } } -class ZippedRDD[T: ClassTag, U: ClassTag]( +private[spark] class ZippedRDD[T: ClassTag, U: ClassTag]( sc: SparkContext, var rdd1: RDD[T], var rdd2: RDD[U]) diff --git a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala index b3f2cb346f7da..bac37bfdaa23f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/InputFormatInfo.scala @@ -27,11 +27,14 @@ import org.apache.hadoop.mapreduce.Job import org.apache.hadoop.util.ReflectionUtils import org.apache.spark.Logging +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.deploy.SparkHadoopUtil /** + * :: DeveloperApi :: * Parses and holds information about inputFormat (and files) specified as a parameter. */ +@DeveloperApi class InputFormatInfo(val configuration: Configuration, val inputFormatClazz: Class[_], val path: String) extends Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 7c5053998f1d6..713aebfa3ce00 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -25,9 +25,11 @@ import java.util.concurrent.LinkedBlockingQueue import scala.collection.mutable.HashMap import org.apache.spark._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics /** + * :: DeveloperApi :: * A logger class to record runtime information for jobs in Spark. This class outputs one log file * for each Spark job, containing tasks start/stop and shuffle information. JobLogger is a subclass * of SparkListener, use addSparkListener to add JobLogger to a SparkContext after the SparkContext @@ -38,7 +40,7 @@ import org.apache.spark.executor.TaskMetrics * to log application information as SparkListenerEvents. To enable this functionality, set * spark.eventLog.enabled to true. */ - +@DeveloperApi @deprecated("Log application information by setting spark.eventLog.enabled.", "1.0.0") class JobLogger(val user: String, val logDirName: String) extends SparkListener with Logging { diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala index 047bd27056120..4cd6cbe189aab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobResult.scala @@ -17,11 +17,17 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A result of a job in the DAGScheduler. */ -private[spark] sealed trait JobResult +@DeveloperApi +sealed trait JobResult -private[spark] case object JobSucceeded extends JobResult +@DeveloperApi +case object JobSucceeded extends JobResult +@DeveloperApi private[spark] case class JobFailed(exception: Exception) extends JobResult diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index d42e67742a4f7..ced20350d5356 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -23,21 +23,28 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.{Logging, TaskEndReason} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.executor.TaskMetrics import org.apache.spark.storage.BlockManagerId import org.apache.spark.util.{Distribution, Utils} +@DeveloperApi sealed trait SparkListenerEvent +@DeveloperApi case class SparkListenerStageSubmitted(stageInfo: StageInfo, properties: Properties = null) extends SparkListenerEvent +@DeveloperApi case class SparkListenerStageCompleted(stageInfo: StageInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskStart(stageId: Int, taskInfo: TaskInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskGettingResult(taskInfo: TaskInfo) extends SparkListenerEvent +@DeveloperApi case class SparkListenerTaskEnd( stageId: Int, taskType: String, @@ -46,20 +53,26 @@ case class SparkListenerTaskEnd( taskMetrics: TaskMetrics) extends SparkListenerEvent +@DeveloperApi case class SparkListenerJobStart(jobId: Int, stageIds: Seq[Int], properties: Properties = null) extends SparkListenerEvent +@DeveloperApi case class SparkListenerJobEnd(jobId: Int, jobResult: JobResult) extends SparkListenerEvent +@DeveloperApi case class SparkListenerEnvironmentUpdate(environmentDetails: Map[String, Seq[(String, String)]]) extends SparkListenerEvent +@DeveloperApi case class SparkListenerBlockManagerAdded(blockManagerId: BlockManagerId, maxMem: Long) extends SparkListenerEvent +@DeveloperApi case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) extends SparkListenerEvent +@DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ @@ -67,8 +80,11 @@ private[spark] case object SparkListenerShutdown extends SparkListenerEvent /** - * Interface for listening to events from the Spark scheduler. + * :: DeveloperApi :: + * Interface for listening to events from the Spark scheduler. Note that this is an internal + * interface which might change in different Spark releases. */ +@DeveloperApi trait SparkListener { /** * Called when a stage completes successfully or fails, with information on the completed stage. @@ -128,8 +144,10 @@ trait SparkListener { } /** + * :: DeveloperApi :: * Simple SparkListener that logs a few summary statistics when each stage completes */ +@DeveloperApi class StatsReportListener extends SparkListener with Logging { import org.apache.spark.scheduler.StatsReportListener._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index 5b40a3eb29b30..b85eabd6bbdbc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -19,8 +19,11 @@ package org.apache.spark.scheduler import collection.mutable.ArrayBuffer +import org.apache.spark.annotation.DeveloperApi + // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. +@DeveloperApi class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, val length: Long, val underlyingSplit: Any) { override def toString(): String = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index eec409b182ac6..9f732f7191465 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -17,12 +17,14 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.storage.RDDInfo /** + * :: DeveloperApi :: * Stores information about a stage to pass from the scheduler to SparkListeners. */ -private[spark] +@DeveloperApi class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala index 6183b125def99..4c62e4dc0bac8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskInfo.scala @@ -17,10 +17,13 @@ package org.apache.spark.scheduler +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * Information about a running task attempt inside a TaskSet. */ -private[spark] +@DeveloperApi class TaskInfo( val taskId: Long, val index: Int, @@ -46,15 +49,15 @@ class TaskInfo( var serializedSize: Int = 0 - def markGettingResult(time: Long = System.currentTimeMillis) { + private[spark] def markGettingResult(time: Long = System.currentTimeMillis) { gettingResultTime = time } - def markSuccessful(time: Long = System.currentTimeMillis) { + private[spark] def markSuccessful(time: Long = System.currentTimeMillis) { finishTime = time } - def markFailed(time: Long = System.currentTimeMillis) { + private[spark] def markFailed(time: Long = System.currentTimeMillis) { finishTime = time failed = true } @@ -83,11 +86,11 @@ class TaskInfo( def duration: Long = { if (!finished) { - throw new UnsupportedOperationException("duration() called on unfinished tasks") + throw new UnsupportedOperationException("duration() called on unfinished task") } else { finishTime - launchTime } } - def timeRunning(currentTime: Long): Long = currentTime - launchTime + private[spark] def timeRunning(currentTime: Long): Long = currentTime - launchTime } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala index 308edb12edd5c..eb920ab0c0b67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskLocality.scala @@ -17,7 +17,10 @@ package org.apache.spark.scheduler -private[spark] object TaskLocality extends Enumeration { +import org.apache.spark.annotation.DeveloperApi + +@DeveloperApi +object TaskLocality extends Enumeration { // Process local is expected to be used ONLY within TaskSetManager for now. val PROCESS_LOCAL, NODE_LOCAL, RACK_LOCAL, ANY = Value diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 18a68b05fa853..5e5883554fcc1 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -21,6 +21,7 @@ import java.io._ import java.nio.ByteBuffer import org.apache.spark.SparkConf +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) @@ -94,8 +95,14 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } /** + * :: DeveloperApi :: * A Spark serializer that uses Java's built-in serialization. + * + * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * Spark. It is intended to be used to serialize/de-serialize data within a single + * Spark application. */ +@DeveloperApi class JavaSerializer(conf: SparkConf) extends Serializer with Externalizable { private var counterReset = conf.getInt("spark.serializer.objectStreamReset", 10000) 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 926e71573be32..d1e8c3ef63622 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -33,6 +33,10 @@ import org.apache.spark.storage.{GetBlock, GotBlock, PutBlock} /** * A Spark serializer that uses the [[https://code.google.com/p/kryo/ Kryo serialization library]]. + * + * Note that this serializer is not guaranteed to be wire-compatible across different versions of + * Spark. It is intended to be used to serialize/de-serialize data within a single + * Spark application. */ class KryoSerializer(conf: SparkConf) extends org.apache.spark.serializer.Serializer diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 099143494b851..9f04dc6e427c0 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -22,20 +22,27 @@ import java.nio.ByteBuffer import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream -import org.apache.spark.util.{ByteBufferInputStream, NextIterator} import org.apache.spark.SparkEnv +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.{ByteBufferInputStream, NextIterator} /** + * :: DeveloperApi :: * A serializer. Because some serialization libraries are not thread safe, this class is used to * create [[org.apache.spark.serializer.SerializerInstance]] objects that do the actual * serialization and are guaranteed to only be called from one thread at a time. * * Implementations of this trait should implement: + * * 1. a zero-arg constructor or a constructor that accepts a [[org.apache.spark.SparkConf]] * as parameter. If both constructors are defined, the latter takes precedence. * * 2. Java serialization interface. + * + * Note that serializers are not required to be wire-compatible across different versions of Spark. + * They are intended to be used to serialize/de-serialize data within a single Spark application. */ +@DeveloperApi trait Serializer { def newInstance(): SerializerInstance } @@ -49,8 +56,10 @@ object Serializer { /** + * :: DeveloperApi :: * An instance of a serializer, for use by one thread at a time. */ +@DeveloperApi trait SerializerInstance { def serialize[T](t: T): ByteBuffer @@ -81,8 +90,10 @@ trait SerializerInstance { /** + * :: DeveloperApi :: * A stream for writing serialized objects. */ +@DeveloperApi trait SerializationStream { def writeObject[T](t: T): SerializationStream def flush(): Unit @@ -98,8 +109,10 @@ trait SerializationStream { /** + * :: DeveloperApi :: * A stream for reading serialized objects. */ +@DeveloperApi trait DeserializationStream { def readObject[T](): T def close(): Unit diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index b021564477c47..a2a729130091f 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -35,11 +35,11 @@ import org.apache.spark.network._ import org.apache.spark.serializer.Serializer import org.apache.spark.util._ -sealed trait Values +private[spark] sealed trait Values -case class ByteBufferValues(buffer: ByteBuffer) extends Values -case class IteratorValues(iterator: Iterator[Any]) extends Values -case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values +private[spark] case class ByteBufferValues(buffer: ByteBuffer) extends Values +private[spark] case class IteratorValues(iterator: Iterator[Any]) extends Values +private[spark] case class ArrayBufferValues(buffer: ArrayBuffer[Any]) extends Values private[spark] class BlockManager( executorId: String, diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index ff6e84cf9819a..07255aa366a6d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,6 +21,7 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.Utils private[spark] @@ -47,6 +48,7 @@ class StorageStatus( } } +@DeveloperApi private[spark] class RDDInfo( val id: Int, diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index c3692f2fd929b..b9f4a5d720b93 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -28,7 +28,7 @@ import scala.collection.generic.Growable * class and modifies it such that only the top K elements are retained. * The top K elements are defined by an implicit Ordering[A]. */ -class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) +private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Ordering[A]) extends Iterable[A] with Growable[A] with Serializable { private val underlying = new JPriorityQueue[A](maxSize, ord) diff --git a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala index db3db87e6618e..93235031f3ad5 100644 --- a/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala +++ b/core/src/main/scala/org/apache/spark/util/CollectionsUtil.scala @@ -22,7 +22,7 @@ import java.util import scala.Array import scala.reflect._ -object CollectionsUtils { +private[spark] object CollectionsUtils { def makeBinarySearch[K <% Ordered[K] : ClassTag] : (Array[K], K) => Int = { classTag[K] match { case ClassTag.Float => diff --git a/core/src/main/scala/org/apache/spark/util/Distribution.scala b/core/src/main/scala/org/apache/spark/util/Distribution.scala index 5b347555fe708..a465298c8c5ab 100644 --- a/core/src/main/scala/org/apache/spark/util/Distribution.scala +++ b/core/src/main/scala/org/apache/spark/util/Distribution.scala @@ -29,7 +29,7 @@ import scala.collection.immutable.IndexedSeq * * Assumes you are giving it a non-empty set of data */ -class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { +private[spark] class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) { require(startIdx < endIdx) def this(data: Traversable[Double]) = this(data.toArray, 0, data.size) java.util.Arrays.sort(data, startIdx, endIdx) @@ -69,7 +69,7 @@ class Distribution(val data: Array[Double], val startIdx: Int, val endIdx: Int) } } -object Distribution { +private[spark] object Distribution { def apply(data: Traversable[Double]): Option[Distribution] = { if (data.size > 0) { diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index a0c07e32fdc98..b5f2ec6831d26 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -36,7 +36,7 @@ import org.apache.spark.io.CompressionCodec * @param compress Whether to compress output * @param overwrite Whether to overwrite existing files */ -class FileLogger( +private[spark] class FileLogger( logDir: String, conf: SparkConf = new SparkConf, outputBufferSize: Int = 8 * 1024, // 8 KB 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 a6b39247a54ca..74fa77b68de0b 100644 --- a/core/src/main/scala/org/apache/spark/util/MutablePair.scala +++ b/core/src/main/scala/org/apache/spark/util/MutablePair.scala @@ -17,13 +17,17 @@ package org.apache.spark.util +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A tuple of 2 elements. This can be used as an alternative to Scala's Tuple2 when we want to * minimize object allocation. * * @param _1 Element 1 of this MutablePair * @param _2 Element 2 of this MutablePair */ +@DeveloperApi case class MutablePair[@specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T1, @specialized(Int, Long, Double, Char, Boolean/* , AnyRef */) T2] (var _1: T1, var _2: T2) diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala index 19bece86b36b4..7cd8f28b12dd6 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedHashSet.scala @@ -22,7 +22,7 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.JavaConversions import scala.collection.mutable.Set -class TimeStampedHashSet[A] extends Set[A] { +private[spark] class TimeStampedHashSet[A] extends Set[A] { val internalMap = new ConcurrentHashMap[A, Long]() def contains(key: A): Boolean = { diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index dc4b8f253f259..3c8f94a416c65 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -21,6 +21,7 @@ import scala.util.Random import org.apache.spark.util.random.XORShiftRandom +@deprecated("Use Vectors.dense from Spark's mllib.linalg package instead.", "1.0.0") class Vector(val elements: Array[Double]) extends Serializable { def length = elements.length diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index b8c852b4ff5c7..025492b177a77 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -19,7 +19,10 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A simple open hash table optimized for the append-only use case, where keys * are never removed, but the value for each key may be changed. * @@ -29,9 +32,9 @@ import java.util.{Arrays, Comparator} * * TODO: Cache the hash values of each key? java.util.HashMap does that. */ -private[spark] -class AppendOnlyMap[K, V](initialCapacity: Int = 64) extends Iterable[(K, - V)] with Serializable { +@DeveloperApi +class AppendOnlyMap[K, V](initialCapacity: Int = 64) + extends Iterable[(K, V)] with Serializable { require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index caa06d5b445b4..dd01ae821f705 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -27,10 +27,12 @@ import com.google.common.io.ByteStreams import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.serializer.Serializer import org.apache.spark.storage.{BlockId, BlockManager} /** + * :: DeveloperApi :: * An append-only map that spills sorted content to disk when there is insufficient space for it * to grow. * @@ -55,8 +57,8 @@ import org.apache.spark.storage.{BlockId, BlockManager} * `spark.shuffle.safetyFraction` specifies an additional margin of safety as a fraction of * this threshold, in case map size estimation is not sufficiently accurate. */ - -private[spark] class ExternalAppendOnlyMap[K, V, C]( +@DeveloperApi +class ExternalAppendOnlyMap[K, V, C]( createCombiner: V => C, mergeValue: (C, V) => C, mergeCombiners: (C, C) => C, diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index c26f23d50024a..62f99f3981793 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -19,14 +19,17 @@ package org.apache.spark.util.collection import scala.reflect.ClassTag +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A fast hash map implementation for nullable keys. This hash map supports insertions and updates, * but not deletions. This map is about 5X faster than java.util.HashMap, while using much less * space overhead. * * Under the hood, it uses our OpenHashSet implementation. */ -private[spark] +@DeveloperApi class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] diff --git a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala index 98569143ee1e3..70f3dd62b9b19 100644 --- a/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/Pseudorandom.scala @@ -17,9 +17,13 @@ package org.apache.spark.util.random +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A class with pseudorandom behavior. */ +@DeveloperApi trait Pseudorandom { /** Set random seed. */ def setSeed(seed: Long) diff --git a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala index 0f1fca4813ba9..37a6b04f5200f 100644 --- a/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala +++ b/core/src/main/scala/org/apache/spark/util/random/RandomSampler.scala @@ -22,7 +22,10 @@ import java.util.Random import cern.jet.random.Poisson import cern.jet.random.engine.DRand +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: * A pseudorandom sampler. It is possible to change the sampled item type. For example, we might * want to add weights for stratified sampling or importance sampling. Should only use * transformations that are tied to the sampler and cannot be applied after sampling. @@ -30,6 +33,7 @@ import cern.jet.random.engine.DRand * @tparam T item type * @tparam U sampled item type */ +@DeveloperApi trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable { /** take a random sample */ @@ -40,6 +44,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable } /** + * :: DeveloperApi :: * A sampler based on Bernoulli trials. * * @param lb lower bound of the acceptance range @@ -47,6 +52,7 @@ trait RandomSampler[T, U] extends Pseudorandom with Cloneable with Serializable * @param complement whether to use the complement of the range specified, default to false * @tparam T item type */ +@DeveloperApi class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) (implicit random: Random = new XORShiftRandom) extends RandomSampler[T, T] { @@ -67,11 +73,13 @@ class BernoulliSampler[T](lb: Double, ub: Double, complement: Boolean = false) } /** + * :: DeveloperApi :: * A sampler based on values drawn from Poisson distribution. * * @param poisson a Poisson random number generator * @tparam T item type */ +@DeveloperApi class PoissonSampler[T](mean: Double) (implicit var poisson: Poisson = new Poisson(mean, new DRand)) extends RandomSampler[T, T] { diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 2245bcbc70f1e..bbd56d2fd13bb 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -24,7 +24,9 @@ external_projects = ["flume", "kafka", "mqtt", "twitter", "zeromq"] sql_projects = ["catalyst", "core", "hive"] - projects = core_projects + external_projects.map { |project_name| "external/" + project_name } + projects = core_projects + projects = projects + external_projects.map { |project_name| "external/" + project_name } + projects = projects + sql_projects.map { |project_name| "sql/" + project_name } puts "Moving to project root and building scaladoc." curr_dir = pwd @@ -42,24 +44,22 @@ source = "../" + project_name + "/target/scala-2.10/api" dest = "api/" + project_name - puts "echo making directory " + dest + puts "making directory " + dest mkdir_p dest # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. puts "cp -r " + source + "/. " + dest cp_r(source + "/.", dest) - end - - sql_projects.each do |project_name| - source = "../sql/" + project_name + "/target/scala-2.10/api/" - dest = "api/sql/" + project_name - puts "echo making directory " + dest - mkdir_p dest + # Append custom JavaScript + js = File.readlines("./js/api-docs.js") + js_file = dest + "/lib/template.js" + File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } - # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. - puts "cp -r " + source + "/. " + dest - cp_r(source + "/.", dest) + # Append custom CSS + css = File.readlines("./css/api-docs.css") + css_file = dest + "/lib/template.css" + File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } end # Build Epydoc for Python diff --git a/docs/css/api-docs.css b/docs/css/api-docs.css new file mode 100644 index 0000000000000..b2d1d7f869790 --- /dev/null +++ b/docs/css/api-docs.css @@ -0,0 +1,18 @@ +/* Dynamically injected style for the API docs */ + +.developer { + background-color: #44751E; +} + +.experimental { + background-color: #257080; +} + +.alphaComponent { + background-color: #bb0000; +} + +.badge { + font-family: Arial, san-serif; + float: right; +} diff --git a/docs/js/api-docs.js b/docs/js/api-docs.js new file mode 100644 index 0000000000000..1414b6d0b81a1 --- /dev/null +++ b/docs/js/api-docs.js @@ -0,0 +1,26 @@ +/* Dynamically injected post-processing code for the API docs */ + +$(document).ready(function() { + var annotations = $("dt:contains('Annotations')").next("dd").children("span.name"); + addBadges(annotations, "AlphaComponent", ":: AlphaComponent ::", "Alpha Component"); + addBadges(annotations, "DeveloperApi", ":: DeveloperApi ::", "Developer API"); + addBadges(annotations, "Experimental", ":: Experimental ::", "Experimental"); +}); + +function addBadges(allAnnotations, name, tag, html) { + var annotations = allAnnotations.filter(":contains('" + name + "')") + var tags = $(".cmt:contains(" + tag + ")") + + // Remove identifier tags from comments + tags.each(function(index) { + var oldHTML = $(this).html(); + var newHTML = oldHTML.replace(tag, ""); + $(this).html(newHTML); + }); + + // Add badges to all containers + tags.prevAll("h4.signature") + .add(annotations.closest("div.fullcommenttop")) + .add(annotations.closest("div.fullcomment").prevAll("h4.signature")) + .prepend(html); +} diff --git a/graphx/src/main/scala/org/apache/spark/graphx/package.scala b/graphx/src/main/scala/org/apache/spark/graphx/package.scala index 425a5164cad24..ff17edeaf8f16 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/package.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/package.scala @@ -19,7 +19,10 @@ package org.apache.spark import org.apache.spark.util.collection.OpenHashSet -/** GraphX is a graph processing framework built on top of Spark. */ +/** + * ALPHA COMPONENT + * GraphX is a graph processing framework built on top of Spark. + */ package object graphx { /** * A 64-bit vertex identifier that uniquely identifies a vertex within a graph. It does not need 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 0f06ea088e1a1..77b5429bad432 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 @@ -90,7 +90,7 @@ trait MutableRow extends Row { def setString(ordinal: Int, value: String) /** - * EXPERIMENTAL + * 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 diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 3193787680d16..d3d4c56bafe41 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -21,6 +21,7 @@ import scala.language.implicitConversions import scala.reflect.runtime.universe.TypeTag import org.apache.spark.SparkContext +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.dsl @@ -32,14 +33,14 @@ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan import org.apache.spark.sql.execution._ /** - * ALPHA COMPONENT - * + * :: AlphaComponent :: * The entry point for running relational queries using Spark. Allows the creation of [[SchemaRDD]] * objects and the execution of SQL queries. * * @groupname userf Spark SQL Functions * @groupname Ungrouped Support functions for language integrated queries. */ +@AlphaComponent class SQLContext(@transient val sparkContext: SparkContext) extends Logging with dsl.ExpressionConversions @@ -63,12 +64,12 @@ class SQLContext(@transient val sparkContext: SparkContext) new this.QueryExecution { val logical = plan } /** - * EXPERIMENTAL - * + * :: Experimental :: * Allows catalyst LogicalPlans to be executed as a SchemaRDD. Note that the LogicalPlan * interface is considered internal, and thus not guranteed to be stable. As a result, using * them directly is not reccomended. */ + @Experimental implicit def logicalPlanToSparkQuery(plan: LogicalPlan): SchemaRDD = new SchemaRDD(this, plan) /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index fc95781448569..16da7fd92bffe 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -18,6 +18,7 @@ package org.apache.spark.sql import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ @@ -26,8 +27,7 @@ import org.apache.spark.sql.catalyst.plans.{Inner, JoinType} import org.apache.spark.sql.catalyst.types.BooleanType /** - * ALPHA COMPONENT - * + * :: AlphaComponent :: * An RDD of [[Row]] objects that has an associated schema. In addition to standard RDD functions, * SchemaRDDs can be used in relational queries, as shown in the examples below. * @@ -90,6 +90,7 @@ import org.apache.spark.sql.catalyst.types.BooleanType * @groupprio schema -1 * @groupname Ungrouped Base RDD Functions */ +@AlphaComponent class SchemaRDD( @transient val sqlContext: SQLContext, @transient protected[spark] val logicalPlan: LogicalPlan) @@ -228,8 +229,7 @@ class SchemaRDD( Filter(ScalaUdf(udf, BooleanType, Seq(UnresolvedAttribute(arg1.name))), logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Filters tuples using a function over a `Dynamic` version of a given Row. DynamicRows use * scala's Dynamic trait to emulate an ORM of in a dynamically typed language. Since the type of * the column is not known at compile time, all attributes are converted to strings before @@ -241,18 +241,19 @@ class SchemaRDD( * * @group Query */ + @Experimental def where(dynamicUdf: (DynamicRow) => Boolean) = new SchemaRDD( sqlContext, Filter(ScalaUdf(dynamicUdf, BooleanType, Seq(WrapDynamic(logicalPlan.output))), logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Returns a sampled version of the underlying dataset. * * @group Query */ + @Experimental def sample( fraction: Double, withReplacement: Boolean = true, @@ -260,8 +261,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Applies the given Generator, or table generating function, to this relation. * * @param generator A table generating function. The API for such functions is likely to change @@ -277,6 +277,7 @@ class SchemaRDD( * * @group Query */ + @Experimental def generate( generator: Generator, join: Boolean = false, @@ -285,8 +286,7 @@ class SchemaRDD( new SchemaRDD(sqlContext, Generate(generator, join, outer, None, logicalPlan)) /** - * EXPERIMENTAL - * + * :: Experimental :: * Adds the rows from this RDD to the specified table. Note in a standard [[SQLContext]] there is * no notion of persistent tables, and thus queries that contain this operator will fail to * optimize. When working with an extension of a SQLContext that has a persistent catalog, such @@ -294,6 +294,7 @@ class SchemaRDD( * * @group schema */ + @Experimental def insertInto(tableName: String, overwrite: Boolean = false) = new SchemaRDD( sqlContext, diff --git a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala rename to tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala index 8cea302eb14c3..8e8c35615a711 100644 --- a/core/src/main/scala/org/apache/spark/storage/StoragePerfTester.scala +++ b/tools/src/main/scala/org/apache/spark/tools/StoragePerfTester.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.storage +package org.apache.spark.tools import java.util.concurrent.{CountDownLatch, Executors} import java.util.concurrent.atomic.AtomicLong @@ -25,7 +25,7 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.Utils /** - * Utility for micro-benchmarking shuffle write performance. + * Internal utility for micro-benchmarking shuffle write performance. * * Writes simulated shuffle output from several threads and records the observed throughput. */ From bde9cc11fee42a0a41ec52d5dc7fa0502ce94f77 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 9 Apr 2014 02:21:15 -0700 Subject: [PATCH 243/397] [SPARK-1357] [MLLIB] Annotate developer and experimental APIs Annotate developer and experimental APIs in MLlib. Author: Xiangrui Meng Closes #298 from mengxr/api and squashes the following commits: 13390e8 [Xiangrui Meng] Merge branch 'master' into api dc4cbb3 [Xiangrui Meng] mark distribute matrices experimental 6b9f8e2 [Xiangrui Meng] add Experimental annotation 8773d0d [Xiangrui Meng] add DeveloperApi annotation da31733 [Xiangrui Meng] update developer and experimental tags 555e0fe [Xiangrui Meng] Merge branch 'master' into api ef1a717 [Xiangrui Meng] mark some constructors private add default parameters to JavaDoc 00ffbcc [Xiangrui Meng] update tree API annotation 0b674fa [Xiangrui Meng] mark decision tree APIs 86b9e34 [Xiangrui Meng] one pass over APIs of GLMs, NaiveBayes, and ALS f21d862 [Xiangrui Meng] Merge branch 'master' into api 2b133d6 [Xiangrui Meng] intial annotation of developer and experimental apis --- .../mllib/api/python/PythonMLLibAPI.scala | 4 ++ .../classification/LogisticRegression.scala | 18 +++---- .../mllib/classification/NaiveBayes.scala | 29 +++++++---- .../spark/mllib/classification/SVM.scala | 21 ++++---- .../spark/mllib/clustering/KMeans.scala | 49 ++++++++++++++++--- .../apache/spark/mllib/linalg/Vectors.scala | 19 ++++--- .../linalg/distributed/CoordinateMatrix.scala | 4 ++ .../distributed/DistributedMatrix.scala | 2 - .../linalg/distributed/IndexedRowMatrix.scala | 11 ++++- .../mllib/linalg/distributed/RowMatrix.scala | 4 ++ .../spark/mllib/optimization/Gradient.scala | 13 +++++ .../mllib/optimization/GradientDescent.scala | 15 ++++-- .../spark/mllib/optimization/Optimizer.scala | 7 +++ .../spark/mllib/optimization/Updater.scala | 13 +++++ .../spark/mllib/recommendation/ALS.scala | 34 ++++++++----- .../MatrixFactorizationModel.scala | 9 ++-- .../GeneralizedLinearAlgorithm.scala | 7 ++- .../apache/spark/mllib/regression/Lasso.scala | 18 ++++--- .../mllib/regression/LinearRegression.scala | 16 +++--- .../mllib/regression/RidgeRegression.scala | 20 ++++---- .../spark/mllib/tree/DecisionTree.scala | 10 ++-- .../spark/mllib/tree/configuration/Algo.scala | 5 ++ .../tree/configuration/FeatureType.scala | 5 ++ .../tree/configuration/QuantileStrategy.scala | 5 ++ .../mllib/tree/configuration/Strategy.scala | 6 ++- .../spark/mllib/tree/impurity/Entropy.scala | 32 +++++++----- .../spark/mllib/tree/impurity/Gini.scala | 10 +++- .../spark/mllib/tree/impurity/Impurity.scala | 12 ++++- .../spark/mllib/tree/impurity/Variance.scala | 8 +++ .../apache/spark/mllib/tree/model/Bin.scala | 1 + .../mllib/tree/model/DecisionTreeModel.scala | 4 ++ .../spark/mllib/tree/model/Filter.scala | 2 +- .../tree/model/InformationGainStats.scala | 5 ++ .../apache/spark/mllib/tree/model/Node.scala | 4 ++ .../apache/spark/mllib/tree/model/Split.scala | 12 +++-- .../spark/mllib/util/DataValidators.scala | 8 +-- .../mllib/util/KMeansDataGenerator.scala | 5 +- .../mllib/util/LinearDataGenerator.scala | 4 ++ .../LogisticRegressionDataGenerator.scala | 5 +- .../spark/mllib/util/MFDataGenerator.scala | 8 +-- .../org/apache/spark/mllib/util/MLUtils.scala | 9 +++- .../spark/mllib/util/SVMDataGenerator.scala | 4 ++ 42 files changed, 355 insertions(+), 122 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index 2df5b0d02b699..ae27c57799873 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.api.python import java.nio.{ByteBuffer, ByteOrder} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.api.java.JavaRDD import org.apache.spark.mllib.classification._ import org.apache.spark.mllib.clustering._ @@ -28,8 +29,11 @@ import org.apache.spark.mllib.regression._ import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: + * * The Java stubs necessary for the Python mllib bindings. */ +@DeveloperApi class PythonMLLibAPI extends Serializable { private def deserializeDoubleVector(bytes: Array[Byte]): Array[Double] = { val packetLength = bytes.length diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala index 798f3a5c94740..4f9eaacf67fe4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/LogisticRegression.scala @@ -55,7 +55,7 @@ class LogisticRegressionModel( this } - override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, + override protected def predictPoint(dataMatrix: Vector, weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept val score = 1.0/ (1.0 + math.exp(-margin)) @@ -71,27 +71,27 @@ class LogisticRegressionModel( * NOTE: Labels used in Logistic Regression should be {0, 1} */ class LogisticRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LogisticRegressionModel] with Serializable { - val gradient = new LogisticGradient() - val updater = new SimpleUpdater() + private val gradient = new LogisticGradient() + private val updater = new SimpleUpdater() override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - override val validators = List(DataValidators.classificationLabels) + override protected val validators = List(DataValidators.binaryLabelValidator) /** * Construct a LogisticRegression object with default parameters */ def this() = this(1.0, 100, 0.0, 1.0) - def createModel(weights: Vector, intercept: Double) = { + override protected def createModel(weights: Vector, intercept: Double) = { new LogisticRegressionModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index e956185319a69..5a45f12f1aa12 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.classification import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, argmax => brzArgmax, sum => brzSum} +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vector @@ -27,11 +28,16 @@ import org.apache.spark.mllib.util.MLUtils import org.apache.spark.rdd.RDD /** + * :: Experimental :: + * * Model for Naive Bayes Classifiers. * - * @param pi Log of class priors, whose dimension is C. - * @param theta Log of class conditional probabilities, whose dimension is CxD. + * @param labels list of labels + * @param pi log of class priors, whose dimension is C, number of labels + * @param theta log of class conditional probabilities, whose dimension is C-by-D, + * where D is number of features */ +@Experimental class NaiveBayesModel( val labels: Array[Double], val pi: Array[Double], @@ -40,14 +46,17 @@ class NaiveBayesModel( private val brzPi = new BDV[Double](pi) private val brzTheta = new BDM[Double](theta.length, theta(0).length) - var i = 0 - while (i < theta.length) { - var j = 0 - while (j < theta(i).length) { - brzTheta(i, j) = theta(i)(j) - j += 1 + { + // Need to put an extra pair of braces to prevent Scala treating `i` as a member. + var i = 0 + while (i < theta.length) { + var j = 0 + while (j < theta(i).length) { + brzTheta(i, j) = theta(i)(j) + j += 1 + } + i += 1 } - i += 1 } override def predict(testData: RDD[Vector]): RDD[Double] = testData.map(predict) @@ -65,7 +74,7 @@ class NaiveBayesModel( * document classification. By making every vector a 0-1 vector, it can also be used as * Bernoulli NB ([[http://tinyurl.com/p7c96j6]]). */ -class NaiveBayes private (var lambda: Double) extends Serializable with Logging { +class NaiveBayes private (private var lambda: Double) extends Serializable with Logging { def this() = this(1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala index e31a08899f8bc..956654b1fe90a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/SVM.scala @@ -55,7 +55,9 @@ class SVMModel( this } - override def predictPoint(dataMatrix: Vector, weightMatrix: Vector, + override protected def predictPoint( + dataMatrix: Vector, + weightMatrix: Vector, intercept: Double) = { val margin = weightMatrix.toBreeze.dot(dataMatrix.toBreeze) + intercept threshold match { @@ -70,28 +72,27 @@ class SVMModel( * NOTE: Labels used in SVM should be {0, 1}. */ class SVMWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[SVMModel] with Serializable { - val gradient = new HingeGradient() - val updater = new SquaredL2Updater() + private val gradient = new HingeGradient() + private val updater = new SquaredL2Updater() override val optimizer = new GradientDescent(gradient, updater) .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) - - override val validators = List(DataValidators.classificationLabels) + override protected val validators = List(DataValidators.binaryLabelValidator) /** * Construct a SVM object with default parameters */ def this() = this(1.0, 100, 1.0, 1.0) - def createModel(weights: Vector, intercept: Double) = { + override protected def createModel(weights: Vector, intercept: Double) = { new SVMModel(weights, intercept) } } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index a78503df3134d..8f565eb60a60f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.ArrayBuffer import breeze.linalg.{DenseVector => BDV, Vector => BV, norm => breezeNorm} +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.{Vector, Vectors} @@ -37,12 +38,17 @@ import org.apache.spark.util.random.XORShiftRandom * to it should be cached by the user. */ class KMeans private ( - var k: Int, - var maxIterations: Int, - var runs: Int, - var initializationMode: String, - var initializationSteps: Int, - var epsilon: Double) extends Serializable with Logging { + private var k: Int, + private var maxIterations: Int, + private var runs: Int, + private var initializationMode: String, + private var initializationSteps: Int, + private var epsilon: Double) extends Serializable with Logging { + + /** + * Constructs a KMeans instance with default parameters: {k: 2, maxIterations: 20, runs: 1, + * initializationMode: "k-means||", initializationSteps: 5, epsilon: 1e-4}. + */ def this() = this(2, 20, 1, KMeans.K_MEANS_PARALLEL, 5, 1e-4) /** Set the number of clusters to create (k). Default: 2. */ @@ -71,6 +77,8 @@ class KMeans private ( } /** + * :: Experimental :: + * * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. @@ -316,8 +324,8 @@ object KMeans { data: RDD[Vector], k: Int, maxIterations: Int, - runs: Int = 1, - initializationMode: String = K_MEANS_PARALLEL): KMeansModel = { + runs: Int, + initializationMode: String): KMeansModel = { new KMeans().setK(k) .setMaxIterations(maxIterations) .setRuns(runs) @@ -325,6 +333,27 @@ object KMeans { .run(data) } + /** + * Trains a k-means model using specified parameters and the default values for unspecified. + */ + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int): KMeansModel = { + train(data, k, maxIterations, 1, K_MEANS_PARALLEL) + } + + /** + * Trains a k-means model using specified parameters and the default values for unspecified. + */ + def train( + data: RDD[Vector], + k: Int, + maxIterations: Int, + runs: Int): KMeansModel = { + train(data, k, maxIterations, runs, K_MEANS_PARALLEL) + } + /** * Returns the index of the closest center to the given point, as well as the squared distance. */ @@ -369,6 +398,10 @@ object KMeans { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } + /** + * :: Experimental :: + */ + @Experimental def main(args: Array[String]) { if (args.length < 4) { println("Usage: KMeans []") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala index 2cea58cd3fd22..99a849f1c66b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/Vectors.scala @@ -64,11 +64,13 @@ trait Vector extends Serializable { /** * Factory methods for [[org.apache.spark.mllib.linalg.Vector]]. + * We don't use the name `Vector` because Scala imports + * [[scala.collection.immutable.Vector]] by default. */ object Vectors { /** - * Creates a dense vector. + * Creates a dense vector from its values. */ @varargs def dense(firstValue: Double, otherValues: Double*): Vector = @@ -158,20 +160,21 @@ class DenseVector(val values: Array[Double]) extends Vector { /** * A sparse vector represented by an index array and an value array. * - * @param n size of the vector. + * @param size size of the vector. * @param indices index array, assume to be strictly increasing. * @param values value array, must have the same length as the index array. */ -class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double]) extends Vector { - - override def size: Int = n +class SparseVector( + override val size: Int, + val indices: Array[Int], + val values: Array[Double]) extends Vector { override def toString: String = { - "(" + n + "," + indices.zip(values).mkString("[", "," ,"]") + ")" + "(" + size + "," + indices.zip(values).mkString("[", "," ,"]") + ")" } override def toArray: Array[Double] = { - val data = new Array[Double](n) + val data = new Array[Double](size) var i = 0 val nnz = indices.length while (i < nnz) { @@ -181,5 +184,5 @@ class SparseVector(val n: Int, val indices: Array[Int], val values: Array[Double data } - private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, n) + private[mllib] override def toBreeze: BV[Double] = new BSV[Double](indices, values, size) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 9194f657494b2..89d5c03d76c42 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.linalg.Vectors @@ -32,6 +33,8 @@ import org.apache.spark.mllib.linalg.Vectors case class MatrixEntry(i: Long, j: Long, value: Double) /** + * :: Experimental :: + * * Represents a matrix in coordinate format. * * @param entries matrix entries @@ -40,6 +43,7 @@ case class MatrixEntry(i: Long, j: Long, value: Double) * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the max column index plus one. */ +@Experimental class CoordinateMatrix( val entries: RDD[MatrixEntry], private var nRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala index 13f72a3c724ef..a0e26ce3bc465 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/DistributedMatrix.scala @@ -19,8 +19,6 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} -import org.apache.spark.mllib.linalg.Matrix - /** * Represents a distributively stored matrix backed by one or more RDDs. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index e110f070bd7c1..24c123ab7eb51 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -19,14 +19,22 @@ package org.apache.spark.mllib.linalg.distributed import breeze.linalg.{DenseMatrix => BDM} +import org.apache.spark.annotation.Experimental import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg._ import org.apache.spark.mllib.linalg.SingularValueDecomposition -/** Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ +/** + * :: Experimental :: + * + * Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. + */ +@Experimental case class IndexedRow(index: Long, vector: Vector) /** + * :: Experimental :: + * * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with * indexed rows. * @@ -36,6 +44,7 @@ case class IndexedRow(index: Long, vector: Vector) * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the size of the first row. */ +@Experimental class IndexedRowMatrix( val rows: RDD[IndexedRow], private var nRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index f59811f18a68f..8d32c1a6dbba1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -23,11 +23,14 @@ import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.Logging /** + * :: Experimental :: + * * Represents a row-oriented distributed Matrix with no meaningful row indices. * * @param rows rows stored as an RDD[Vector] @@ -36,6 +39,7 @@ import org.apache.spark.Logging * @param nCols number of columns. A non-positive value means unknown, and then the number of * columns will be determined by the size of the first row. */ +@Experimental class RowMatrix( val rows: RDD[Vector], private var nRows: Long, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 20654284965ed..1176dc9dbc08d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -19,11 +19,15 @@ package org.apache.spark.mllib.optimization import breeze.linalg.{axpy => brzAxpy} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: + * * Class used to compute the gradient for a loss function, given a single data point. */ +@DeveloperApi abstract class Gradient extends Serializable { /** * Compute the gradient and loss given the features of a single data point. @@ -51,9 +55,12 @@ abstract class Gradient extends Serializable { } /** + * :: DeveloperApi :: + * * Compute gradient and loss for a logistic loss function, as used in binary classification. * See also the documentation for the precise formulation. */ +@DeveloperApi class LogisticGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze @@ -92,11 +99,14 @@ class LogisticGradient extends Gradient { } /** + * :: DeveloperApi :: + * * Compute gradient and loss for a Least-squared loss function, as used in linear regression. * This is correct for the averaged least squares loss function (mean squared error) * L = 1/n ||A weights-y||^2 * See also the documentation for the precise formulation. */ +@DeveloperApi class LeastSquaresGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze @@ -124,10 +134,13 @@ class LeastSquaresGradient extends Gradient { } /** + * :: DeveloperApi :: + * * Compute gradient and loss for a Hinge loss function, as used in SVM binary classification. * See also the documentation for the precise formulation. * NOTE: This assumes that the labels are {0,1} */ +@DeveloperApi class HingeGradient extends Gradient { override def compute(data: Vector, label: Double, weights: Vector): (Vector, Double) = { val brzData = data.toBreeze diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index d0777ffd63ff8..04267d967dcad 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -19,18 +19,22 @@ package org.apache.spark.mllib.optimization import scala.collection.mutable.ArrayBuffer -import breeze.linalg.{Vector => BV, DenseVector => BDV} +import breeze.linalg.{DenseVector => BDV} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: + * * Class used to solve an optimization problem using Gradient Descent. * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. */ -class GradientDescent(var gradient: Gradient, var updater: Updater) +@DeveloperApi +class GradientDescent(private var gradient: Gradient, private var updater: Updater) extends Optimizer with Logging { private var stepSize: Double = 1.0 @@ -107,7 +111,12 @@ class GradientDescent(var gradient: Gradient, var updater: Updater) } -// Top-level method to run gradient descent. +/** + * :: DeveloperApi :: + * + * Top-level method to run gradient descent. + */ +@DeveloperApi object GradientDescent extends Logging { /** * Run stochastic gradient descent (SGD) in parallel using mini batches. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index f9ce908a5f3b0..0a313f3104b14 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -19,8 +19,15 @@ package org.apache.spark.mllib.optimization import org.apache.spark.rdd.RDD +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.Vector +/** + * :: DeveloperApi :: + * + * Trait for optimization problem solvers. + */ +@DeveloperApi trait Optimizer extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index 3b7754cd7ac28..e67816796c6b1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -21,9 +21,12 @@ import scala.math._ import breeze.linalg.{norm => brzNorm, axpy => brzAxpy, Vector => BV} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.linalg.{Vectors, Vector} /** + * :: DeveloperApi :: + * * Class used to perform steps (weight update) using Gradient Descent methods. * * For general minimization problems, or for regularized problems of the form @@ -35,6 +38,7 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} * The updater is responsible to also perform the update coming from the * regularization term R(w) (if any regularization is used). */ +@DeveloperApi abstract class Updater extends Serializable { /** * Compute an updated value for weights given the gradient, stepSize, iteration number and @@ -59,9 +63,12 @@ abstract class Updater extends Serializable { } /** + * :: DeveloperApi :: + * * A simple updater for gradient descent *without* any regularization. * Uses a step-size decreasing with the square root of the number of iterations. */ +@DeveloperApi class SimpleUpdater extends Updater { override def compute( weightsOld: Vector, @@ -78,6 +85,8 @@ class SimpleUpdater extends Updater { } /** + * :: DeveloperApi :: + * * Updater for L1 regularized problems. * R(w) = ||w||_1 * Uses a step-size decreasing with the square root of the number of iterations. @@ -95,6 +104,7 @@ class SimpleUpdater extends Updater { * * Equivalently, set weight component to signum(w) * max(0.0, abs(w) - shrinkageVal) */ +@DeveloperApi class L1Updater extends Updater { override def compute( weightsOld: Vector, @@ -120,10 +130,13 @@ class L1Updater extends Updater { } /** + * :: DeveloperApi :: + * * Updater for L2 regularized problems. * R(w) = 1/2 ||w||^2 * Uses a step-size decreasing with the square root of the number of iterations. */ +@DeveloperApi class SquaredL2Updater extends Updater { override def compute( weightsOld: Vector, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 3124fac326d22..60cbb1c1e1d86 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -22,6 +22,10 @@ import scala.math.{abs, sqrt} import scala.util.Random import scala.util.Sorting +import com.esotericsoftware.kryo.Kryo +import org.jblas.{DoubleMatrix, SimpleBlas, Solve} + +import org.apache.spark.annotation.Experimental import org.apache.spark.broadcast.Broadcast import org.apache.spark.{Logging, HashPartitioner, Partitioner, SparkContext, SparkConf} import org.apache.spark.storage.StorageLevel @@ -29,10 +33,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ -import com.esotericsoftware.kryo.Kryo -import org.jblas.{DoubleMatrix, SimpleBlas, Solve} - - /** * Out-link information for a user or product block. This includes the original user/product IDs * of the elements within this block, and the list of destination blocks that each user or @@ -90,14 +90,19 @@ case class Rating(val user: Int, val product: Int, val rating: Double) * preferences rather than explicit ratings given to items. */ class ALS private ( - var numBlocks: Int, - var rank: Int, - var iterations: Int, - var lambda: Double, - var implicitPrefs: Boolean, - var alpha: Double, - var seed: Long = System.nanoTime() + private var numBlocks: Int, + private var rank: Int, + private var iterations: Int, + private var lambda: Double, + private var implicitPrefs: Boolean, + private var alpha: Double, + private var seed: Long = System.nanoTime() ) extends Serializable with Logging { + + /** + * Constructs an ALS instance with default parameters: {numBlocks: -1, rank: 10, iterations: 10, + * lambda: 0.01, implicitPrefs: false, alpha: 1.0}. + */ def this() = this(-1, 10, 10, 0.01, false, 1.0) /** @@ -127,11 +132,18 @@ class ALS private ( this } + /** Sets whether to use implicit preference. Default: false. */ def setImplicitPrefs(implicitPrefs: Boolean): ALS = { this.implicitPrefs = implicitPrefs this } + /** + * :: Experimental :: + * + * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. + */ + @Experimental def setAlpha(alpha: Double): ALS = { this.alpha = alpha this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index 443fc5de5bf04..e05224fc7caf2 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -17,13 +17,14 @@ package org.apache.spark.mllib.recommendation +import org.jblas._ + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.api.java.JavaRDD import org.apache.spark.rdd.RDD import org.apache.spark.SparkContext._ import org.apache.spark.mllib.api.python.PythonMLLibAPI -import org.jblas._ -import org.apache.spark.api.java.JavaRDD - /** * Model representing the result of matrix factorization. @@ -68,6 +69,8 @@ class MatrixFactorizationModel( } /** + * :: DeveloperApi :: + * * Predict the rating of many users for many products. * This is a Java stub for python predictAll() * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 80dc0f12ff84f..c24f5afb99686 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.regression import breeze.linalg.{DenseVector => BDV, SparseVector => BSV} +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.RDD import org.apache.spark.mllib.optimization._ @@ -79,7 +80,8 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] protected val validators: Seq[RDD[LabeledPoint] => Boolean] = List() - val optimizer: Optimizer + /** The optimizer to solve the problem. */ + def optimizer: Optimizer /** Whether to add intercept (default: true). */ protected var addIntercept: Boolean = true @@ -100,8 +102,11 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] } /** + * :: Experimental :: + * * Set if the algorithm should validate data before training. Default true. */ + @Experimental def setValidateData(validateData: Boolean): this.type = { this.validateData = validateData this diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala index 25920d0dc976e..5f0812fd2e0eb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/Lasso.scala @@ -52,15 +52,16 @@ class LassoModel( * See also the documentation for the precise formulation. */ class LassoWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LassoModel] with Serializable { - val gradient = new LeastSquaresGradient() - val updater = new L1Updater() - @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + private val gradient = new LeastSquaresGradient() + private val updater = new L1Updater() + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) @@ -69,7 +70,8 @@ class LassoWithSGD private ( super.setIntercept(false) /** - * Construct a Lasso object with default parameters + * Construct a Lasso object with default parameters: {stepSize: 1.0, numIterations: 100, + * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala index 9ed927994e795..228fa8db3e721 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/LinearRegression.scala @@ -52,19 +52,21 @@ class LinearRegressionModel( * See also the documentation for the precise formulation. */ class LinearRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var miniBatchFraction: Double) + private var stepSize: Double, + private var numIterations: Int, + private var miniBatchFraction: Double) extends GeneralizedLinearAlgorithm[LinearRegressionModel] with Serializable { - val gradient = new LeastSquaresGradient() - val updater = new SimpleUpdater() - val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + private val gradient = new LeastSquaresGradient() + private val updater = new SimpleUpdater() + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) .setNumIterations(numIterations) .setMiniBatchFraction(miniBatchFraction) /** - * Construct a LinearRegression object with default parameters + * Construct a LinearRegression object with default parameters: {stepSize: 1.0, + * numIterations: 100, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala index 1f17d2107f940..e702027c7c170 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RidgeRegression.scala @@ -52,16 +52,17 @@ class RidgeRegressionModel( * See also the documentation for the precise formulation. */ class RidgeRegressionWithSGD private ( - var stepSize: Double, - var numIterations: Int, - var regParam: Double, - var miniBatchFraction: Double) - extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable { + private var stepSize: Double, + private var numIterations: Int, + private var regParam: Double, + private var miniBatchFraction: Double) + extends GeneralizedLinearAlgorithm[RidgeRegressionModel] with Serializable { - val gradient = new LeastSquaresGradient() - val updater = new SquaredL2Updater() + private val gradient = new LeastSquaresGradient() + private val updater = new SquaredL2Updater() - @transient val optimizer = new GradientDescent(gradient, updater).setStepSize(stepSize) + override val optimizer = new GradientDescent(gradient, updater) + .setStepSize(stepSize) .setNumIterations(numIterations) .setRegParam(regParam) .setMiniBatchFraction(miniBatchFraction) @@ -70,7 +71,8 @@ class RidgeRegressionWithSGD private ( super.setIntercept(false) /** - * Construct a RidgeRegression object with default parameters + * Construct a RidgeRegression object with default parameters: {stepSize: 1.0, numIterations: 100, + * regParam: 1.0, miniBatchFraction: 1.0}. */ def this() = this(1.0, 100, 1.0, 1.0) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index dee9594a9dd79..c8a966cd5f5a8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -19,6 +19,7 @@ package org.apache.spark.mllib.tree import scala.util.control.Breaks._ +import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ import org.apache.spark.mllib.regression.LabeledPoint @@ -33,13 +34,16 @@ import org.apache.spark.util.random.XORShiftRandom import org.apache.spark.mllib.linalg.{Vector, Vectors} /** + * :: Experimental :: + * * A class that implements a decision tree algorithm for classification and regression. It * supports both continuous and categorical features. * @param strategy The configuration parameters for the tree algorithm which specify the type * of algorithm (classification, regression, etc.), feature type (continuous, * categorical), depth of the tree, quantile calculation strategy, etc. */ -class DecisionTree private(val strategy: Strategy) extends Serializable with Logging { +@Experimental +class DecisionTree (private val strategy: Strategy) extends Serializable with Logging { /** * Method to train a decision tree model over an RDD @@ -1024,7 +1028,7 @@ object DecisionTree extends Serializable with Logging { } } - val usage = """ + private val usage = """ Usage: DecisionTreeRunner [slices] --algo --trainDataDir path --testDataDir path --maxDepth num [--impurity ] [--maxBins num] @@ -1113,7 +1117,7 @@ object DecisionTree extends Serializable with Logging { * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is * the label, and the second element represents the feature values (an array of Double). */ - def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { + private def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { sc.textFile(dir).map { line => val parts = line.trim().split(",") val label = parts(0).toDouble diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 2dd1f0f27b8f5..017f84f3b9e8b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -17,9 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: + * * Enum to select the algorithm for the decision tree */ +@Experimental object Algo extends Enumeration { type Algo = Value val Classification, Regression = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala index 09ee0586c58fa..c0254c32c2dce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala @@ -17,9 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: + * * Enum to describe whether a feature is "continuous" or "categorical" */ +@Experimental object FeatureType extends Enumeration { type FeatureType = Value val Continuous, Categorical = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala index 2457a480c2a14..b3e8b224beeaa 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala @@ -17,9 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental + /** + * :: Experimental :: + * * Enum for selecting the quantile calculation strategy */ +@Experimental object QuantileStrategy extends Enumeration { type QuantileStrategy = Value val Sort, MinMax, ApproxHist = Value diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index df565f3eb8859..482faaa9e7256 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.tree.configuration +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.tree.impurity.Impurity import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ /** + * :: Experimental :: + * * Stores all the configuration options for tree construction * @param algo classification or regression * @param impurity criterion used for information gain calculation @@ -34,10 +37,11 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ * 1, 2, ... , k-1. It's important to note that features are * zero-indexed. */ +@Experimental class Strategy ( val algo: Algo, val impurity: Impurity, val maxDepth: Int, val maxBins: Int = 100, val quantileCalculationStrategy: QuantileStrategy = Sort, - val categoricalFeaturesInfo: Map[Int,Int] = Map[Int,Int]()) extends Serializable + val categoricalFeaturesInfo: Map[Int, Int] = Map[Int, Int]()) extends Serializable diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index b93995fcf9441..55c43f2fcf9c5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -17,31 +17,39 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during * binary classification. */ +@Experimental object Entropy extends Impurity { - def log2(x: Double) = scala.math.log(x) / scala.math.log(2) + private[tree] def log2(x: Double) = scala.math.log(x) / scala.math.log(2) /** + * :: DeveloperApi :: + * * entropy calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 * @return entropy value */ - def calculate(c0: Double, c1: Double): Double = { - if (c0 == 0 || c1 == 0) { - 0 - } else { - val total = c0 + c1 - val f0 = c0 / total - val f1 = c1 / total - -(f0 * log2(f0)) - (f1 * log2(f1)) - } - } + @DeveloperApi + override def calculate(c0: Double, c1: Double): Double = { + if (c0 == 0 || c1 == 0) { + 0 + } else { + val total = c0 + c1 + val f0 = c0 / total + val f1 = c1 / total + -(f0 * log2(f0)) - (f1 * log2(f1)) + } + } - def calculate(count: Double, sum: Double, sumSquares: Double): Double = + override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Entropy.calculate") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index c0407554a91b3..c923b8e8f4cf1 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -17,19 +17,27 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Class for calculating the * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] * during binary classification. */ +@Experimental object Gini extends Impurity { /** + * :: DeveloperApi :: + * * Gini coefficient calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 * @return Gini coefficient value */ + @DeveloperApi override def calculate(c0: Double, c1: Double): Double = { if (c0 == 0 || c1 == 0) { 0 @@ -41,6 +49,6 @@ object Gini extends Impurity { } } - def calculate(count: Double, sum: Double, sumSquares: Double): Double = + override def calculate(count: Double, sum: Double, sumSquares: Double): Double = throw new UnsupportedOperationException("Gini.calculate") } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index a4069063af2ad..f407796596c6c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -17,26 +17,36 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Trait for calculating information gain. */ +@Experimental trait Impurity extends Serializable { /** + * :: DeveloperApi :: + * * information calculation for binary classification * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 * @return information value */ + @DeveloperApi def calculate(c0 : Double, c1 : Double): Double /** + * :: DeveloperApi :: + * * information calculation for regression * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels * @return information value */ + @DeveloperApi def calculate(count: Double, sum: Double, sumSquares: Double): Double - } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index b74577dcec167..2c64644f4ed0f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -17,19 +17,27 @@ package org.apache.spark.mllib.tree.impurity +import org.apache.spark.annotation.{DeveloperApi, Experimental} + /** + * :: Experimental :: + * * Class for calculating variance during regression */ +@Experimental object Variance extends Impurity { override def calculate(c0: Double, c1: Double): Double = throw new UnsupportedOperationException("Variance.calculate") /** + * :: DeveloperApi :: + * * variance calculation * @param count number of instances * @param sum sum of labels * @param sumSquares summation of squares of the labels */ + @DeveloperApi override def calculate(count: Double, sum: Double, sumSquares: Double): Double = { val squaredLoss = sumSquares - (sum * sum) / count squaredLoss / count diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala index a57faa13745f7..2d71e1e366069 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Bin.scala @@ -30,4 +30,5 @@ import org.apache.spark.mllib.tree.configuration.FeatureType._ * @param featureType type of feature -- categorical or continuous * @param category categorical label value accepted in the bin */ +private[tree] case class Bin(lowSplit: Split, highSplit: Split, featureType: FeatureType, category: Double) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index a6dca84a2ce09..0f76f4a049057 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -17,15 +17,19 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.tree.configuration.Algo._ import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector /** + * :: Experimental :: + * * Model to store the decision tree parameters * @param topNode root node * @param algo algorithm type -- classification or regression */ +@Experimental class DecisionTreeModel(val topNode: Node, val algo: Algo) extends Serializable { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala index ebc9595eafef3..2deaf4ae8dcab 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Filter.scala @@ -22,7 +22,7 @@ package org.apache.spark.mllib.tree.model * @param split split specifying the feature index, type and threshold * @param comparison integer specifying <,=,> */ -case class Filter(split: Split, comparison: Int) { +private[tree] case class Filter(split: Split, comparison: Int) { // Comparison -1,0,1 signifies <.=,> override def toString = " split = " + split + "comparison = " + comparison } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index 99bf79cf12e45..d36b58e92ced6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -17,7 +17,11 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi + /** + * :: DeveloperApi :: + * * Information gain statistics for each split * @param gain information gain value * @param impurity current node impurity @@ -25,6 +29,7 @@ package org.apache.spark.mllib.tree.model * @param rightImpurity right node impurity * @param predict predicted value */ +@DeveloperApi class InformationGainStats( val gain: Double, val impurity: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index aac3f9ce308f7..339972141498c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -17,11 +17,14 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.mllib.tree.configuration.FeatureType._ import org.apache.spark.mllib.linalg.Vector /** + * :: DeveloperApi :: + * * Node in a decision tree * @param id integer node id * @param predict predicted value at the node @@ -31,6 +34,7 @@ import org.apache.spark.mllib.linalg.Vector * @param rightNode right child * @param stats information gain stats */ +@DeveloperApi class Node ( val id: Int, val predict: Double, diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 4e64a81dda74e..8bbb343079b49 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -17,20 +17,24 @@ package org.apache.spark.mllib.tree.model +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** + * :: DeveloperApi :: + * * Split applied to a feature * @param feature feature index * @param threshold threshold for continuous feature * @param featureType type of feature -- categorical or continuous * @param categories accepted values for categorical variables */ +@DeveloperApi case class Split( feature: Int, threshold: Double, featureType: FeatureType, - categories: List[Double]){ + categories: List[Double]) { override def toString = "Feature = " + feature + ", threshold = " + threshold + ", featureType = " + featureType + @@ -42,7 +46,7 @@ case class Split( * @param feature feature index * @param featureType type of feature -- categorical or continuous */ -class DummyLowSplit(feature: Int, featureType: FeatureType) +private[tree] class DummyLowSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MinValue, featureType, List()) /** @@ -50,7 +54,7 @@ class DummyLowSplit(feature: Int, featureType: FeatureType) * @param feature feature index * @param featureType type of feature -- categorical or continuous */ -class DummyHighSplit(feature: Int, featureType: FeatureType) +private[tree] class DummyHighSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MaxValue, featureType, List()) /** @@ -59,6 +63,6 @@ class DummyHighSplit(feature: Int, featureType: FeatureType) * @param feature feature index * @param featureType type of feature -- categorical or continuous */ -class DummyCategoricalSplit(feature: Int, featureType: FeatureType) +private[tree] class DummyCategoricalSplit(feature: Int, featureType: FeatureType) extends Split(feature, Double.MaxValue, featureType, List()) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 8b55bce7c4bec..230c409e1be33 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -17,23 +17,25 @@ package org.apache.spark.mllib.util +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.Logging import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint /** + * :: DeveloperApi :: + * * A collection of methods used to validate data before applying ML algorithms. */ +@DeveloperApi object DataValidators extends Logging { /** * Function to check if labels used for classification are either zero or one. * - * @param data - input data set that needs to be checked - * * @return True if labels are all zero or one, false otherwise. */ - val classificationLabels: RDD[LabeledPoint] => Boolean = { data => + val binaryLabelValidator: RDD[LabeledPoint] => Boolean = { data => val numInvalid = data.filter(x => x.label != 1.0 && x.label != 0.0).count() if (numInvalid != 0) { logError("Classification labels should be 0 or 1. Found " + numInvalid + " invalid labels") diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index 9109189dff52f..e693d13703987 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -19,15 +19,18 @@ package org.apache.spark.mllib.util import scala.util.Random +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: + * * Generate test data for KMeans. This class first chooses k cluster centers * from a d-dimensional Gaussian distribution scaled by factor r and then creates a Gaussian * cluster with scale 1 around each center. */ - +@DeveloperApi object KMeansDataGenerator { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 81e4eda2a68c4..140ff92869176 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -22,16 +22,20 @@ import scala.util.Random import org.jblas.DoubleMatrix +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** + * :: DeveloperApi :: + * * Generate sample data used for Linear Data. This class generates * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the * response variable `Y`. */ +@DeveloperApi object LinearDataGenerator { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index 61498dcc2be00..ca06b9ad58538 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -19,16 +19,19 @@ package org.apache.spark.mllib.util import scala.util.Random +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.linalg.Vectors /** + * :: DeveloperApi :: + * * Generate test data for LogisticRegression. This class chooses positive labels * with probability `probOne` and scales features for positive examples by `eps`. */ - +@DeveloperApi object LogisticRegressionDataGenerator { /** diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 348aba1dea5b6..3bd86d6813375 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -21,10 +21,13 @@ import scala.util.Random import org.jblas.DoubleMatrix +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD /** + * :: DeveloperApi :: + * * Generate RDD(s) containing data for Matrix Factorization. * * This method samples training entries according to the oversampling factor @@ -47,9 +50,8 @@ import org.apache.spark.rdd.RDD * test (Boolean) Whether to create testing RDD. * testSampFact (Double) Percentage of training data to use as test data. */ - -object MFDataGenerator{ - +@DeveloperApi +object MFDataGenerator { def main(args: Array[String]) { if (args.length < 2) { println("Usage: MFDataGenerator " + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 83d1bd3fd57fe..7f9804deaf33f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.mllib.util import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} +import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint @@ -122,6 +123,8 @@ object MLUtils { loadLibSVMData(sc, path, labelParser, numFeatures, sc.defaultMinSplits) /** + * :: Experimental :: + * * Load labeled data from a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. @@ -131,6 +134,7 @@ object MLUtils { * @return An RDD of LabeledPoint. Each labeled point has two elements: the first element is * the label, and the second element represents the feature values (an array of Double). */ + @Experimental def loadLabeledData(sc: SparkContext, dir: String): RDD[LabeledPoint] = { sc.textFile(dir).map { line => val parts = line.split(',') @@ -141,6 +145,8 @@ object MLUtils { } /** + * :: Experimental :: + * * Save labeled data to a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. @@ -148,6 +154,7 @@ object MLUtils { * @param data An RDD of LabeledPoints containing data to be saved. * @param dir Directory to save the data. */ + @Experimental def saveLabeledData(data: RDD[LabeledPoint], dir: String) { val dataStr = data.map(x => x.label + "," + x.features.toArray.mkString(" ")) dataStr.saveAsTextFile(dir) @@ -165,7 +172,7 @@ object MLUtils { * xColMean - Row vector with mean for every column (or feature) of the input data * xColSd - Row vector standard deviation for every column (or feature) of the input data. */ - def computeStats( + private[mllib] def computeStats( data: RDD[LabeledPoint], numFeatures: Int, numExamples: Long): (Double, Vector, Vector) = { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index e300c3dbe1fe0..87a6f2a0c3976 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -21,15 +21,19 @@ import scala.util.Random import org.jblas.DoubleMatrix +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vectors import org.apache.spark.mllib.regression.LabeledPoint /** + * :: DeveloperApi :: + * * Generate sample data used for SVM. This class generates uniform random values * for the features and adds Gaussian noise with weight 0.1 to generate labels. */ +@DeveloperApi object SVMDataGenerator { def main(args: Array[String]) { From eb5f2b64230faa69a53815cb61bcc87aeb233d20 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Wed, 9 Apr 2014 15:24:33 -0700 Subject: [PATCH 244/397] SPARK-1407 drain event queue before stopping event logger Author: Kan Zhang Closes #366 from kanzhang/SPARK-1407 and squashes the following commits: cd0629f [Kan Zhang] code refactoring and adding test b073ee6 [Kan Zhang] SPARK-1407 drain event queue before stopping event logger --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../spark/scheduler/LiveListenerBus.scala | 32 +++++++------ .../spark/scheduler/SparkListenerSuite.scala | 45 +++++++++++++++++++ .../apache/spark/examples/SparkHdfsLR.scala | 2 +- 4 files changed, 67 insertions(+), 16 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index f7750514ae13d..76305237b03d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -931,7 +931,6 @@ class SparkContext(config: SparkConf) extends Logging { /** Shut down the SparkContext. */ def stop() { ui.stop() - eventLogger.foreach(_.stop()) // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. val dagSchedulerCopy = dagScheduler @@ -940,13 +939,14 @@ class SparkContext(config: SparkConf) extends Logging { metadataCleaner.cancel() cleaner.foreach(_.stop()) dagSchedulerCopy.stop() - listenerBus.stop() taskScheduler = null // TODO: Cache.stop()? env.stop() SparkEnv.set(null) ShuffleMapTask.clearCache() ResultTask.clearCache() + listenerBus.stop() + eventLogger.foreach(_.stop()) logInfo("Successfully stopped SparkContext") } else { logInfo("SparkContext already stopped") diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 353a48661b0f7..76f3e327d60b8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -36,6 +36,22 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false private var started = false + private val listenerThread = new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + postToAll(event) + } + } + } + + // Exposed for testing + @volatile private[spark] var stopCalled = false /** * Start sending events to attached listeners. @@ -48,20 +64,8 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { if (started) { throw new IllegalStateException("Listener bus already started!") } + listenerThread.start() started = true - new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return - } - postToAll(event) - } - } - }.start() } def post(event: SparkListenerEvent) { @@ -93,9 +97,11 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { } def stop() { + stopCalled = true if (!started) { throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") } post(SparkListenerShutdown) + listenerThread.join() } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 7c843772bc2e0..dc704e07a81de 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.scheduler +import java.util.concurrent.Semaphore + import scala.collection.mutable import org.scalatest.{BeforeAndAfter, BeforeAndAfterAll, FunSuite} @@ -72,6 +74,49 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc } } + test("bus.stop() waits for the event queue to completely drain") { + @volatile var drained = false + + // Tells the listener to stop blocking + val listenerWait = new Semaphore(1) + + // When stop has returned + val stopReturned = new Semaphore(1) + + class BlockingListener extends SparkListener { + override def onJobEnd(jobEnd: SparkListenerJobEnd) = { + listenerWait.acquire() + drained = true + } + } + + val bus = new LiveListenerBus + val blockingListener = new BlockingListener + + bus.addListener(blockingListener) + bus.start() + bus.post(SparkListenerJobEnd(0, JobSucceeded)) + + // the queue should not drain immediately + assert(!drained) + + new Thread("ListenerBusStopper") { + override def run() { + // stop() will block until notify() is called below + bus.stop() + stopReturned.release(1) + } + }.start() + + while (!bus.stopCalled) { + Thread.sleep(10) + } + + listenerWait.release() + stopReturned.acquire() + assert(drained) + } + test("basic creation of StageInfo") { val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index e698b9bf376e1..038afbcba80a3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -73,6 +73,6 @@ object SparkHdfsLR { } println("Final w: " + w) - System.exit(0) + sc.stop() } } From 0adc932add413a1754107b21d5ecfb38c0c3a4eb Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Wed, 9 Apr 2014 17:08:17 -0700 Subject: [PATCH 245/397] [SPARK-1357 (fix)] remove empty line after :: DeveloperApi/Experimental :: Remove empty line after :: DeveloperApi/Experimental :: in comments to make the original doc show up in the preview of the generated html docs. Thanks @andrewor14 ! Author: Xiangrui Meng Closes #373 from mengxr/api and squashes the following commits: 9c35bdc [Xiangrui Meng] remove the empty line after :: DeveloperApi/Experimental :: --- .../mllib/api/python/PythonMLLibAPI.scala | 1 - .../mllib/classification/NaiveBayes.scala | 1 - .../spark/mllib/clustering/KMeans.scala | 4 -- .../linalg/distributed/CoordinateMatrix.scala | 1 - .../linalg/distributed/IndexedRowMatrix.scala | 2 - .../mllib/linalg/distributed/RowMatrix.scala | 1 - .../spark/mllib/optimization/Gradient.scala | 4 -- .../mllib/optimization/GradientDescent.scala | 2 - .../spark/mllib/optimization/Optimizer.scala | 1 - .../spark/mllib/optimization/Updater.scala | 4 -- .../spark/mllib/recommendation/ALS.scala | 1 - .../MatrixFactorizationModel.scala | 1 - .../GeneralizedLinearAlgorithm.scala | 1 - .../spark/mllib/tree/DecisionTree.scala | 1 - .../spark/mllib/tree/configuration/Algo.scala | 1 - .../tree/configuration/FeatureType.scala | 1 - .../tree/configuration/QuantileStrategy.scala | 1 - .../mllib/tree/configuration/Strategy.scala | 1 - .../spark/mllib/tree/impurity/Entropy.scala | 2 - .../spark/mllib/tree/impurity/Gini.scala | 2 - .../spark/mllib/tree/impurity/Impurity.scala | 3 -- .../spark/mllib/tree/impurity/Variance.scala | 2 - .../mllib/tree/model/DecisionTreeModel.scala | 1 - .../tree/model/InformationGainStats.scala | 1 - .../apache/spark/mllib/tree/model/Node.scala | 1 - .../apache/spark/mllib/tree/model/Split.scala | 1 - .../spark/mllib/util/DataValidators.scala | 1 - .../mllib/util/KMeansDataGenerator.scala | 1 - .../mllib/util/LinearDataGenerator.scala | 1 - .../LogisticRegressionDataGenerator.scala | 1 - .../spark/mllib/util/MFDataGenerator.scala | 43 +++++++++---------- .../org/apache/spark/mllib/util/MLUtils.scala | 2 - .../spark/mllib/util/SVMDataGenerator.scala | 1 - 33 files changed, 21 insertions(+), 71 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala index ae27c57799873..a6c049e517ee0 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/api/python/PythonMLLibAPI.scala @@ -30,7 +30,6 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * * The Java stubs necessary for the Python mllib bindings. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala index 5a45f12f1aa12..18658850a2f64 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/classification/NaiveBayes.scala @@ -29,7 +29,6 @@ import org.apache.spark.rdd.RDD /** * :: Experimental :: - * * Model for Naive Bayes Classifiers. * * @param labels list of labels diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 8f565eb60a60f..90cf8525df523 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -78,7 +78,6 @@ class KMeans private ( /** * :: Experimental :: - * * Set the number of runs of the algorithm to execute in parallel. We initialize the algorithm * this many times with random starting conditions (configured by the initialization mode), then * return the best clustering found over any run. Default: 1. @@ -398,9 +397,6 @@ object KMeans { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } - /** - * :: Experimental :: - */ @Experimental def main(args: Array[String]) { if (args.length < 4) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala index 89d5c03d76c42..56b8fdcda66eb 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/CoordinateMatrix.scala @@ -34,7 +34,6 @@ case class MatrixEntry(i: Long, j: Long, value: Double) /** * :: Experimental :: - * * Represents a matrix in coordinate format. * * @param entries matrix entries diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala index 24c123ab7eb51..132b3af72d9ce 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/IndexedRowMatrix.scala @@ -26,7 +26,6 @@ import org.apache.spark.mllib.linalg.SingularValueDecomposition /** * :: Experimental :: - * * Represents a row of [[org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix]]. */ @Experimental @@ -34,7 +33,6 @@ case class IndexedRow(index: Long, vector: Vector) /** * :: Experimental :: - * * Represents a row-oriented [[org.apache.spark.mllib.linalg.distributed.DistributedMatrix]] with * indexed rows. * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index 8d32c1a6dbba1..f65f43dd3007b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -30,7 +30,6 @@ import org.apache.spark.Logging /** * :: Experimental :: - * * Represents a row-oriented distributed Matrix with no meaningful row indices. * * @param rows rows stored as an RDD[Vector] diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala index 1176dc9dbc08d..679842f831c2a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Gradient.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: - * * Class used to compute the gradient for a loss function, given a single data point. */ @DeveloperApi @@ -56,7 +55,6 @@ abstract class Gradient extends Serializable { /** * :: DeveloperApi :: - * * Compute gradient and loss for a logistic loss function, as used in binary classification. * See also the documentation for the precise formulation. */ @@ -100,7 +98,6 @@ class LogisticGradient extends Gradient { /** * :: DeveloperApi :: - * * Compute gradient and loss for a Least-squared loss function, as used in linear regression. * This is correct for the averaged least squares loss function (mean squared error) * L = 1/n ||A weights-y||^2 @@ -135,7 +132,6 @@ class LeastSquaresGradient extends Gradient { /** * :: DeveloperApi :: - * * Compute gradient and loss for a Hinge loss function, as used in SVM binary classification. * See also the documentation for the precise formulation. * NOTE: This assumes that the labels are {0,1} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala index 04267d967dcad..f60417f21d4b9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/GradientDescent.scala @@ -28,7 +28,6 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: - * * Class used to solve an optimization problem using Gradient Descent. * @param gradient Gradient function to be used. * @param updater Updater to be used to update weights after every iteration. @@ -113,7 +112,6 @@ class GradientDescent(private var gradient: Gradient, private var updater: Updat /** * :: DeveloperApi :: - * * Top-level method to run gradient descent. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index 0a313f3104b14..e41d9bbe18c37 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.Vector /** * :: DeveloperApi :: - * * Trait for optimization problem solvers. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala index e67816796c6b1..3ed3a5b9b3843 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Updater.scala @@ -26,7 +26,6 @@ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** * :: DeveloperApi :: - * * Class used to perform steps (weight update) using Gradient Descent methods. * * For general minimization problems, or for regularized problems of the form @@ -64,7 +63,6 @@ abstract class Updater extends Serializable { /** * :: DeveloperApi :: - * * A simple updater for gradient descent *without* any regularization. * Uses a step-size decreasing with the square root of the number of iterations. */ @@ -86,7 +84,6 @@ class SimpleUpdater extends Updater { /** * :: DeveloperApi :: - * * Updater for L1 regularized problems. * R(w) = ||w||_1 * Uses a step-size decreasing with the square root of the number of iterations. @@ -131,7 +128,6 @@ class L1Updater extends Updater { /** * :: DeveloperApi :: - * * Updater for L2 regularized problems. * R(w) = 1/2 ||w||^2 * Uses a step-size decreasing with the square root of the number of iterations. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 60cbb1c1e1d86..5cc47de8ffdfc 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -140,7 +140,6 @@ class ALS private ( /** * :: Experimental :: - * * Sets the constant used in computing confidence in implicit ALS. Default: 1.0. */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala index e05224fc7caf2..471546cd82c7d 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/MatrixFactorizationModel.scala @@ -70,7 +70,6 @@ class MatrixFactorizationModel( /** * :: DeveloperApi :: - * * Predict the rating of many users for many products. * This is a Java stub for python predictAll() * diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index c24f5afb99686..3bd0017aa196a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -103,7 +103,6 @@ abstract class GeneralizedLinearAlgorithm[M <: GeneralizedLinearModel] /** * :: Experimental :: - * * Set if the algorithm should validate data before training. Default true. */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index c8a966cd5f5a8..3019447ce4cd9 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -35,7 +35,6 @@ import org.apache.spark.mllib.linalg.{Vector, Vectors} /** * :: Experimental :: - * * A class that implements a decision tree algorithm for classification and regression. It * supports both continuous and categorical features. * @param strategy The configuration parameters for the tree algorithm which specify the type diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala index 017f84f3b9e8b..79a01f58319e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Algo.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * * Enum to select the algorithm for the decision tree */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala index c0254c32c2dce..f4c877232750f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/FeatureType.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * * Enum to describe whether a feature is "continuous" or "categorical" */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala index b3e8b224beeaa..7da976e55a722 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/QuantileStrategy.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.Experimental /** * :: Experimental :: - * * Enum for selecting the quantile calculation strategy */ @Experimental diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala index 482faaa9e7256..8767aca47cd5a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/configuration/Strategy.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.tree.configuration.QuantileStrategy._ /** * :: Experimental :: - * * Stores all the configuration options for tree construction * @param algo classification or regression * @param impurity criterion used for information gain calculation diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala index 55c43f2fcf9c5..60f43e9278d2a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Entropy.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Class for calculating [[http://en.wikipedia.org/wiki/Binary_entropy_function entropy]] during * binary classification. */ @@ -32,7 +31,6 @@ object Entropy extends Impurity { /** * :: DeveloperApi :: - * * entropy calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala index c923b8e8f4cf1..c51d76d9b4c5b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Gini.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Class for calculating the * [[http://en.wikipedia.org/wiki/Decision_tree_learning#Gini_impurity Gini impurity]] * during binary classification. @@ -31,7 +30,6 @@ object Gini extends Impurity { /** * :: DeveloperApi :: - * * Gini coefficient calculation * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala index f407796596c6c..8eab247cf0932 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Impurity.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Trait for calculating information gain. */ @Experimental @@ -29,7 +28,6 @@ trait Impurity extends Serializable { /** * :: DeveloperApi :: - * * information calculation for binary classification * @param c0 count of instances with label 0 * @param c1 count of instances with label 1 @@ -40,7 +38,6 @@ trait Impurity extends Serializable { /** * :: DeveloperApi :: - * * information calculation for regression * @param count number of instances * @param sum sum of labels diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala index 2c64644f4ed0f..47d07122af30f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/impurity/Variance.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.{DeveloperApi, Experimental} /** * :: Experimental :: - * * Class for calculating variance during regression */ @Experimental @@ -31,7 +30,6 @@ object Variance extends Impurity { /** * :: DeveloperApi :: - * * variance calculation * @param count number of instances * @param sum sum of labels diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala index 0f76f4a049057..bf692ca8c4bd7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/DecisionTreeModel.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.Vector /** * :: Experimental :: - * * Model to store the decision tree parameters * @param topNode root node * @param algo algorithm type -- classification or regression diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala index d36b58e92ced6..cc8a24cce9614 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/InformationGainStats.scala @@ -21,7 +21,6 @@ import org.apache.spark.annotation.DeveloperApi /** * :: DeveloperApi :: - * * Information gain statistics for each split * @param gain information gain value * @param impurity current node impurity diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala index 339972141498c..682f213f411a7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Node.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.linalg.Vector /** * :: DeveloperApi :: - * * Node in a decision tree * @param id integer node id * @param predict predicted value at the node diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala index 8bbb343079b49..d7ffd386c05ee 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/model/Split.scala @@ -22,7 +22,6 @@ import org.apache.spark.mllib.tree.configuration.FeatureType.FeatureType /** * :: DeveloperApi :: - * * Split applied to a feature * @param feature feature index * @param threshold threshold for continuous feature diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala index 230c409e1be33..45f95482a1def 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/DataValidators.scala @@ -24,7 +24,6 @@ import org.apache.spark.mllib.regression.LabeledPoint /** * :: DeveloperApi :: - * * A collection of methods used to validate data before applying ML algorithms. */ @DeveloperApi diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala index e693d13703987..6eaebaf7dba9f 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/KMeansDataGenerator.scala @@ -25,7 +25,6 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: - * * Generate test data for KMeans. This class first chooses k cluster centers * from a d-dimensional Gaussian distribution scaled by factor r and then creates a Gaussian * cluster with scale 1 around each center. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala index 140ff92869176..c8e160d00c2d6 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LinearDataGenerator.scala @@ -30,7 +30,6 @@ import org.apache.spark.mllib.regression.LabeledPoint /** * :: DeveloperApi :: - * * Generate sample data used for Linear Data. This class generates * uniformly random values for every feature and adds Gaussian noise with mean `eps` to the * response variable `Y`. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala index ca06b9ad58538..c82cd8fd4641c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/LogisticRegressionDataGenerator.scala @@ -27,7 +27,6 @@ import org.apache.spark.mllib.linalg.Vectors /** * :: DeveloperApi :: - * * Generate test data for LogisticRegression. This class chooses positive labels * with probability `probOne` and scales features for positive examples by `eps`. */ diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 3bd86d6813375..3f413faca6bb4 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -27,29 +27,28 @@ import org.apache.spark.rdd.RDD /** * :: DeveloperApi :: + * Generate RDD(s) containing data for Matrix Factorization. * -* Generate RDD(s) containing data for Matrix Factorization. -* -* This method samples training entries according to the oversampling factor -* 'trainSampFact', which is a multiplicative factor of the number of -* degrees of freedom of the matrix: rank*(m+n-rank). -* -* It optionally samples entries for a testing matrix using -* 'testSampFact', the percentage of the number of training entries -* to use for testing. -* -* This method takes the following inputs: -* sparkMaster (String) The master URL. -* outputPath (String) Directory to save output. -* m (Int) Number of rows in data matrix. -* n (Int) Number of columns in data matrix. -* rank (Int) Underlying rank of data matrix. -* trainSampFact (Double) Oversampling factor. -* noise (Boolean) Whether to add gaussian noise to training data. -* sigma (Double) Standard deviation of added gaussian noise. -* test (Boolean) Whether to create testing RDD. -* testSampFact (Double) Percentage of training data to use as test data. -*/ + * This method samples training entries according to the oversampling factor + * 'trainSampFact', which is a multiplicative factor of the number of + * degrees of freedom of the matrix: rank*(m+n-rank). + * + * It optionally samples entries for a testing matrix using + * 'testSampFact', the percentage of the number of training entries + * to use for testing. + * + * This method takes the following inputs: + * sparkMaster (String) The master URL. + * outputPath (String) Directory to save output. + * m (Int) Number of rows in data matrix. + * n (Int) Number of columns in data matrix. + * rank (Int) Underlying rank of data matrix. + * trainSampFact (Double) Oversampling factor. + * noise (Boolean) Whether to add gaussian noise to training data. + * sigma (Double) Standard deviation of added gaussian noise. + * test (Boolean) Whether to create testing RDD. + * testSampFact (Double) Percentage of training data to use as test data. + */ @DeveloperApi object MFDataGenerator { def main(args: Array[String]) { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index 7f9804deaf33f..ac2360c429e2b 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -124,7 +124,6 @@ object MLUtils { /** * :: Experimental :: - * * Load labeled data from a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. @@ -146,7 +145,6 @@ object MLUtils { /** * :: Experimental :: - * * Save labeled data to a file. The data format used here is * , ... * where , are feature values in Double and is the corresponding label as Double. diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala index 87a6f2a0c3976..ba8190b0e07e8 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/SVMDataGenerator.scala @@ -29,7 +29,6 @@ import org.apache.spark.mllib.regression.LabeledPoint /** * :: DeveloperApi :: - * * Generate sample data used for SVM. This class generates uniform random values * for the features and adds Gaussian noise with weight 0.1 to generate labels. */ From 8ca3b2bc90a63b23a03f339e390174cd7a672b40 Mon Sep 17 00:00:00 2001 From: William Benton Date: Wed, 9 Apr 2014 18:56:27 -0700 Subject: [PATCH 246/397] SPARK-729: Closures not always serialized at capture time [SPARK-729](https://spark-project.atlassian.net/browse/SPARK-729) concerns when free variables in closure arguments to transformations are captured. Currently, it is possible for closures to get the environment in which they are serialized (not the environment in which they are created). There are a few possible approaches to solving this problem and this PR will discuss some of them. The approach I took has the advantage of being simple, obviously correct, and minimally-invasive, but it preserves something that has been bothering me about Spark's closure handling, so I'd like to discuss an alternative and get some feedback on whether or not it is worth pursuing. ## What I did The basic approach I took depends on the work I did for #143, and so this PR is based atop that. Specifically: #143 modifies `ClosureCleaner.clean` to preemptively determine whether or not closures are serializable immediately upon closure cleaning (rather than waiting for an job involving that closure to be scheduled). Thus non-serializable closure exceptions will be triggered by the line defining the closure rather than triggered where the closure is used. Since the easiest way to determine whether or not a closure is serializable is to attempt to serialize it, the code in #143 is creating a serialized closure as part of `ClosureCleaner.clean`. `clean` currently modifies its argument, but the method in `SparkContext` that wraps it to return a value (a reference to the modified-in-place argument). This branch modifies `ClosureCleaner.clean` so that it returns a value: if it is cleaning a serializable closure, it returns the result of deserializing its serialized argument; therefore it is returning a closure with an environment captured at cleaning time. `SparkContext.clean` then returns the result of `ClosureCleaner.clean`, rather than a reference to its modified-in-place argument. I've added tests for this behavior (777a1bc). The pull request as it stands, given the changes in #143, is nearly trivial. There is some overhead from deserializing the closure, but it is minimal and the benefit of obvious operational correctness (vs. a more sophisticated but harder-to-validate transformation in `ClosureCleaner`) seems pretty important. I think this is a fine way to solve this problem, but it's not perfect. ## What we might want to do The thing that has been bothering me about Spark's handling of closures is that it seems like we should be able to statically ensure that cleaning and serialization happen exactly once for a given closure. If we serialize a closure in order to determine whether or not it is serializable, we should be able to hang on to the generated byte buffer and use it instead of re-serializing the closure later. By replacing closures with instances of a sum type that encodes whether or not a closure has been cleaned or serialized, we could handle clean, to-be-cleaned, and serialized closures separately with case matches. Here's a somewhat-concrete sketch (taken from my git stash) of what this might look like: ```scala package org.apache.spark.util import java.nio.ByteBuffer import scala.reflect.ClassManifest sealed abstract class ClosureBox[T] { def func: T } final case class RawClosure[T](func: T) extends ClosureBox[T] {} final case class CleanedClosure[T](func: T) extends ClosureBox[T] {} final case class SerializedClosure[T](func: T, bytebuf: ByteBuffer) extends ClosureBox[T] {} object ClosureBoxImplicits { implicit def closureBoxFromFunc[T <: AnyRef](fun: T) = new RawClosure[T](fun) } ``` With these types declared, we'd be able to change `ClosureCleaner.clean` to take a `ClosureBox[T=>U]` (possibly generated by implicit conversion) and return a `ClosureBox[T=>U]` (either a `CleanedClosure[T=>U]` or a `SerializedClosure[T=>U]`, depending on whether or not serializability-checking was enabled) instead of a `T=>U`. A case match could thus short-circuit cleaning or serializing closures that had already been cleaned or serialized (both in `ClosureCleaner` and in the closure serializer). Cleaned-and-serialized closures would be represented by a boxed tuple of the original closure and a serialized copy (complete with an environment quiesced at transformation time). Additional implicit conversions could convert from `ClosureBox` instances to the underlying function type where appropriate. Tracking this sort of state in the type system seems like the right thing to do to me. ### Why we might not want to do that _It's pretty invasive._ Every function type used by every `RDD` subclass would have to change to reflect that they expected a `ClosureBox[T=>U]` instead of a `T=>U`. This obscures what's going on and is not a little ugly. Although I really like the idea of using the type system to enforce the clean-or-serialize once discipline, it might not be worth adding another layer of types (even if we could hide some of the extra boilerplate with judicious application of implicit conversions). _It statically guarantees a property whose absence is unlikely to cause any serious problems as it stands._ It appears that all closures are currently dynamically cleaned once and it's not obvious that repeated closure-cleaning is likely to be a problem in the future. Furthermore, serializing closures is relatively cheap, so doing it once to check for serialization and once again to actually ship them across the wire doesn't seem like a big deal. Taken together, these seem like a high price to pay for statically guaranteeing that closures are operated upon only once. ## Other possibilities I felt like the serialize-and-deserialize approach was best due to its obvious simplicity. But it would be possible to do a more sophisticated transformation within `ClosureCleaner.clean`. It might also be possible for `clean` to modify its argument in a way so that whether or not a given closure had been cleaned would be apparent upon inspection; this would buy us some of the operational benefits of the `ClosureBox` approach but not the static cleanliness. I'm interested in any feedback or discussion on whether or not the problems with the type-based approach indeed outweigh the advantage, as well as of approaches to this issue and to closure handling in general. Author: William Benton Closes #189 from willb/spark-729 and squashes the following commits: f4cafa0 [William Benton] Stylistic changes and cleanups b3d9c86 [William Benton] Fixed style issues in tests 9b56ce0 [William Benton] Added array-element capture test 97e9d91 [William Benton] Split closure-serializability failure tests 12ef6e3 [William Benton] Skip proactive closure capture for runJob 8ee3ee7 [William Benton] Predictable closure environment capture 12c63a7 [William Benton] Added tests for variable capture in closures d6e8dd6 [William Benton] Don't check serializability of DStream transforms. 4ecf841 [William Benton] Make proactive serializability checking optional. d8df3db [William Benton] Adds proactive closure-serializablilty checking 21b4b06 [William Benton] Test cases for SPARK-897. d5947b3 [William Benton] Ensure assertions in Graph.apply are asserted. --- .../scala/org/apache/spark/SparkContext.scala | 16 +++- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../apache/spark/util/ClosureCleaner.scala | 21 ++++- .../scala/org/apache/spark/FailureSuite.scala | 17 +++- .../ProactiveClosureSerializationSuite.scala | 94 +++++++++++++++++++ .../spark/util/ClosureCleanerSuite.scala | 68 ++++++++++++++ .../org/apache/spark/graphx/GraphSuite.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 8 +- 8 files changed, 218 insertions(+), 14 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 76305237b03d5..545807ffbce55 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1002,7 +1002,9 @@ class SparkContext(config: SparkConf) extends Logging { require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") } val callSite = getCallSite - val cleanedFunc = clean(func) + // There's no need to check this function for serializability, + // since it will be run right away. + val cleanedFunc = clean(func, false) logInfo("Starting job: " + callSite) val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, @@ -1135,14 +1137,18 @@ class SparkContext(config: SparkConf) extends Logging { def cancelAllJobs() { dagScheduler.cancelAllJobs() } - + /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) + * + * @param f closure to be cleaned and optionally serialized + * @param captureNow whether or not to serialize this closure and capture any free + * variables immediately; defaults to true. If this is set and f is not serializable, + * it will raise an exception. */ - private[spark] def clean[F <: AnyRef](f: F): F = { - ClosureCleaner.clean(f) - f + private[spark] def clean[F <: AnyRef : ClassTag](f: F, captureNow: Boolean = true): F = { + ClosureCleaner.clean(f, captureNow) } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3437b2cac19c2..e363ea777d8eb 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -660,14 +660,16 @@ abstract class RDD[T: ClassTag]( * Applies a function f to all elements of this RDD. */ def foreach(f: T => Unit) { - sc.runJob(this, (iter: Iterator[T]) => iter.foreach(f)) + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) } /** * Applies a function f to each partition of this RDD. */ def foreachPartition(f: Iterator[T] => Unit) { - sc.runJob(this, (iter: Iterator[T]) => f(iter)) + val cleanF = sc.clean(f) + sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index cdbbc65292188..e474b1a850d65 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -22,10 +22,14 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.Map import scala.collection.mutable.Set +import scala.reflect.ClassTag + import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ import org.apache.spark.Logging +import org.apache.spark.SparkEnv +import org.apache.spark.SparkException private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -101,7 +105,7 @@ private[spark] object ClosureCleaner extends Logging { } } - def clean(func: AnyRef) { + def clean[F <: AnyRef : ClassTag](func: F, captureNow: Boolean = true): F = { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) @@ -150,6 +154,21 @@ private[spark] object ClosureCleaner extends Logging { field.setAccessible(true) field.set(func, outer) } + + if (captureNow) { + cloneViaSerializing(func) + } else { + func + } + } + + private def cloneViaSerializing[T: ClassTag](func: T): T = { + try { + val serializer = SparkEnv.get.closureSerializer.newInstance() + serializer.deserialize[T](serializer.serialize[T](func)) + } catch { + case ex: Exception => throw new SparkException("Task not serializable: " + ex.toString) + } } private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 12dbebcb28644..4f9300419e6f8 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -107,7 +107,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } - test("failure because task closure is not serializable") { + test("failure because closure in final-stage task is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable @@ -118,6 +118,13 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() + } + + test("failure because closure in early-stage task is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() @@ -125,6 +132,13 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) + FailureSuiteState.clear() + } + + test("failure because closure in foreach task is not serializable") { + sc = new SparkContext("local[1,1]", "test") + val a = new NonSerializable + // Non-serializable closure in foreach function val thrown2 = intercept[SparkException] { sc.parallelize(1 to 10, 2).foreach(x => println(a)) @@ -135,5 +149,6 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } + // TODO: Need to add tests with shuffle fetch failures. } diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala new file mode 100644 index 0000000000000..76662264e7e94 --- /dev/null +++ b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala @@ -0,0 +1,94 @@ +/* + * 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.serializer; + +import java.io.NotSerializableException + +import org.scalatest.FunSuite + +import org.apache.spark.rdd.RDD +import org.apache.spark.SparkException +import org.apache.spark.SharedSparkContext + +/* A trivial (but unserializable) container for trivial functions */ +class UnserializableClass { + def op[T](x: T) = x.toString + + def pred[T](x: T) = x.toString.length % 2 == 0 +} + +class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { + + def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) + + test("throws expected serialization exceptions on actions") { + val (data, uc) = fixture + + val ex = intercept[SparkException] { + data.map(uc.op(_)).count + } + + assert(ex.getMessage.matches(".*Task not serializable.*")) + } + + // There is probably a cleaner way to eliminate boilerplate here, but we're + // iterating over a map from transformation names to functions that perform that + // transformation on a given RDD, creating one test case for each + + for (transformation <- + Map("map" -> map _, "flatMap" -> flatMap _, "filter" -> filter _, "mapWith" -> mapWith _, + "mapPartitions" -> mapPartitions _, "mapPartitionsWithIndex" -> mapPartitionsWithIndex _, + "mapPartitionsWithContext" -> mapPartitionsWithContext _, "filterWith" -> filterWith _)) { + val (name, xf) = transformation + + test(s"$name transformations throw proactive serialization exceptions") { + val (data, uc) = fixture + + val ex = intercept[SparkException] { + xf(data, uc) + } + + assert(ex.getMessage.matches(".*Task not serializable.*"), s"RDD.$name doesn't proactively throw NotSerializableException") + } + } + + def map(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.map(y => uc.op(y)) + + def mapWith(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapWith(x => x.toString)((x,y) => x + uc.op(y)) + + def flatMap(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.flatMap(y=>Seq(uc.op(y))) + + def filter(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.filter(y=>uc.pred(y)) + + def filterWith(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.filterWith(x => x.toString)((x,y) => uc.pred(y)) + + def mapPartitions(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapPartitions(_.map(y => uc.op(y))) + + def mapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapPartitionsWithIndex((_, it) => it.map(y => uc.op(y))) + + def mapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = + x.mapPartitionsWithContext((_, it) => it.map(y => uc.op(y))) + +} diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 439e5644e20a3..c635da6cacd70 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -50,6 +50,27 @@ class ClosureCleanerSuite extends FunSuite { val obj = new TestClassWithNesting(1) assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 } + + test("capturing free variables in closures at RDD definition") { + val obj = new TestCaptureVarClass() + val (ones, onesPlusZeroes) = obj.run() + + assert(ones === onesPlusZeroes) + } + + test("capturing free variable fields in closures at RDD definition") { + val obj = new TestCaptureFieldClass() + val (ones, onesPlusZeroes) = obj.run() + + assert(ones === onesPlusZeroes) + } + + test("capturing arrays in closures at RDD definition") { + val obj = new TestCaptureArrayEltClass() + val (observed, expected) = obj.run() + + assert(observed === expected) + } } // A non-serializable class we create in closures to make sure that we aren't @@ -143,3 +164,50 @@ class TestClassWithNesting(val y: Int) extends Serializable { } } } + +class TestCaptureFieldClass extends Serializable { + class ZeroBox extends Serializable { + var zero = 0 + } + + def run(): (Int, Int) = { + val zb = new ZeroBox + + withSpark(new SparkContext("local", "test")) {sc => + val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) + val onesPlusZeroes = ones.map(_ + zb.zero) + + zb.zero = 5 + + (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) + } + } +} + +class TestCaptureArrayEltClass extends Serializable { + def run(): (Int, Int) = { + withSpark(new SparkContext("local", "test")) {sc => + val rdd = sc.parallelize(1 to 10) + val data = Array(1, 2, 3) + val expected = data(0) + val mapped = rdd.map(x => data(0)) + data(0) = 4 + (mapped.first, expected) + } + } +} + +class TestCaptureVarClass extends Serializable { + def run(): (Int, Int) = { + var zero = 0 + + withSpark(new SparkContext("local", "test")) {sc => + val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) + val onesPlusZeroes = ones.map(_ + zero) + + zero = 5 + + (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) + } + } +} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index 28d34dd9a1a41..c65e36636fe10 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -62,7 +62,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) - graph.triplets.map { et => + graph.triplets.collect.map { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d043200f71a0b..4759b629a9931 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -539,7 +539,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = { - transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r))) + transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r), false)) } /** @@ -547,7 +547,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { - val cleanedF = context.sparkContext.clean(transformFunc) + val cleanedF = context.sparkContext.clean(transformFunc, false) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 1) cleanedF(rdds.head.asInstanceOf[RDD[T]], time) @@ -562,7 +562,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc) + val cleanedF = ssc.sparkContext.clean(transformFunc, false) transformWith(other, (rdd1: RDD[T], rdd2: RDD[U], time: Time) => cleanedF(rdd1, rdd2)) } @@ -573,7 +573,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc) + val cleanedF = ssc.sparkContext.clean(transformFunc, false) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 2) val rdd1 = rdds(0).asInstanceOf[RDD[T]] From e55cc4bae52a3de728939244780abc662713b768 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 10 Apr 2014 00:37:21 -0700 Subject: [PATCH 247/397] SPARK-1446: Spark examples should not do a System.exit Spark examples should exit nice using SparkContext.stop() method, rather than System.exit System.exit can cause issues like in SPARK-1407 Author: Sandeep Closes #370 from techaddict/1446 and squashes the following commits: e9234cf [Sandeep] SPARK-1446: Spark examples should not do a System.exit Spark examples should exit nice using SparkContext.stop() method, rather than System.exit System.exit can cause issues like in SPARK-1407 --- .../org/apache/spark/examples/JavaHdfsLR.java | 2 +- .../apache/spark/examples/JavaLogQuery.java | 2 +- .../apache/spark/examples/JavaPageRank.java | 2 +- .../org/apache/spark/examples/JavaTC.java | 2 +- .../apache/spark/examples/JavaWordCount.java | 6 +++--- .../apache/spark/mllib/examples/JavaALS.java | 2 +- .../spark/mllib/examples/JavaKMeans.java | 2 +- .../apache/spark/mllib/examples/JavaLR.java | 2 +- .../apache/spark/examples/BroadcastTest.scala | 2 +- .../spark/examples/CassandraCQLTest.scala | 2 +- .../examples/ExceptionHandlingTest.scala | 2 +- .../apache/spark/examples/GroupByTest.scala | 5 ++--- .../org/apache/spark/examples/HBaseTest.scala | 6 +++--- .../org/apache/spark/examples/HdfsTest.scala | 2 +- .../org/apache/spark/examples/LocalALS.scala | 2 +- .../spark/examples/MultiBroadcastTest.scala | 2 +- .../examples/SimpleSkewedGroupByTest.scala | 7 +++---- .../spark/examples/SkewedGroupByTest.scala | 7 +++---- .../org/apache/spark/examples/SparkALS.scala | 4 ++-- .../apache/spark/examples/SparkHdfsLR.scala | 2 +- .../apache/spark/examples/SparkKMeans.scala | 20 +++++++++---------- .../org/apache/spark/examples/SparkLR.scala | 2 +- .../apache/spark/examples/SparkPageRank.scala | 3 +-- .../org/apache/spark/examples/SparkTC.scala | 2 +- .../spark/examples/SparkTachyonHdfsLR.scala | 2 +- .../spark/examples/SparkTachyonPi.scala | 6 +++--- .../bagel/WikipediaPageRankStandalone.scala | 2 +- .../spark/examples/mllib/TallSkinnyPCA.scala | 2 +- .../spark/examples/mllib/TallSkinnySVD.scala | 2 +- .../streaming/examples/HdfsWordCount.scala | 1 - .../streaming/examples/KafkaWordCount.scala | 5 ++--- .../streaming/examples/MQTTWordCount.scala | 2 +- .../streaming/examples/QueueStream.scala | 11 +++++----- .../examples/StatefulNetworkWordCount.scala | 2 +- .../streaming/examples/ZeroMQWordCount.scala | 2 +- 35 files changed, 60 insertions(+), 67 deletions(-) diff --git a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java index 6b49244ba459d..bd96274021756 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaHdfsLR.java @@ -138,6 +138,6 @@ public static void main(String[] args) { System.out.print("Final w: "); printWeights(w); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java index 617e4a6d045e0..2a4278d3c30e5 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaLogQuery.java @@ -126,6 +126,6 @@ public Stats call(Stats stats, Stats stats2) { for (Tuple2 t : output) { System.out.println(t._1() + "\t" + t._2()); } - System.exit(0); + jsc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java index 8513ba07e7705..e31f676f5fd4c 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaPageRank.java @@ -117,6 +117,6 @@ public Double call(Double sum) { System.out.println(tuple._1() + " has rank: " + tuple._2() + "."); } - System.exit(0); + ctx.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaTC.java b/examples/src/main/java/org/apache/spark/examples/JavaTC.java index 6cfe25c80ecc6..1d776940f06c6 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaTC.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaTC.java @@ -96,6 +96,6 @@ public Tuple2 call(Tuple2 e) { } while (nextCount != oldCount); System.out.println("TC has " + tc.count() + " edges."); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java index 3ae1d8f7ca938..87c1b80981961 100644 --- a/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java +++ b/examples/src/main/java/org/apache/spark/examples/JavaWordCount.java @@ -48,14 +48,14 @@ public Iterable call(String s) { return Arrays.asList(SPACE.split(s)); } }); - + JavaPairRDD ones = words.mapToPair(new PairFunction() { @Override public Tuple2 call(String s) { return new Tuple2(s, 1); } }); - + JavaPairRDD counts = ones.reduceByKey(new Function2() { @Override public Integer call(Integer i1, Integer i2) { @@ -67,6 +67,6 @@ public Integer call(Integer i1, Integer i2) { for (Tuple2 tuple : output) { System.out.println(tuple._1() + ": " + tuple._2()); } - System.exit(0); + ctx.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java index 64a3a04fb7296..c516199d61c72 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaALS.java @@ -85,6 +85,6 @@ public static void main(String[] args) { outputDir + "/productFeatures"); System.out.println("Final user/product features written to " + outputDir); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java index 7b0ec36424e97..7461609ab9e8f 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaKMeans.java @@ -79,6 +79,6 @@ public static void main(String[] args) { double cost = model.computeCost(points.rdd()); System.out.println("Cost: " + cost); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java index cd8879ff886e2..e3ab87cc722f3 100644 --- a/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java +++ b/examples/src/main/java/org/apache/spark/mllib/examples/JavaLR.java @@ -77,6 +77,6 @@ public static void main(String[] args) { System.out.print("Final w: " + model.weights()); - System.exit(0); + sc.stop(); } } diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index 4d2f45df85fc6..c8c916bb45e00 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -56,6 +56,6 @@ object BroadcastTest { println("Iteration %d took %.0f milliseconds".format(i, (System.nanoTime - startTime) / 1E6)) } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index ee283ce6abac2..1f8d7cb5995b8 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -58,7 +58,7 @@ import org.apache.spark.SparkContext._ prod_id, quantity) VALUES ('charlie', 1385983649000, 'iphone', 2); */ - + /** * This example demonstrates how to read and write to cassandra column family created using CQL3 * using Spark. diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index fdb976dfc6aba..be7d39549a28d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -34,6 +34,6 @@ object ExceptionHandlingTest { } } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 36534e59353cd..29114c6dabcdb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -28,7 +28,7 @@ object GroupByTest { "Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") System.exit(1) } - + var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 var valSize = if (args.length > 3) args(3).toInt else 1000 @@ -52,7 +52,6 @@ object GroupByTest { println(pairs1.groupByKey(numReducers).count) - System.exit(0) + sc.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index 65d67356be2f6..700121d16dd60 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -30,7 +30,7 @@ object HBaseTest { val conf = HBaseConfiguration.create() - // Other options for configuring scan behavior are available. More information available at + // Other options for configuring scan behavior are available. More information available at // http://hbase.apache.org/apidocs/org/apache/hadoop/hbase/mapreduce/TableInputFormat.html conf.set(TableInputFormat.INPUT_TABLE, args(1)) @@ -41,12 +41,12 @@ object HBaseTest { admin.createTable(tableDesc) } - val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], + val hBaseRDD = sc.newAPIHadoopRDD(conf, classOf[TableInputFormat], classOf[org.apache.hadoop.hbase.io.ImmutableBytesWritable], classOf[org.apache.hadoop.hbase.client.Result]) hBaseRDD.count() - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index c3597d94a224e..dd6d5205133be 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -32,6 +32,6 @@ object HdfsTest { val end = System.currentTimeMillis() println("Iteration " + iter + " took " + (end-start) + " ms") } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala index 0095cb8425456..37ad4bd0999bd 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LocalALS.scala @@ -120,7 +120,7 @@ object LocalALS { } } printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) - + val R = generateR() // Initialize m and u randomly diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 4aef04fc060b6..97321ab8f41db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -51,6 +51,6 @@ object MultiBroadcastTest { // Collect the small RDD so we can print the observed sizes locally. observedSizes.collect().foreach(i => println(i)) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 1fdb324b89f3a..d05eedd31caa0 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -27,7 +27,7 @@ object SimpleSkewedGroupByTest { System.err.println("Usage: SimpleSkewedGroupByTest " + "[numMappers] [numKVPairs] [valSize] [numReducers] [ratio]") System.exit(1) - } + } var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 @@ -58,14 +58,13 @@ object SimpleSkewedGroupByTest { }.cache // Enforce that everything has been calculated and in cache pairs1.count - + println("RESULT: " + pairs1.groupByKey(numReducers).count) // Print how many keys each reducer got (for debugging) // println("RESULT: " + pairs1.groupByKey(numReducers) // .map{case (k,v) => (k, v.size)} // .collectAsMap) - System.exit(0) + sc.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 966478fe4a258..fd9f043247d18 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -27,7 +27,7 @@ object SkewedGroupByTest { System.err.println( "Usage: GroupByTest [numMappers] [numKVPairs] [KeySize] [numReducers]") System.exit(1) - } + } var numMappers = if (args.length > 1) args(1).toInt else 2 var numKVPairs = if (args.length > 2) args(2).toInt else 1000 @@ -53,10 +53,9 @@ object SkewedGroupByTest { }.cache() // Enforce that everything has been calculated and in cache pairs1.count() - + println(pairs1.groupByKey(numReducers).count()) - System.exit(0) + sc.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index f59ab7e7cc24a..68f151a2c47fe 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -112,7 +112,7 @@ object SparkALS { val sc = new SparkContext(host, "SparkALS", System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - + val R = generateR() // Initialize m and u randomly @@ -137,6 +137,6 @@ object SparkALS { println() } - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index 038afbcba80a3..d8de8745c15d9 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -52,7 +52,7 @@ object SparkHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 9fe24652358f3..1a8b21618e23a 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -28,16 +28,16 @@ import org.apache.spark.SparkContext._ object SparkKMeans { val R = 1000 // Scaling factor val rand = new Random(42) - + def parseVector(line: String): Vector = { new Vector(line.split(' ').map(_.toDouble)) } - + def closestPoint(p: Vector, centers: Array[Vector]): Int = { var index = 0 var bestIndex = 0 var closest = Double.PositiveInfinity - + for (i <- 0 until centers.length) { val tempDist = p.squaredDist(centers(i)) if (tempDist < closest) { @@ -45,7 +45,7 @@ object SparkKMeans { bestIndex = i } } - + bestIndex } @@ -60,22 +60,22 @@ object SparkKMeans { val data = lines.map(parseVector _).cache() val K = args(2).toInt val convergeDist = args(3).toDouble - + val kPoints = data.takeSample(withReplacement = false, K, 42).toArray var tempDist = 1.0 while(tempDist > convergeDist) { val closest = data.map (p => (closestPoint(p, kPoints), (p, 1))) - + val pointStats = closest.reduceByKey{case ((x1, y1), (x2, y2)) => (x1 + x2, y1 + y2)} - + val newPoints = pointStats.map {pair => (pair._1, pair._2._1 / pair._2._2)}.collectAsMap() - + tempDist = 0.0 for (i <- 0 until K) { tempDist += kPoints(i).squaredDist(newPoints(i)) } - + for (newP <- newPoints) { kPoints(newP._1) = newP._2 } @@ -84,6 +84,6 @@ object SparkKMeans { println("Final centers:") kPoints.foreach(println) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index c54a55bdb4a11..3a2699d4d996b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -66,6 +66,6 @@ object SparkLR { } println("Final w: " + w) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index d203f4d20e15f..45b6e10f3ea9e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -57,7 +57,6 @@ object SparkPageRank { val output = ranks.collect() output.foreach(tup => println(tup._1 + " has rank: " + tup._2 + ".")) - System.exit(0) + ctx.stop() } } - diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 24e8afa26bc5f..eb47cf027cb10 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -70,6 +70,6 @@ object SparkTC { } while (nextCount != oldCount) println("TC has " + tc.count() + " edges.") - System.exit(0) + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 53b303d658386..5698d4746495d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -75,6 +75,6 @@ object SparkTachyonHdfsLR { } println("Final w: " + w) - System.exit(0) + sc.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index ce78f0876ed7c..2b207fd8d3e16 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -34,10 +34,10 @@ object SparkTachyonPi { } val spark = new SparkContext(args(0), "SparkTachyonPi", System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) - + val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices - + val rdd = spark.parallelize(1 to n, slices) rdd.persist(StorageLevel.OFF_HEAP) val count = rdd.map { i => @@ -46,7 +46,7 @@ object SparkTachyonPi { if (x * x + y * y < 1) 1 else 0 }.reduce(_ + _) println("Pi is roughly " + 4.0 * count / n) - + spark.stop() } } diff --git a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala index 7aac6a13597e6..dee3cb6c0abae 100644 --- a/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala +++ b/examples/src/main/scala/org/apache/spark/examples/bagel/WikipediaPageRankStandalone.scala @@ -79,7 +79,7 @@ object WikipediaPageRankStandalone { val time = (System.currentTimeMillis - startTime) / 1000.0 println("Completed %d iterations in %f seconds: %f seconds per iteration" .format(numIterations, time, time / numIterations)) - System.exit(0) + sc.stop() } def parseArticle(line: String): (String, Array[String]) = { diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index a177435e606ab..61b9655cd3759 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.examples.mllib - + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 49d09692c8e4a..9aeebf58eabfb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -16,7 +16,7 @@ */ package org.apache.spark.examples.mllib - + import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.mllib.linalg.distributed.RowMatrix import org.apache.spark.mllib.linalg.Vectors diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 954bcc9b6ef5d..1c0ce3111e290 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -53,4 +53,3 @@ object HdfsWordCount { ssc.awaitTermination() } } - diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 6bccd1d88401a..cca0be2cbb9c9 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -61,7 +61,7 @@ object KafkaWordCount { val wordCounts = words.map(x => (x, 1L)) .reduceByKeyAndWindow(add _, subtract _, Minutes(10), Seconds(2), 2) wordCounts.print() - + ssc.start() ssc.awaitTermination() } @@ -83,7 +83,7 @@ object KafkaWordCountProducer { val props = new Properties() props.put("metadata.broker.list", brokers) props.put("serializer.class", "kafka.serializer.StringEncoder") - + val config = new ProducerConfig(props) val producer = new Producer[String, String](config) @@ -102,4 +102,3 @@ object KafkaWordCountProducer { } } - diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index 0a68ac84c2424..656222e0c1b31 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -26,7 +26,7 @@ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.mqtt._ /** - * A simple Mqtt publisher for demonstration purposes, repeatedly publishes + * A simple Mqtt publisher for demonstration purposes, repeatedly publishes * Space separated String Message "hello mqtt demo for spark streaming" */ object MQTTPublisher { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index 4d4968ba6ae3e..612ecf7b7821a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -24,7 +24,7 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ object QueueStream { - + def main(args: Array[String]) { if (args.length < 1) { System.err.println("Usage: QueueStream ") @@ -37,23 +37,22 @@ object QueueStream { val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) - // Create the queue through which RDDs can be pushed to + // Create the queue through which RDDs can be pushed to // a QueueInputDStream val rddQueue = new SynchronizedQueue[RDD[Int]]() - + // Create the QueueInputDStream and use it do some processing val inputStream = ssc.queueStream(rddQueue) val mappedStream = inputStream.map(x => (x % 10, 1)) val reducedStream = mappedStream.reduceByKey(_ + _) - reducedStream.print() + reducedStream.print() ssc.start() - + // Create and push some RDDs into for (i <- 1 to 30) { rddQueue += ssc.sparkContext.makeRDD(1 to 1000, 10) Thread.sleep(1000) } ssc.stop() - System.exit(0) } } diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index c2d84a8e0861e..14f65a2f8d46c 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -58,7 +58,7 @@ object StatefulNetworkWordCount { ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the - // words in input stream of \n delimited test (eg. generated by 'nc') + // words in input stream of \n delimited test (eg. generated by 'nc') val lines = ssc.socketTextStream(args(1), args(2).toInt) val words = lines.flatMap(_.split(" ")) val wordDstream = words.map(x => (x, 1)) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 35f8f885f8f0e..445d2028582af 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -60,7 +60,7 @@ object SimpleZeroMQPublisher { * To work with zeroMQ, some native libraries have to be installed. * Install zeroMQ (release 2.1) core libraries. [ZeroMQ Install guide] * (http://www.zeromq.org/intro:get-the-software) - * + * * Usage: ZeroMQWordCount * In local mode, should be 'local[n]' with n > 1 * and describe where zeroMq publisher is running. From e6d4a74d2d92345985c1603f9b526a6347adb7cf Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 10 Apr 2014 02:10:40 -0700 Subject: [PATCH 248/397] Revert "SPARK-729: Closures not always serialized at capture time" This reverts commit 8ca3b2bc90a63b23a03f339e390174cd7a672b40. --- .../scala/org/apache/spark/SparkContext.scala | 16 +--- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../apache/spark/util/ClosureCleaner.scala | 21 +---- .../scala/org/apache/spark/FailureSuite.scala | 17 +--- .../ProactiveClosureSerializationSuite.scala | 94 ------------------- .../spark/util/ClosureCleanerSuite.scala | 68 -------------- .../org/apache/spark/graphx/GraphSuite.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 8 +- 8 files changed, 14 insertions(+), 218 deletions(-) delete mode 100644 core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 545807ffbce55..76305237b03d5 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1002,9 +1002,7 @@ class SparkContext(config: SparkConf) extends Logging { require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") } val callSite = getCallSite - // There's no need to check this function for serializability, - // since it will be run right away. - val cleanedFunc = clean(func, false) + val cleanedFunc = clean(func) logInfo("Starting job: " + callSite) val start = System.nanoTime dagScheduler.runJob(rdd, cleanedFunc, partitions, callSite, allowLocal, @@ -1137,18 +1135,14 @@ class SparkContext(config: SparkConf) extends Logging { def cancelAllJobs() { dagScheduler.cancelAllJobs() } - + /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) - * - * @param f closure to be cleaned and optionally serialized - * @param captureNow whether or not to serialize this closure and capture any free - * variables immediately; defaults to true. If this is set and f is not serializable, - * it will raise an exception. */ - private[spark] def clean[F <: AnyRef : ClassTag](f: F, captureNow: Boolean = true): F = { - ClosureCleaner.clean(f, captureNow) + private[spark] def clean[F <: AnyRef](f: F): F = { + ClosureCleaner.clean(f) + f } /** diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e363ea777d8eb..3437b2cac19c2 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -660,16 +660,14 @@ abstract class RDD[T: ClassTag]( * Applies a function f to all elements of this RDD. */ def foreach(f: T => Unit) { - val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => iter.foreach(cleanF)) + sc.runJob(this, (iter: Iterator[T]) => iter.foreach(f)) } /** * Applies a function f to each partition of this RDD. */ def foreachPartition(f: Iterator[T] => Unit) { - val cleanF = sc.clean(f) - sc.runJob(this, (iter: Iterator[T]) => cleanF(iter)) + sc.runJob(this, (iter: Iterator[T]) => f(iter)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index e474b1a850d65..cdbbc65292188 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -22,14 +22,10 @@ import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.Map import scala.collection.mutable.Set -import scala.reflect.ClassTag - import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.{ClassReader, ClassVisitor, MethodVisitor, Type} import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ import org.apache.spark.Logging -import org.apache.spark.SparkEnv -import org.apache.spark.SparkException private[spark] object ClosureCleaner extends Logging { // Get an ASM class reader for a given class from the JAR that loaded it @@ -105,7 +101,7 @@ private[spark] object ClosureCleaner extends Logging { } } - def clean[F <: AnyRef : ClassTag](func: F, captureNow: Boolean = true): F = { + def clean(func: AnyRef) { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) @@ -154,21 +150,6 @@ private[spark] object ClosureCleaner extends Logging { field.setAccessible(true) field.set(func, outer) } - - if (captureNow) { - cloneViaSerializing(func) - } else { - func - } - } - - private def cloneViaSerializing[T: ClassTag](func: T): T = { - try { - val serializer = SparkEnv.get.closureSerializer.newInstance() - serializer.deserialize[T](serializer.serialize[T](func)) - } catch { - case ex: Exception => throw new SparkException("Task not serializable: " + ex.toString) - } } private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { diff --git a/core/src/test/scala/org/apache/spark/FailureSuite.scala b/core/src/test/scala/org/apache/spark/FailureSuite.scala index 4f9300419e6f8..12dbebcb28644 100644 --- a/core/src/test/scala/org/apache/spark/FailureSuite.scala +++ b/core/src/test/scala/org/apache/spark/FailureSuite.scala @@ -107,7 +107,7 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } - test("failure because closure in final-stage task is not serializable") { + test("failure because task closure is not serializable") { sc = new SparkContext("local[1,1]", "test") val a = new NonSerializable @@ -118,13 +118,6 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown.getClass === classOf[SparkException]) assert(thrown.getMessage.contains("NotSerializableException")) - FailureSuiteState.clear() - } - - test("failure because closure in early-stage task is not serializable") { - sc = new SparkContext("local[1,1]", "test") - val a = new NonSerializable - // Non-serializable closure in an earlier stage val thrown1 = intercept[SparkException] { sc.parallelize(1 to 10, 2).map(x => (x, a)).partitionBy(new HashPartitioner(3)).count() @@ -132,13 +125,6 @@ class FailureSuite extends FunSuite with LocalSparkContext { assert(thrown1.getClass === classOf[SparkException]) assert(thrown1.getMessage.contains("NotSerializableException")) - FailureSuiteState.clear() - } - - test("failure because closure in foreach task is not serializable") { - sc = new SparkContext("local[1,1]", "test") - val a = new NonSerializable - // Non-serializable closure in foreach function val thrown2 = intercept[SparkException] { sc.parallelize(1 to 10, 2).foreach(x => println(a)) @@ -149,6 +135,5 @@ class FailureSuite extends FunSuite with LocalSparkContext { FailureSuiteState.clear() } - // TODO: Need to add tests with shuffle fetch failures. } diff --git a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala b/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala deleted file mode 100644 index 76662264e7e94..0000000000000 --- a/core/src/test/scala/org/apache/spark/serializer/ProactiveClosureSerializationSuite.scala +++ /dev/null @@ -1,94 +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.serializer; - -import java.io.NotSerializableException - -import org.scalatest.FunSuite - -import org.apache.spark.rdd.RDD -import org.apache.spark.SparkException -import org.apache.spark.SharedSparkContext - -/* A trivial (but unserializable) container for trivial functions */ -class UnserializableClass { - def op[T](x: T) = x.toString - - def pred[T](x: T) = x.toString.length % 2 == 0 -} - -class ProactiveClosureSerializationSuite extends FunSuite with SharedSparkContext { - - def fixture = (sc.parallelize(0 until 1000).map(_.toString), new UnserializableClass) - - test("throws expected serialization exceptions on actions") { - val (data, uc) = fixture - - val ex = intercept[SparkException] { - data.map(uc.op(_)).count - } - - assert(ex.getMessage.matches(".*Task not serializable.*")) - } - - // There is probably a cleaner way to eliminate boilerplate here, but we're - // iterating over a map from transformation names to functions that perform that - // transformation on a given RDD, creating one test case for each - - for (transformation <- - Map("map" -> map _, "flatMap" -> flatMap _, "filter" -> filter _, "mapWith" -> mapWith _, - "mapPartitions" -> mapPartitions _, "mapPartitionsWithIndex" -> mapPartitionsWithIndex _, - "mapPartitionsWithContext" -> mapPartitionsWithContext _, "filterWith" -> filterWith _)) { - val (name, xf) = transformation - - test(s"$name transformations throw proactive serialization exceptions") { - val (data, uc) = fixture - - val ex = intercept[SparkException] { - xf(data, uc) - } - - assert(ex.getMessage.matches(".*Task not serializable.*"), s"RDD.$name doesn't proactively throw NotSerializableException") - } - } - - def map(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.map(y => uc.op(y)) - - def mapWith(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapWith(x => x.toString)((x,y) => x + uc.op(y)) - - def flatMap(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.flatMap(y=>Seq(uc.op(y))) - - def filter(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.filter(y=>uc.pred(y)) - - def filterWith(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.filterWith(x => x.toString)((x,y) => uc.pred(y)) - - def mapPartitions(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitions(_.map(y => uc.op(y))) - - def mapPartitionsWithIndex(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitionsWithIndex((_, it) => it.map(y => uc.op(y))) - - def mapPartitionsWithContext(x: RDD[String], uc: UnserializableClass): RDD[String] = - x.mapPartitionsWithContext((_, it) => it.map(y => uc.op(y))) - -} diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index c635da6cacd70..439e5644e20a3 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -50,27 +50,6 @@ class ClosureCleanerSuite extends FunSuite { val obj = new TestClassWithNesting(1) assert(obj.run() === 96) // 4 * (1+2+3+4) + 4 * (1+2+3+4) + 16 * 1 } - - test("capturing free variables in closures at RDD definition") { - val obj = new TestCaptureVarClass() - val (ones, onesPlusZeroes) = obj.run() - - assert(ones === onesPlusZeroes) - } - - test("capturing free variable fields in closures at RDD definition") { - val obj = new TestCaptureFieldClass() - val (ones, onesPlusZeroes) = obj.run() - - assert(ones === onesPlusZeroes) - } - - test("capturing arrays in closures at RDD definition") { - val obj = new TestCaptureArrayEltClass() - val (observed, expected) = obj.run() - - assert(observed === expected) - } } // A non-serializable class we create in closures to make sure that we aren't @@ -164,50 +143,3 @@ class TestClassWithNesting(val y: Int) extends Serializable { } } } - -class TestCaptureFieldClass extends Serializable { - class ZeroBox extends Serializable { - var zero = 0 - } - - def run(): (Int, Int) = { - val zb = new ZeroBox - - withSpark(new SparkContext("local", "test")) {sc => - val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) - val onesPlusZeroes = ones.map(_ + zb.zero) - - zb.zero = 5 - - (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) - } - } -} - -class TestCaptureArrayEltClass extends Serializable { - def run(): (Int, Int) = { - withSpark(new SparkContext("local", "test")) {sc => - val rdd = sc.parallelize(1 to 10) - val data = Array(1, 2, 3) - val expected = data(0) - val mapped = rdd.map(x => data(0)) - data(0) = 4 - (mapped.first, expected) - } - } -} - -class TestCaptureVarClass extends Serializable { - def run(): (Int, Int) = { - var zero = 0 - - withSpark(new SparkContext("local", "test")) {sc => - val ones = sc.parallelize(Array(1, 1, 1, 1, 1)) - val onesPlusZeroes = ones.map(_ + zero) - - zero = 5 - - (ones.reduce(_ + _), onesPlusZeroes.reduce(_ + _)) - } - } -} diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala index c65e36636fe10..28d34dd9a1a41 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphSuite.scala @@ -62,7 +62,7 @@ class GraphSuite extends FunSuite with LocalSparkContext { assert( graph.edges.count() === rawEdges.size ) // Vertices not explicitly provided but referenced by edges should be created automatically assert( graph.vertices.count() === 100) - graph.triplets.collect.map { et => + graph.triplets.map { et => assert((et.srcId < 10 && et.srcAttr) || (et.srcId >= 10 && !et.srcAttr)) assert((et.dstId < 10 && et.dstAttr) || (et.dstId >= 10 && !et.dstAttr)) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index 4759b629a9931..d043200f71a0b 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -539,7 +539,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: RDD[T] => RDD[U]): DStream[U] = { - transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r), false)) + transform((r: RDD[T], t: Time) => context.sparkContext.clean(transformFunc(r))) } /** @@ -547,7 +547,7 @@ abstract class DStream[T: ClassTag] ( * on each RDD of 'this' DStream. */ def transform[U: ClassTag](transformFunc: (RDD[T], Time) => RDD[U]): DStream[U] = { - val cleanedF = context.sparkContext.clean(transformFunc, false) + val cleanedF = context.sparkContext.clean(transformFunc) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 1) cleanedF(rdds.head.asInstanceOf[RDD[T]], time) @@ -562,7 +562,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U]) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc, false) + val cleanedF = ssc.sparkContext.clean(transformFunc) transformWith(other, (rdd1: RDD[T], rdd2: RDD[U], time: Time) => cleanedF(rdd1, rdd2)) } @@ -573,7 +573,7 @@ abstract class DStream[T: ClassTag] ( def transformWith[U: ClassTag, V: ClassTag]( other: DStream[U], transformFunc: (RDD[T], RDD[U], Time) => RDD[V] ): DStream[V] = { - val cleanedF = ssc.sparkContext.clean(transformFunc, false) + val cleanedF = ssc.sparkContext.clean(transformFunc) val realTransformFunc = (rdds: Seq[RDD[_]], time: Time) => { assert(rdds.length == 2) val rdd1 = rdds(0).asInstanceOf[RDD[T]] From a74fbbbca8f0d89b2e0e4e8751a93d33efc4fa9e Mon Sep 17 00:00:00 2001 From: witgo Date: Thu, 10 Apr 2014 10:35:24 -0700 Subject: [PATCH 249/397] Fix SPARK-1413: Parquet messes up stdout and stdin when used in Spark REPL Author: witgo Closes #325 from witgo/SPARK-1413 and squashes the following commits: e57cd8e [witgo] use scala reflection to access and call the SLF4JBridgeHandler methods 45c8f40 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 5e35d87 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 0d5f819 [witgo] review commit 45e5b70 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 fa69dcf [witgo] Merge branch 'master' into SPARK-1413 3c98dc4 [witgo] Merge branch 'master' into SPARK-1413 38160cb [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 ba09bcd [witgo] remove set the parquet log level a63d574 [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 5231ecd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 3feb635 [witgo] parquet logger use parent handler fa00d5d [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 8bb6ffd [witgo] enableLogForwarding note fix edd9630 [witgo] move to f447f50 [witgo] merging master 5ad52bd [witgo] Merge branch 'master' of https://github.com/apache/spark into SPARK-1413 76670c1 [witgo] review commit 70f3c64 [witgo] Fix SPARK-1413 --- .../main/scala/org/apache/spark/Logging.scala | 20 +++++++++--- .../spark/sql/parquet/ParquetRelation.scala | 31 +++++-------------- 2 files changed, 23 insertions(+), 28 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index e5e15617acb10..9d429dceeb858 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -28,7 +28,7 @@ import org.apache.spark.annotation.DeveloperApi * Utility trait for classes that want to log data. Creates a SLF4J logger for the class and allows * logging messages at different levels using methods that only evaluate parameters lazily if the * log level is enabled. - * + * * NOTE: DO NOT USE this class outside of Spark. It is intended as an internal utility. * This will likely be changed or removed in future releases. */ @@ -60,7 +60,7 @@ trait Logging { protected def logDebug(msg: => String) { if (log.isDebugEnabled) log.debug(msg) } - + protected def logTrace(msg: => String) { if (log.isTraceEnabled) log.trace(msg) } @@ -117,10 +117,10 @@ trait Logging { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" val classLoader = this.getClass.getClassLoader Option(classLoader.getResource(defaultLogProps)) match { - case Some(url) => + case Some(url) => PropertyConfigurator.configure(url) log.info(s"Using Spark's default log4j profile: $defaultLogProps") - case None => + case None => System.err.println(s"Spark was unable to load $defaultLogProps") } } @@ -135,4 +135,16 @@ trait Logging { private object Logging { @volatile private var initialized = false val initLock = new Object() + try { + // We use reflection here to handle the case where users remove the + // slf4j-to-jul bridge order to route their logs to JUL. + val bridgeClass = Class.forName("org.slf4j.bridge.SLF4JBridgeHandler") + bridgeClass.getMethod("removeHandlersForRootLogger").invoke(null) + val installed = bridgeClass.getMethod("isInstalled").invoke(null).asInstanceOf[Boolean] + if (!installed) { + bridgeClass.getMethod("install").invoke(null) + } + } catch { + case e: ClassNotFoundException => // can't log anything yet so just fail silently + } } 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 505ad0a2c77c1..4d7c86a3a4fc7 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 @@ -82,30 +82,13 @@ private[sql] case class ParquetRelation(val path: String) private[sql] object ParquetRelation { def enableLogForwarding() { - // Note: Parquet does not use forwarding to parent loggers which - // is required for the JUL-SLF4J bridge to work. Also there is - // a default logger that appends to Console which needs to be - // reset. - import org.slf4j.bridge.SLF4JBridgeHandler - import java.util.logging.Logger - import java.util.logging.LogManager - - val loggerNames = Seq( - "parquet.hadoop.ColumnChunkPageWriteStore", - "parquet.hadoop.InternalParquetRecordWriter", - "parquet.hadoop.ParquetRecordReader", - "parquet.hadoop.ParquetInputFormat", - "parquet.hadoop.ParquetOutputFormat", - "parquet.hadoop.ParquetFileReader", - "parquet.hadoop.InternalParquetRecordReader", - "parquet.hadoop.codec.CodecConfig") - LogManager.getLogManager.reset() - SLF4JBridgeHandler.install() - for(name <- loggerNames) { - val logger = Logger.getLogger(name) - logger.setParent(Logger.getLogger(Logger.GLOBAL_LOGGER_NAME)) - logger.setUseParentHandlers(true) - } + // Note: Logger.getLogger("parquet") has a default logger + // that appends to Console which needs to be cleared. + val parquetLogger = java.util.logging.Logger.getLogger("parquet") + parquetLogger.getHandlers.foreach(parquetLogger.removeHandler) + // TODO(witgo): Need to set the log level ? + // if(parquetLogger.getLevel != null) parquetLogger.setLevel(null) + if (!parquetLogger.getUseParentHandlers) parquetLogger.setUseParentHandlers(true) } // The element type for the RDDs that this relation maps to. From 79820fe825ed7c09d55f50503b7ab53d4585e5f7 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 10 Apr 2014 10:39:34 -0700 Subject: [PATCH 250/397] [SPARK-1276] Add a HistoryServer to render persisted UI The new feature of event logging, introduced in #42, allows the user to persist the details of his/her Spark application to storage, and later replay these events to reconstruct an after-the-fact SparkUI. Currently, however, a persisted UI can only be rendered through the standalone Master. This greatly limits the use case of this new feature as many people also run Spark on Yarn / Mesos. This PR introduces a new entity called the HistoryServer, which, given a log directory, keeps track of all completed applications independently of a Spark Master. Unlike Master, the HistoryServer needs not be running while the application is still running. It is relatively light-weight in that it only maintains static information of applications and performs no scheduling. To quickly test it out, generate event logs with ```spark.eventLog.enabled=true``` and run ```sbin/start-history-server.sh ```. Your HistoryServer awaits on port 18080. Comments and feedback are most welcome. --- A few other changes introduced in this PR include refactoring the WebUI interface, which is beginning to have a lot of duplicate code now that we have added more functionality to it. Two new SparkListenerEvents have been introduced (SparkListenerApplicationStart/End) to keep track of application name and start/finish times. This PR also clarifies the semantics of the ReplayListenerBus introduced in #42. A potential TODO in the future (not part of this PR) is to render live applications in addition to just completed applications. This is useful when applications fail, a condition that our current HistoryServer does not handle unless the user manually signals application completion (by creating the APPLICATION_COMPLETION file). Handling live applications becomes significantly more challenging, however, because it is now necessary to render the same SparkUI multiple times. To avoid reading the entire log every time, which is inefficient, we must handle reading the log from where we previously left off, but this becomes fairly complicated because we must deal with the arbitrary behavior of each input stream. Author: Andrew Or Closes #204 from andrewor14/master and squashes the following commits: 7b7234c [Andrew Or] Finished -> Completed b158d98 [Andrew Or] Address Patrick's comments 69d1b41 [Andrew Or] Do not block on posting SparkListenerApplicationEnd 19d5dd0 [Andrew Or] Merge github.com:apache/spark f7f5bf0 [Andrew Or] Make history server's web UI port a Spark configuration 2dfb494 [Andrew Or] Decouple checking for application completion from replaying d02dbaa [Andrew Or] Expose Spark version and include it in event logs 2282300 [Andrew Or] Add documentation for the HistoryServer 567474a [Andrew Or] Merge github.com:apache/spark 6edf052 [Andrew Or] Merge github.com:apache/spark 19e1fb4 [Andrew Or] Address Thomas' comments 248cb3d [Andrew Or] Limit number of live applications + add configurability a3598de [Andrew Or] Do not close file system with ReplayBus + fix bind address bc46fc8 [Andrew Or] Merge github.com:apache/spark e2f4ff9 [Andrew Or] Merge github.com:apache/spark 050419e [Andrew Or] Merge github.com:apache/spark 81b568b [Andrew Or] Fix strange error messages... 0670743 [Andrew Or] Decouple page rendering from loading files from disk 1b2f391 [Andrew Or] Minor changes a9eae7e [Andrew Or] Merge branch 'master' of github.com:apache/spark d5154da [Andrew Or] Styling and comments 5dbfbb4 [Andrew Or] Merge branch 'master' of github.com:apache/spark 60bc6d5 [Andrew Or] First complete implementation of HistoryServer (only for finished apps) 7584418 [Andrew Or] Report application start/end times to HistoryServer 8aac163 [Andrew Or] Add basic application table c086bd5 [Andrew Or] Add HistoryServer and scripts ++ Refactor WebUI interface --- bin/spark-class | 8 +- bin/spark-class2.cmd | 7 +- .../scala/org/apache/spark/SparkContext.scala | 26 +- .../spark/deploy/ApplicationDescription.scala | 4 +- .../spark/deploy/SparkUIContainer.scala | 50 +++ .../spark/deploy/history/HistoryServer.scala | 287 ++++++++++++++++++ .../history/HistoryServerArguments.scala | 76 +++++ .../spark/deploy/history/IndexPage.scala | 82 +++++ .../apache/spark/deploy/master/Master.scala | 62 ++-- .../spark/deploy/master/ui/MasterWebUI.scala | 43 +-- .../spark/deploy/worker/ui/WorkerWebUI.scala | 22 +- .../scheduler/ApplicationEventListener.scala | 50 +++ .../scheduler/EventLoggingListener.scala | 146 ++++++++- .../spark/scheduler/ReplayListenerBus.scala | 65 ++-- .../spark/scheduler/SparkListener.scala | 15 + .../spark/scheduler/SparkListenerBus.scala | 4 + .../cluster/SparkDeploySchedulerBackend.scala | 2 +- .../apache/spark/storage/FileSegment.scala | 2 +- .../scala/org/apache/spark/ui/SparkUI.scala | 49 +-- .../scala/org/apache/spark/ui/WebUI.scala | 21 +- .../apache/spark/ui/env/EnvironmentUI.scala | 3 +- .../apache/spark/ui/exec/ExecutorsUI.scala | 3 +- .../org/apache/spark/ui/jobs/IndexPage.scala | 3 +- .../apache/spark/ui/jobs/JobProgressUI.scala | 3 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/StagePage.scala | 3 +- .../spark/ui/storage/BlockManagerUI.scala | 3 +- .../apache/spark/ui/storage/IndexPage.scala | 3 +- .../org/apache/spark/ui/storage/RDDPage.scala | 3 +- .../org/apache/spark/util/FileLogger.scala | 27 +- .../org/apache/spark/util/JsonProtocol.scala | 31 ++ .../scala/org/apache/spark/util/Utils.scala | 8 +- .../ui/jobs/JobProgressListenerSuite.scala | 2 +- .../apache/spark/util/JsonProtocolSuite.scala | 24 +- docs/monitoring.md | 70 ++++- .../apache/spark/repl/SparkILoopInit.scala | 4 +- sbin/start-history-server.sh | 37 +++ sbin/stop-history-server.sh | 25 ++ 38 files changed, 1075 insertions(+), 201 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala create mode 100644 core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala create mode 100644 core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala create mode 100755 sbin/start-history-server.sh create mode 100755 sbin/stop-history-server.sh diff --git a/bin/spark-class b/bin/spark-class index 76fde3e448891..1b0d309cc5b1c 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -47,9 +47,9 @@ DEFAULT_MEM=${SPARK_MEM:-512m} SPARK_DAEMON_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS -Dspark.akka.logLifecycleEvents=true" -# Add java opts and memory settings for master, worker, executors, and repl. +# Add java opts and memory settings for master, worker, history server, executors, and repl. case "$1" in - # Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. + # Master, Worker, and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. 'org.apache.spark.deploy.master.Master') OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_MASTER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} @@ -58,6 +58,10 @@ case "$1" in OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_WORKER_OPTS" OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} ;; + 'org.apache.spark.deploy.history.HistoryServer') + OUR_JAVA_OPTS="$SPARK_DAEMON_JAVA_OPTS $SPARK_HISTORY_OPTS" + OUR_JAVA_MEM=${SPARK_DAEMON_MEMORY:-$DEFAULT_MEM} + ;; # Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. 'org.apache.spark.executor.CoarseGrainedExecutorBackend') diff --git a/bin/spark-class2.cmd b/bin/spark-class2.cmd index f488cfdbeceb6..4302c1b6b7ff4 100755 --- a/bin/spark-class2.cmd +++ b/bin/spark-class2.cmd @@ -45,14 +45,17 @@ if "x%OUR_JAVA_MEM%"=="x" set OUR_JAVA_MEM=512m set SPARK_DAEMON_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% -Dspark.akka.logLifecycleEvents=true -rem Add java opts and memory settings for master, worker, executors, and repl. -rem Master and Worker use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. +rem Add java opts and memory settings for master, worker, history server, executors, and repl. +rem Master, Worker and HistoryServer use SPARK_DAEMON_JAVA_OPTS (and specific opts) + SPARK_DAEMON_MEMORY. if "%1"=="org.apache.spark.deploy.master.Master" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_MASTER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% ) else if "%1"=="org.apache.spark.deploy.worker.Worker" ( set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_WORKER_OPTS% if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% +) else if "%1"=="org.apache.spark.deploy.history.HistoryServer" ( + set OUR_JAVA_OPTS=%SPARK_DAEMON_JAVA_OPTS% %SPARK_HISTORY_OPTS% + if not "x%SPARK_DAEMON_MEMORY%"=="x" set OUR_JAVA_MEM=%SPARK_DAEMON_MEMORY% rem Executors use SPARK_JAVA_OPTS + SPARK_EXECUTOR_MEMORY. ) else if "%1"=="org.apache.spark.executor.CoarseGrainedExecutorBackend" ( diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 76305237b03d5..e6c9b7000d819 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -219,15 +219,12 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val eventLogger: Option[EventLoggingListener] = { if (conf.getBoolean("spark.eventLog.enabled", false)) { val logger = new EventLoggingListener(appName, conf) + logger.start() listenerBus.addListener(logger) Some(logger) } else None } - // Information needed to replay logged events, if any - private[spark] val eventLoggingInfo: Option[EventLoggingInfo] = - eventLogger.map { logger => Some(logger.info) }.getOrElse(None) - // At this point, all relevant SparkListeners have been registered, so begin releasing events listenerBus.start() @@ -292,6 +289,7 @@ class SparkContext(config: SparkConf) extends Logging { cleaner.foreach(_.start()) postEnvironmentUpdate() + postApplicationStart() /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ val hadoopConfiguration: Configuration = { @@ -777,6 +775,9 @@ class SparkContext(config: SparkConf) extends Logging { listenerBus.addListener(listener) } + /** The version of Spark on which this application is running. */ + def version = SparkContext.SPARK_VERSION + /** * Return a map from the slave to the max memory available for caching and the remaining * memory available for caching. @@ -930,6 +931,7 @@ class SparkContext(config: SparkConf) extends Logging { /** Shut down the SparkContext. */ def stop() { + postApplicationEnd() ui.stop() // Do this only if not stopped already - best case effort. // prevent NPE if stopped more than once. @@ -1175,6 +1177,20 @@ class SparkContext(config: SparkConf) extends Logging { /** Register a new RDD, returning its RDD ID */ private[spark] def newRddId(): Int = nextRddId.getAndIncrement() + /** Post the application start event */ + private def postApplicationStart() { + listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) + } + + /** + * Post the application end event to all listeners immediately, rather than adding it + * to the event queue for it to be asynchronously processed eventually. Otherwise, a race + * condition exists in which the listeners may stop before this event has been propagated. + */ + private def postApplicationEnd() { + listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis)) + } + /** Post the environment update event once the task scheduler is ready */ private def postEnvironmentUpdate() { if (taskScheduler != null) { @@ -1200,6 +1216,8 @@ class SparkContext(config: SparkConf) extends Logging { */ object SparkContext extends Logging { + private[spark] val SPARK_VERSION = "1.0.0" + private[spark] val SPARK_JOB_DESCRIPTION = "spark.job.description" private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" diff --git a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala index 15fa8a7679874..86305d2ea8a09 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ApplicationDescription.scala @@ -17,8 +17,6 @@ package org.apache.spark.deploy -import org.apache.spark.scheduler.EventLoggingInfo - private[spark] class ApplicationDescription( val name: String, val maxCores: Option[Int], @@ -26,7 +24,7 @@ private[spark] class ApplicationDescription( val command: Command, val sparkHome: Option[String], var appUiUrl: String, - val eventLogInfo: Option[EventLoggingInfo] = None) + val eventLogDir: Option[String] = None) extends Serializable { val user = System.getProperty("user.name", "") diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala new file mode 100644 index 0000000000000..33fceae4ff489 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala @@ -0,0 +1,50 @@ +/* + * 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.deploy + +import org.apache.spark.ui.{SparkUI, WebUI} + +private[spark] abstract class SparkUIContainer(name: String) extends WebUI(name) { + + /** Attach a SparkUI to this container. Only valid after bind(). */ + def attachUI(ui: SparkUI) { + assert(serverInfo.isDefined, + "%s must be bound to a server before attaching SparkUIs".format(name)) + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } + + /** Detach a SparkUI from this container. Only valid after bind(). */ + def detachUI(ui: SparkUI) { + assert(serverInfo.isDefined, + "%s must be bound to a server before detaching SparkUIs".format(name)) + val rootHandler = serverInfo.get.rootHandler + for (handler <- ui.handlers) { + if (handler.isStarted) { + handler.stop() + } + rootHandler.removeHandler(handler) + } + } + +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala new file mode 100644 index 0000000000000..97d2ba9deed33 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -0,0 +1,287 @@ +/* + * 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.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileStatus, Path} +import org.eclipse.jetty.servlet.ServletContextHandler + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.SparkUIContainer +import org.apache.spark.scheduler._ +import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils + +/** + * A web server that renders SparkUIs of completed applications. + * + * For the standalone mode, MasterWebUI already achieves this functionality. Thus, the + * main use case of the HistoryServer is in other deploy modes (e.g. Yarn or Mesos). + * + * The logging directory structure is as follows: Within the given base directory, each + * application's event logs are maintained in the application's own sub-directory. This + * is the same structure as maintained in the event log write code path in + * EventLoggingListener. + * + * @param baseLogDir The base directory in which event logs are found + */ +class HistoryServer( + val baseLogDir: String, + conf: SparkConf) + extends SparkUIContainer("History Server") with Logging { + + import HistoryServer._ + + private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) + private val localHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) + private val port = WEB_UI_PORT + private val securityManager = new SecurityManager(conf) + private val indexPage = new IndexPage(this) + + // A timestamp of when the disk was last accessed to check for log updates + private var lastLogCheckTime = -1L + + // Number of completed applications found in this directory + private var numCompletedApplications = 0 + + @volatile private var stopped = false + + /** + * A background thread that periodically checks for event log updates on disk. + * + * If a log check is invoked manually in the middle of a period, this thread re-adjusts the + * time at which it performs the next log check to maintain the same period as before. + * + * TODO: Add a mechanism to update manually. + */ + private val logCheckingThread = new Thread { + override def run() { + while (!stopped) { + val now = System.currentTimeMillis + if (now - lastLogCheckTime > UPDATE_INTERVAL_MS) { + checkForLogs() + Thread.sleep(UPDATE_INTERVAL_MS) + } else { + // If the user has manually checked for logs recently, wait until + // UPDATE_INTERVAL_MS after the last check time + Thread.sleep(lastLogCheckTime + UPDATE_INTERVAL_MS - now) + } + } + } + } + + private val handlers = Seq[ServletContextHandler]( + createStaticHandler(STATIC_RESOURCE_DIR, "/static"), + createServletHandler("/", + (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager) + ) + + // A mapping of application ID to its history information, which includes the rendered UI + val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + + /** + * Start the history server. + * + * This starts a background thread that periodically synchronizes information displayed on + * this UI with the event logs in the provided base directory. + */ + def start() { + logCheckingThread.start() + } + + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) + } catch { + case e: Exception => + logError("Failed to bind HistoryServer", e) + System.exit(1) + } + } + + /** + * Check for any updates to event logs in the base directory. This is only effective once + * the server has been bound. + * + * If a new completed application is found, the server renders the associated SparkUI + * from the application's event logs, attaches this UI to itself, and stores metadata + * information for this application. + * + * If the logs for an existing completed application are no longer found, the server + * removes all associated information and detaches the SparkUI. + */ + def checkForLogs() = synchronized { + if (serverInfo.isDefined) { + lastLogCheckTime = System.currentTimeMillis + logDebug("Checking for logs. Time is now %d.".format(lastLogCheckTime)) + try { + val logStatus = fileSystem.listStatus(new Path(baseLogDir)) + val logDirs = if (logStatus != null) logStatus.filter(_.isDir).toSeq else Seq[FileStatus]() + val logInfos = logDirs + .sortBy { dir => getModificationTime(dir) } + .map { dir => (dir, EventLoggingListener.parseLoggingInfo(dir.getPath, fileSystem)) } + .filter { case (dir, info) => info.applicationComplete } + + // Logging information for applications that should be retained + val retainedLogInfos = logInfos.takeRight(RETAINED_APPLICATIONS) + val retainedAppIds = retainedLogInfos.map { case (dir, _) => dir.getPath.getName } + + // Remove any applications that should no longer be retained + appIdToInfo.foreach { case (appId, info) => + if (!retainedAppIds.contains(appId)) { + detachUI(info.ui) + appIdToInfo.remove(appId) + } + } + + // Render the application's UI if it is not already there + retainedLogInfos.foreach { case (dir, info) => + val appId = dir.getPath.getName + if (!appIdToInfo.contains(appId)) { + renderSparkUI(dir, info) + } + } + + // Track the total number of completed applications observed this round + numCompletedApplications = logInfos.size + + } catch { + case t: Throwable => logError("Exception in checking for event log updates", t) + } + } else { + logWarning("Attempted to check for event log updates before binding the server.") + } + } + + /** + * Render a new SparkUI from the event logs if the associated application is completed. + * + * HistoryServer looks for a special file that indicates application completion in the given + * directory. If this file exists, the associated application is regarded to be completed, in + * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. + */ + private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) { + val path = logDir.getPath + val appId = path.getName + val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) + val ui = new SparkUI(replayBus, appId, "/history/" + appId) + val appListener = new ApplicationEventListener + replayBus.addListener(appListener) + + // Do not call ui.bind() to avoid creating a new server for each application + ui.start() + replayBus.replay() + if (appListener.applicationStarted) { + attachUI(ui) + val appName = appListener.appName + val sparkUser = appListener.sparkUser + val startTime = appListener.startTime + val endTime = appListener.endTime + val lastUpdated = getModificationTime(logDir) + ui.setAppName(appName + " (completed)") + appIdToInfo(appId) = ApplicationHistoryInfo(appId, appName, startTime, endTime, + lastUpdated, sparkUser, path, ui) + } + } + + /** Stop the server and close the file system. */ + override def stop() { + super.stop() + stopped = true + fileSystem.close() + } + + /** Return the address of this server. */ + def getAddress: String = "http://" + publicHost + ":" + boundPort + + /** Return the number of completed applications found, whether or not the UI is rendered. */ + def getNumApplications: Int = numCompletedApplications + + /** Return when this directory was last modified. */ + private def getModificationTime(dir: FileStatus): Long = { + try { + val logFiles = fileSystem.listStatus(dir.getPath) + if (logFiles != null && !logFiles.isEmpty) { + logFiles.map(_.getModificationTime).max + } else { + dir.getModificationTime + } + } catch { + case t: Throwable => + logError("Exception in accessing modification time of %s".format(dir.getPath), t) + -1L + } + } +} + +/** + * The recommended way of starting and stopping a HistoryServer is through the scripts + * start-history-server.sh and stop-history-server.sh. The path to a base log directory + * is must be specified, while the requested UI port is optional. For example: + * + * ./sbin/spark-history-server.sh /tmp/spark-events + * ./sbin/spark-history-server.sh hdfs://1.2.3.4:9000/spark-events + * + * This launches the HistoryServer as a Spark daemon. + */ +object HistoryServer { + private val conf = new SparkConf + + // Interval between each check for event log updates + val UPDATE_INTERVAL_MS = conf.getInt("spark.history.updateInterval", 10) * 1000 + + // How many applications to retain + val RETAINED_APPLICATIONS = conf.getInt("spark.history.retainedApplications", 250) + + // The port to which the web UI is bound + val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) + + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR + + def main(argStrings: Array[String]) { + val args = new HistoryServerArguments(argStrings) + val server = new HistoryServer(args.logDir, conf) + server.bind() + server.start() + + // Wait until the end of the world... or if the HistoryServer process is manually stopped + while(true) { Thread.sleep(Int.MaxValue) } + server.stop() + } +} + + +private[spark] case class ApplicationHistoryInfo( + id: String, + name: String, + startTime: Long, + endTime: Long, + lastUpdated: Long, + sparkUser: String, + logDirPath: Path, + ui: SparkUI) { + def started = startTime != -1 + def completed = endTime != -1 +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala new file mode 100644 index 0000000000000..943c061743dbd --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServerArguments.scala @@ -0,0 +1,76 @@ +/* + * 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.deploy.history + +import java.net.URI + +import org.apache.hadoop.fs.Path + +import org.apache.spark.util.Utils + +/** + * Command-line parser for the master. + */ +private[spark] class HistoryServerArguments(args: Array[String]) { + var logDir = "" + + parse(args.toList) + + private def parse(args: List[String]): Unit = { + args match { + case ("--dir" | "-d") :: value :: tail => + logDir = value + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case Nil => + + case _ => + printUsageAndExit(1) + } + validateLogDir() + } + + private def validateLogDir() { + if (logDir == "") { + System.err.println("Logging directory must be specified.") + printUsageAndExit(1) + } + val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) + val path = new Path(logDir) + if (!fileSystem.exists(path)) { + System.err.println("Logging directory specified does not exist: %s".format(logDir)) + printUsageAndExit(1) + } + if (!fileSystem.getFileStatus(path).isDir) { + System.err.println("Logging directory specified is not a directory: %s".format(logDir)) + printUsageAndExit(1) + } + } + + private def printUsageAndExit(exitCode: Int) { + System.err.println( + "Usage: HistoryServer [options]\n" + + "\n" + + "Options:\n" + + " -d DIR, --dir DIR Location of event log files") + System.exit(exitCode) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala new file mode 100644 index 0000000000000..54dffffec71c5 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala @@ -0,0 +1,82 @@ +/* + * 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.deploy.history + +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{UIUtils, WebUI} + +private[spark] class IndexPage(parent: HistoryServer) { + + def render(request: HttpServletRequest): Seq[Node] = { + val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } + val appTable = UIUtils.listingTable(appHeader, appRow, appRows) + val content = +

    +
    +
      +
    • Event Log Location: {parent.baseLogDir}
    • +
    + { + if (parent.appIdToInfo.size > 0) { +

    + Showing {parent.appIdToInfo.size}/{parent.getNumApplications} + Completed Application{if (parent.getNumApplications > 1) "s" else ""} +

    ++ + appTable + } else { +

    No Completed Applications Found

    + } + } +
    +
    + UIUtils.basicSparkPage(content, "History Server") + } + + private val appHeader = Seq( + "App Name", + "Started", + "Completed", + "Duration", + "Spark User", + "Log Directory", + "Last Updated") + + private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { + val appName = if (info.started) info.name else info.logDirPath.getName + val uiAddress = parent.getAddress + info.ui.basePath + val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" + val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not completed" + val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L + val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" + val sparkUser = if (info.started) info.sparkUser else "Unknown user" + val logDirectory = info.logDirPath.getName + val lastUpdated = WebUI.formatDate(info.lastUpdated) + + {appName} + {startTime} + {endTime} + {duration} + {sparkUser} + {logDirectory} + {lastUpdated} + + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 95bd62e88db2b..2446e86cb6672 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -29,6 +29,7 @@ import akka.actor._ import akka.pattern.ask import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} import akka.serialization.SerializationExtension +import org.apache.hadoop.fs.FileSystem import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkException} import org.apache.spark.deploy.{ApplicationDescription, DriverDescription, ExecutorState} @@ -37,7 +38,7 @@ import org.apache.spark.deploy.master.DriverState.DriverState import org.apache.spark.deploy.master.MasterMessages._ import org.apache.spark.deploy.master.ui.MasterWebUI import org.apache.spark.metrics.MetricsSystem -import org.apache.spark.scheduler.ReplayListenerBus +import org.apache.spark.scheduler.{EventLoggingListener, ReplayListenerBus} import org.apache.spark.ui.SparkUI import org.apache.spark.util.{AkkaUtils, Utils} @@ -45,7 +46,8 @@ private[spark] class Master( host: String, port: Int, webUiPort: Int, - val securityMgr: SecurityManager) extends Actor with Logging { + val securityMgr: SecurityManager) + extends Actor with Logging { import context.dispatcher // to use Akka's scheduler.schedule() @@ -71,6 +73,7 @@ private[spark] class Master( var nextAppNumber = 0 val appIdToUI = new HashMap[String, SparkUI] + val fileSystemsUsed = new HashSet[FileSystem] val drivers = new HashSet[DriverInfo] val completedDrivers = new ArrayBuffer[DriverInfo] @@ -149,6 +152,7 @@ private[spark] class Master( override def postStop() { webUi.stop() + fileSystemsUsed.foreach(_.close()) masterMetricsSystem.stop() applicationMetricsSystem.stop() persistenceEngine.close() @@ -630,11 +634,7 @@ private[spark] class Master( waitingApps -= app // If application events are logged, use them to rebuild the UI - startPersistedSparkUI(app).map { ui => - app.desc.appUiUrl = ui.basePath - appIdToUI(app.id) = ui - webUi.attachUI(ui) - }.getOrElse { + if (!rebuildSparkUI(app)) { // Avoid broken links if the UI is not reconstructed app.desc.appUiUrl = "" } @@ -654,30 +654,34 @@ private[spark] class Master( } /** - * Start a new SparkUI rendered from persisted storage. If this is unsuccessful for any reason, - * return None. Otherwise return the reconstructed UI. + * Rebuild a new SparkUI from the given application's event logs. + * Return whether this is successful. */ - def startPersistedSparkUI(app: ApplicationInfo): Option[SparkUI] = { + def rebuildSparkUI(app: ApplicationInfo): Boolean = { val appName = app.desc.name - val eventLogInfo = app.desc.eventLogInfo.getOrElse { return None } - val eventLogDir = eventLogInfo.logDir - val eventCompressionCodec = eventLogInfo.compressionCodec - val appConf = new SparkConf - eventCompressionCodec.foreach { codec => - appConf.set("spark.eventLog.compress", "true") - appConf.set("spark.io.compression.codec", codec) - } - val replayerBus = new ReplayListenerBus(appConf) - val ui = new SparkUI( - appConf, - replayerBus, - "%s (finished)".format(appName), - "/history/%s".format(app.id)) - - // Do not call ui.bind() to avoid creating a new server for each application - ui.start() - val success = replayerBus.replay(eventLogDir) - if (success) Some(ui) else None + val eventLogDir = app.desc.eventLogDir.getOrElse { return false } + val fileSystem = Utils.getHadoopFileSystem(eventLogDir) + val eventLogInfo = EventLoggingListener.parseLoggingInfo(eventLogDir, fileSystem) + val eventLogPaths = eventLogInfo.logPaths + val compressionCodec = eventLogInfo.compressionCodec + if (!eventLogPaths.isEmpty) { + try { + val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) + val ui = new SparkUI(replayBus, appName + " (completed)", "/history/" + app.id) + ui.start() + replayBus.replay() + app.desc.appUiUrl = ui.basePath + appIdToUI(app.id) = ui + webUi.attachUI(ui) + return true + } catch { + case t: Throwable => + logError("Exception in replaying log for application %s (%s)".format(appName, app.id), t) + } + } else { + logWarning("Application %s (%s) has no valid logs: %s".format(appName, app.id, eventLogDir)) + } + false } /** Generate a new app ID given a app's submission date */ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 01d9f52f4b7b4..30c8ade408a5a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -22,8 +22,9 @@ import javax.servlet.http.HttpServletRequest import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging +import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.{ServerInfo, SparkUI} +import org.apache.spark.ui.SparkUI import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -31,7 +32,9 @@ import org.apache.spark.util.{AkkaUtils, Utils} * Web UI server for the standalone master. */ private[spark] -class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { +class MasterWebUI(val master: Master, requestedPort: Int) + extends SparkUIContainer("MasterWebUI") with Logging { + val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) @@ -39,7 +42,6 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { private val port = requestedPort private val applicationPage = new ApplicationPage(this) private val indexPage = new IndexPage(this) - private var serverInfo: Option[ServerInfo] = None private val handlers: Seq[ServletContextHandler] = { master.masterMetricsSystem.getServletHandlers ++ @@ -57,47 +59,18 @@ class MasterWebUI(val master: Master, requestedPort: Int) extends Logging { ) } - def bind() { + /** Bind to the HTTP server behind this web interface. */ + override def bind() { try { serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Master JettyUtils", e) + logError("Failed to create Master web UI", e) System.exit(1) } } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - - /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ - def attachUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - rootHandler.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } - } - } - - /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ - def detachUI(ui: SparkUI) { - assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - if (handler.isStarted) { - handler.stop() - } - rootHandler.removeHandler(handler) - } - } - - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a Master UI that was not bound to a server!") - serverInfo.get.server.stop() - } } private[spark] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 650f3da5ce3ff..5625a44549aaa 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -24,7 +24,7 @@ import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.Logging import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{JettyUtils, ServerInfo, SparkUI, UIUtils} +import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -33,15 +33,14 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends Logging { + extends WebUI("WorkerWebUI") with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) private val host = Utils.localHostName() private val port = requestedPort.getOrElse( - worker.conf.get("worker.ui.port", WorkerWebUI.DEFAULT_PORT).toInt) + worker.conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) private val indexPage = new IndexPage(this) - private var serverInfo: Option[ServerInfo] = None private val handlers: Seq[ServletContextHandler] = { worker.metricsSystem.getServletHandlers ++ @@ -58,19 +57,18 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I ) } - def bind() { + /** Bind to the HTTP server behind this web interface. */ + override def bind() { try { - serverInfo = Some(JettyUtils.startJettyServer("0.0.0.0", port, handlers, worker.conf)) + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, worker.conf)) logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) } catch { case e: Exception => - logError("Failed to create Worker JettyUtils", e) + logError("Failed to create Worker web UI", e) System.exit(1) } } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - private def log(request: HttpServletRequest): String = { val defaultBytes = 100 * 1024 @@ -187,13 +185,9 @@ class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[I (startByte, endByte) } - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a Worker UI that was not bound to a server!") - serverInfo.get.server.stop() - } } private[spark] object WorkerWebUI { + val DEFAULT_PORT=8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR - val DEFAULT_PORT="8081" } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala new file mode 100644 index 0000000000000..affda13df6531 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -0,0 +1,50 @@ +/* + * 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.scheduler + +/** + * A simple listener for application events. + * + * This listener expects to hear events from a single application only. If events + * from multiple applications are seen, the behavior is unspecified. + */ +private[spark] class ApplicationEventListener extends SparkListener { + var appName = "" + var sparkUser = "" + var startTime = -1L + var endTime = -1L + + def applicationStarted = startTime != -1 + + def applicationFinished = endTime != -1 + + def applicationDuration: Long = { + val difference = endTime - startTime + if (applicationStarted && applicationFinished && difference > 0) difference else -1L + } + + override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { + appName = applicationStart.appName + startTime = applicationStart.time + sparkUser = applicationStart.sparkUser + } + + override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { + endTime = applicationEnd.time + } +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index 217f8825c2ae9..b983c16af14f4 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -17,11 +17,14 @@ package org.apache.spark.scheduler +import scala.collection.mutable + +import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.{Logging, SparkConf, SparkContext} import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, FileLogger} +import org.apache.spark.util.{FileLogger, JsonProtocol} /** * A SparkListener that logs events to persistent storage. @@ -36,6 +39,8 @@ import org.apache.spark.util.{JsonProtocol, FileLogger} private[spark] class EventLoggingListener(appName: String, conf: SparkConf) extends SparkListener with Logging { + import EventLoggingListener._ + private val shouldCompress = conf.getBoolean("spark.eventLog.compress", false) private val shouldOverwrite = conf.getBoolean("spark.eventLog.overwrite", false) private val outputBufferSize = conf.getInt("spark.eventLog.buffer.kb", 100) * 1024 @@ -46,17 +51,21 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) private val logger = new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) - // Information needed to replay the events logged by this listener later - val info = { - val compressionCodec = if (shouldCompress) { - Some(conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC)) - } else None - EventLoggingInfo(logDir, compressionCodec) + /** + * Begin logging events. + * If compression is used, log a file that indicates which compression library is used. + */ + def start() { + logInfo("Logging events to %s".format(logDir)) + if (shouldCompress) { + val codec = conf.get("spark.io.compression.codec", CompressionCodec.DEFAULT_COMPRESSION_CODEC) + logger.newFile(COMPRESSION_CODEC_PREFIX + codec) + } + logger.newFile(SPARK_VERSION_PREFIX + SparkContext.SPARK_VERSION) + logger.newFile(LOG_PREFIX + logger.fileIndex) } - logInfo("Logging events to %s".format(logDir)) - - /** Log the event as JSON */ + /** Log the event as JSON. */ private def logEvent(event: SparkListenerEvent, flushLogger: Boolean = false) { val eventJson = compact(render(JsonProtocol.sparkEventToJson(event))) logger.logLine(eventJson) @@ -90,9 +99,118 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) logEvent(event, flushLogger = true) override def onUnpersistRDD(event: SparkListenerUnpersistRDD) = logEvent(event, flushLogger = true) + override def onApplicationStart(event: SparkListenerApplicationStart) = + logEvent(event, flushLogger = true) + override def onApplicationEnd(event: SparkListenerApplicationEnd) = + logEvent(event, flushLogger = true) + + /** + * Stop logging events. + * In addition, create an empty special file to indicate application completion. + */ + def stop() = { + logger.newFile(APPLICATION_COMPLETE) + logger.stop() + } +} + +private[spark] object EventLoggingListener extends Logging { + val LOG_PREFIX = "EVENT_LOG_" + val SPARK_VERSION_PREFIX = "SPARK_VERSION_" + val COMPRESSION_CODEC_PREFIX = "COMPRESSION_CODEC_" + val APPLICATION_COMPLETE = "APPLICATION_COMPLETE" + + // A cache for compression codecs to avoid creating the same codec many times + private val codecMap = new mutable.HashMap[String, CompressionCodec] + + def isEventLogFile(fileName: String): Boolean = { + fileName.startsWith(LOG_PREFIX) + } + + def isSparkVersionFile(fileName: String): Boolean = { + fileName.startsWith(SPARK_VERSION_PREFIX) + } + + def isCompressionCodecFile(fileName: String): Boolean = { + fileName.startsWith(COMPRESSION_CODEC_PREFIX) + } + + def isApplicationCompleteFile(fileName: String): Boolean = { + fileName == APPLICATION_COMPLETE + } + + def parseSparkVersion(fileName: String): String = { + if (isSparkVersionFile(fileName)) { + fileName.replaceAll(SPARK_VERSION_PREFIX, "") + } else "" + } + + def parseCompressionCodec(fileName: String): String = { + if (isCompressionCodecFile(fileName)) { + fileName.replaceAll(COMPRESSION_CODEC_PREFIX, "") + } else "" + } + + /** + * Parse the event logging information associated with the logs in the given directory. + * + * Specifically, this looks for event log files, the Spark version file, the compression + * codec file (if event logs are compressed), and the application completion file (if the + * application has run to completion). + */ + def parseLoggingInfo(logDir: Path, fileSystem: FileSystem): EventLoggingInfo = { + try { + val fileStatuses = fileSystem.listStatus(logDir) + val filePaths = + if (fileStatuses != null) { + fileStatuses.filter(!_.isDir).map(_.getPath).toSeq + } else { + Seq[Path]() + } + if (filePaths.isEmpty) { + logWarning("No files found in logging directory %s".format(logDir)) + } + EventLoggingInfo( + logPaths = filePaths.filter { path => isEventLogFile(path.getName) }, + sparkVersion = filePaths + .find { path => isSparkVersionFile(path.getName) } + .map { path => parseSparkVersion(path.getName) } + .getOrElse(""), + compressionCodec = filePaths + .find { path => isCompressionCodecFile(path.getName) } + .map { path => + val codec = EventLoggingListener.parseCompressionCodec(path.getName) + val conf = new SparkConf + conf.set("spark.io.compression.codec", codec) + codecMap.getOrElseUpdate(codec, CompressionCodec.createCodec(conf)) + }, + applicationComplete = filePaths.exists { path => isApplicationCompleteFile(path.getName) } + ) + } catch { + case t: Throwable => + logError("Exception in parsing logging info from directory %s".format(logDir), t) + EventLoggingInfo.empty + } + } - def stop() = logger.stop() + /** + * Parse the event logging information associated with the logs in the given directory. + */ + def parseLoggingInfo(logDir: String, fileSystem: FileSystem): EventLoggingInfo = { + parseLoggingInfo(new Path(logDir), fileSystem) + } } -// If compression is not enabled, compressionCodec is None -private[spark] case class EventLoggingInfo(logDir: String, compressionCodec: Option[String]) + +/** + * Information needed to process the event logs associated with an application. + */ +private[spark] case class EventLoggingInfo( + logPaths: Seq[Path], + sparkVersion: String, + compressionCodec: Option[CompressionCodec], + applicationComplete: Boolean = false) + +private[spark] object EventLoggingInfo { + def empty = EventLoggingInfo(Seq[Path](), "", None, applicationComplete = false) +} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index db76178b65501..b03665fd56d33 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -18,7 +18,6 @@ package org.apache.spark.scheduler import java.io.InputStream -import java.net.URI import scala.io.Source @@ -26,63 +25,47 @@ import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ -import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.Logging import org.apache.spark.io.CompressionCodec -import org.apache.spark.util.{JsonProtocol, Utils} +import org.apache.spark.util.JsonProtocol /** - * An EventBus that replays logged events from persisted storage + * A SparkListenerBus that replays logged events from persisted storage. + * + * This class expects files to be appropriately prefixed as specified in EventLoggingListener. + * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. */ -private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus with Logging { - private val compressed = conf.getBoolean("spark.eventLog.compress", false) +private[spark] class ReplayListenerBus( + logPaths: Seq[Path], + fileSystem: FileSystem, + compressionCodec: Option[CompressionCodec]) + extends SparkListenerBus with Logging { - // Only used if compression is enabled - private lazy val compressionCodec = CompressionCodec.createCodec(conf) + private var replayed = false - /** - * Return a list of paths representing log files in the given directory. - */ - private def getLogFilePaths(logDir: String, fileSystem: FileSystem): Array[Path] = { - val path = new Path(logDir) - if (!fileSystem.exists(path) || !fileSystem.getFileStatus(path).isDir) { - logWarning("Log path provided is not a valid directory: %s".format(logDir)) - return Array[Path]() - } - val logStatus = fileSystem.listStatus(path) - if (logStatus == null || !logStatus.exists(!_.isDir)) { - logWarning("Log path provided contains no log files: %s".format(logDir)) - return Array[Path]() - } - logStatus.filter(!_.isDir).map(_.getPath).sortBy(_.getName) + if (logPaths.length == 0) { + logWarning("Log path provided contains no log files.") } /** * Replay each event in the order maintained in the given logs. + * This should only be called exactly once. */ - def replay(logDir: String): Boolean = { - val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - val logPaths = getLogFilePaths(logDir, fileSystem) - if (logPaths.length == 0) { - return false - } - + def replay() { + assert(!replayed, "ReplayListenerBus cannot replay events more than once") logPaths.foreach { path => // Keep track of input streams at all levels to close them later // This is necessary because an exception can occur in between stream initializations var fileStream: Option[InputStream] = None var bufferedStream: Option[InputStream] = None var compressStream: Option[InputStream] = None - var currentLine = "" + var currentLine = "" try { - currentLine = "" fileStream = Some(fileSystem.open(path)) bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) - compressStream = - if (compressed) { - Some(compressionCodec.compressedInputStream(bufferedStream.get)) - } else bufferedStream + compressStream = Some(wrapForCompression(bufferedStream.get)) - // Parse each line as an event and post it to all attached listeners + // Parse each line as an event and post the event to all attached listeners val lines = Source.fromInputStream(compressStream.get).getLines() lines.foreach { line => currentLine = line @@ -98,7 +81,11 @@ private[spark] class ReplayListenerBus(conf: SparkConf) extends SparkListenerBus compressStream.foreach(_.close()) } } - fileSystem.close() - true + replayed = true + } + + /** If a compression codec is specified, wrap the given stream in a compression stream. */ + private def wrapForCompression(stream: InputStream): InputStream = { + compressionCodec.map(_.compressedInputStream(stream)).getOrElse(stream) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala index ced20350d5356..378cf1aaebe7b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListener.scala @@ -75,6 +75,11 @@ case class SparkListenerBlockManagerRemoved(blockManagerId: BlockManagerId) @DeveloperApi case class SparkListenerUnpersistRDD(rddId: Int) extends SparkListenerEvent +case class SparkListenerApplicationStart(appName: String, time: Long, sparkUser: String) + extends SparkListenerEvent + +case class SparkListenerApplicationEnd(time: Long) extends SparkListenerEvent + /** An event used in the listener to shutdown the listener daemon thread. */ private[spark] case object SparkListenerShutdown extends SparkListenerEvent @@ -141,6 +146,16 @@ trait SparkListener { * Called when an RDD is manually unpersisted by the application */ def onUnpersistRDD(unpersistRDD: SparkListenerUnpersistRDD) { } + + /** + * Called when the application starts + */ + def onApplicationStart(applicationStart: SparkListenerApplicationStart) { } + + /** + * Called when the application ends + */ + def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { } } /** diff --git a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala index 729e120497571..d6df193d9bcf8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SparkListenerBus.scala @@ -61,6 +61,10 @@ private[spark] trait SparkListenerBus { sparkListeners.foreach(_.onBlockManagerRemoved(blockManagerRemoved)) case unpersistRDD: SparkListenerUnpersistRDD => sparkListeners.foreach(_.onUnpersistRDD(unpersistRDD)) + case applicationStart: SparkListenerApplicationStart => + sparkListeners.foreach(_.onApplicationStart(applicationStart)) + case applicationEnd: SparkListenerApplicationEnd => + sparkListeners.foreach(_.onApplicationEnd(applicationEnd)) case SparkListenerShutdown => } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 25b7472a99cdb..936e9db80573d 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -49,7 +49,7 @@ private[spark] class SparkDeploySchedulerBackend( "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, - sparkHome, sc.ui.appUIAddress, sc.eventLoggingInfo) + sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) client = new AppClient(sc.env.actorSystem, masters, appDesc, this, conf) client.start() diff --git a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala index 555486830a769..132502b75f8cd 100644 --- a/core/src/main/scala/org/apache/spark/storage/FileSegment.scala +++ b/core/src/main/scala/org/apache/spark/storage/FileSegment.scala @@ -23,6 +23,6 @@ import java.io.File * References a particular segment of a file (potentially the entire file), * based off an offset and a length. */ -private[spark] class FileSegment(val file: File, val offset: Long, val length : Long) { +private[spark] class FileSegment(val file: File, val offset: Long, val length: Long) { override def toString = "(name=%s, offset=%d, length=%d)".format(file.getName, offset, length) } diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index f53df7fbedf39..b8e6e15880bf5 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -34,23 +34,22 @@ private[spark] class SparkUI( val sc: SparkContext, conf: SparkConf, val listenerBus: SparkListenerBus, - val appName: String, + var appName: String, val basePath: String = "") - extends Logging { + extends WebUI("SparkUI") with Logging { def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) - def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = - this(null, conf, listenerBus, appName, basePath) + def this(listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, new SparkConf, listenerBus, appName, basePath) // If SparkContext is not provided, assume the associated application is not live val live = sc != null val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - private val bindHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(bindHost) - private val port = conf.get("spark.ui.port", SparkUI.DEFAULT_PORT).toInt - private var serverInfo: Option[ServerInfo] = None + private val localHost = Utils.localHostName() + private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) + private val port = conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) private val storage = new BlockManagerUI(this) private val jobs = new JobProgressUI(this) @@ -77,20 +76,10 @@ private[spark] class SparkUI( // Maintain executor storage status through Spark events val storageStatusListener = new StorageStatusListener - /** Bind the HTTP server which backs this web interface */ - def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) - logInfo("Started Spark Web UI at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Spark JettyUtils", e) - System.exit(1) - } + def setAppName(name: String) { + appName = name } - def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) - /** Initialize all components of the server */ def start() { storage.start() @@ -106,9 +95,21 @@ private[spark] class SparkUI( listenerBus.addListener(exec.listener) } - def stop() { - assert(serverInfo.isDefined, "Attempted to stop a SparkUI that was not bound to a server!") - serverInfo.get.server.stop() + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) + logInfo("Started Spark web UI at http://%s:%d".format(publicHost, boundPort)) + } catch { + case e: Exception => + logError("Failed to create Spark web UI", e) + System.exit(1) + } + } + + /** Stop the server behind this web interface. Only valid after bind(). */ + override def stop() { + super.stop() logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } @@ -117,6 +118,6 @@ private[spark] class SparkUI( } private[spark] object SparkUI { - val DEFAULT_PORT = "4040" + val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index a7b872f3445a4..2cc7582eca8a3 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -20,6 +20,25 @@ package org.apache.spark.ui import java.text.SimpleDateFormat import java.util.Date +private[spark] abstract class WebUI(name: String) { + protected var serverInfo: Option[ServerInfo] = None + + /** + * Bind to the HTTP server behind this web interface. + * Overridden implementation should set serverInfo. + */ + def bind() { } + + /** Return the actual port to which this server is bound. Only valid after bind(). */ + def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) + + /** Stop the server behind this web interface. Only valid after bind(). */ + def stop() { + assert(serverInfo.isDefined, "Attempted to stop %s before binding to a server!".format(name)) + serverInfo.get.server.stop() + } +} + /** * Utilities used throughout the web UI. */ @@ -45,6 +64,6 @@ private[spark] object WebUI { return "%.0f min".format(minutes) } val hours = minutes / 60 - return "%.1f h".format(hours) + "%.1f h".format(hours) } } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala index 23e90c34d5b33..33df97187ea78 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala @@ -29,10 +29,11 @@ import org.apache.spark.ui.JettyUtils._ import org.apache.spark.ui.Page.Environment private[ui] class EnvironmentUI(parent: SparkUI) { - private val appName = parent.appName private val basePath = parent.basePath private var _listener: Option[EnvironmentListener] = None + private def appName = parent.appName + lazy val listener = _listener.get def start() { diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala index 031ed88a493a8..77a38a1d3aa7c 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala @@ -33,10 +33,11 @@ import org.apache.spark.ui.{SparkUI, UIUtils} import org.apache.spark.util.Utils private[ui] class ExecutorsUI(parent: SparkUI) { - private val appName = parent.appName private val basePath = parent.basePath private var _listener: Option[ExecutorsListener] = None + private def appName = parent.appName + lazy val listener = _listener.get def start() { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 70d62b66a4829..f811aff616bcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -27,13 +27,14 @@ import org.apache.spark.ui.UIUtils /** Page showing list of all ongoing and recently finished stages and pools */ private[ui] class IndexPage(parent: JobProgressUI) { - private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index b2c67381cc3da..ad1a12cdc4e36 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -29,7 +29,6 @@ import org.apache.spark.util.Utils /** Web UI showing progress status of all jobs in the given SparkContext. */ private[ui] class JobProgressUI(parent: SparkUI) { - val appName = parent.appName val basePath = parent.basePath val live = parent.live val sc = parent.sc @@ -42,6 +41,8 @@ private[ui] class JobProgressUI(parent: SparkUI) { private val poolPage = new PoolPage(this) private var _listener: Option[JobProgressListener] = None + def appName = parent.appName + def start() { val conf = if (live) sc.conf else new SparkConf _listener = Some(new JobProgressListener(conf)) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index bd33182b70059..3638e6035ba81 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -27,12 +27,13 @@ import org.apache.spark.ui.UIUtils /** Page showing specific pool details */ private[ui] class PoolPage(parent: JobProgressUI) { - private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val poolName = request.getParameter("poolname") diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0c55f2ee7e944..0bcbd7461cc5b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -28,10 +28,11 @@ import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ private[ui] class StagePage(parent: JobProgressUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val stageId = request.getParameter("id").toInt diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala index a7b24ff695214..16996a2da1e72 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala @@ -30,7 +30,6 @@ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ private[ui] class BlockManagerUI(parent: SparkUI) { - val appName = parent.appName val basePath = parent.basePath private val indexPage = new IndexPage(this) @@ -39,6 +38,8 @@ private[ui] class BlockManagerUI(parent: SparkUI) { lazy val listener = _listener.get + def appName = parent.appName + def start() { _listener = Some(new BlockManagerListener(parent.storageStatusListener)) } diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala index 0fa461e5e9d27..4f6acc30a88c4 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala @@ -28,10 +28,11 @@ import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ private[ui] class IndexPage(parent: BlockManagerUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { val rdds = listener.rddInfoList diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 3f42eba4ece00..75ee9976d7b5f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -28,10 +28,11 @@ import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ private[ui] class RDDPage(parent: BlockManagerUI) { - private val appName = parent.appName private val basePath = parent.basePath private lazy val listener = parent.listener + private def appName = parent.appName + def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt val storageStatusList = listener.storageStatusList diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index b5f2ec6831d26..0080a8b342b05 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -49,7 +49,7 @@ private[spark] class FileLogger( } private val fileSystem = Utils.getHadoopFileSystem(new URI(logDir)) - private var fileIndex = 0 + var fileIndex = 0 // Only used if compression is enabled private lazy val compressionCodec = CompressionCodec.createCodec(conf) @@ -57,10 +57,9 @@ private[spark] class FileLogger( // Only defined if the file system scheme is not local private var hadoopDataStream: Option[FSDataOutputStream] = None - private var writer: Option[PrintWriter] = { - createLogDir() - Some(createWriter()) - } + private var writer: Option[PrintWriter] = None + + createLogDir() /** * Create a logging directory with the given path. @@ -84,8 +83,8 @@ private[spark] class FileLogger( /** * Create a new writer for the file identified by the given path. */ - private def createWriter(): PrintWriter = { - val logPath = logDir + "/" + fileIndex + private def createWriter(fileName: String): PrintWriter = { + val logPath = logDir + "/" + fileName val uri = new URI(logPath) /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). @@ -147,13 +146,17 @@ private[spark] class FileLogger( } /** - * Start a writer for a new file if one does not already exit. + * Start a writer for a new file, closing the existing one if it exists. + * @param fileName Name of the new file, defaulting to the file index if not provided. */ - def start() { - writer.getOrElse { - fileIndex += 1 - writer = Some(createWriter()) + def newFile(fileName: String = "") { + fileIndex += 1 + writer.foreach(_.close()) + val name = fileName match { + case "" => fileIndex.toString + case _ => fileName } + writer = Some(createWriter(name)) } /** diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 19654892bf661..d990fd49ef834 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -62,6 +62,10 @@ private[spark] object JsonProtocol { blockManagerRemovedToJson(blockManagerRemoved) case unpersistRDD: SparkListenerUnpersistRDD => unpersistRDDToJson(unpersistRDD) + case applicationStart: SparkListenerApplicationStart => + applicationStartToJson(applicationStart) + case applicationEnd: SparkListenerApplicationEnd => + applicationEndToJson(applicationEnd) // Not used, but keeps compiler happy case SparkListenerShutdown => JNothing @@ -157,6 +161,18 @@ private[spark] object JsonProtocol { ("RDD ID" -> unpersistRDD.rddId) } + def applicationStartToJson(applicationStart: SparkListenerApplicationStart): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationStart)) ~ + ("App Name" -> applicationStart.appName) ~ + ("Timestamp" -> applicationStart.time) ~ + ("User" -> applicationStart.sparkUser) + } + + def applicationEndToJson(applicationEnd: SparkListenerApplicationEnd): JValue = { + ("Event" -> Utils.getFormattedClassName(applicationEnd)) ~ + ("Timestamp" -> applicationEnd.time) + } + /** ------------------------------------------------------------------- * * JSON serialization methods for classes SparkListenerEvents depend on | @@ -346,6 +362,8 @@ private[spark] object JsonProtocol { val blockManagerAdded = Utils.getFormattedClassName(SparkListenerBlockManagerAdded) val blockManagerRemoved = Utils.getFormattedClassName(SparkListenerBlockManagerRemoved) val unpersistRDD = Utils.getFormattedClassName(SparkListenerUnpersistRDD) + val applicationStart = Utils.getFormattedClassName(SparkListenerApplicationStart) + val applicationEnd = Utils.getFormattedClassName(SparkListenerApplicationEnd) (json \ "Event").extract[String] match { case `stageSubmitted` => stageSubmittedFromJson(json) @@ -359,6 +377,8 @@ private[spark] object JsonProtocol { case `blockManagerAdded` => blockManagerAddedFromJson(json) case `blockManagerRemoved` => blockManagerRemovedFromJson(json) case `unpersistRDD` => unpersistRDDFromJson(json) + case `applicationStart` => applicationStartFromJson(json) + case `applicationEnd` => applicationEndFromJson(json) } } @@ -430,6 +450,17 @@ private[spark] object JsonProtocol { SparkListenerUnpersistRDD((json \ "RDD ID").extract[Int]) } + def applicationStartFromJson(json: JValue): SparkListenerApplicationStart = { + val appName = (json \ "App Name").extract[String] + val time = (json \ "Timestamp").extract[Long] + val sparkUser = (json \ "User").extract[String] + SparkListenerApplicationStart(appName, time, sparkUser) + } + + def applicationEndFromJson(json: JValue): SparkListenerApplicationEnd = { + SparkListenerApplicationEnd((json \ "Timestamp").extract[Long]) + } + /** --------------------------------------------------------------------- * * JSON deserialization methods for classes SparkListenerEvents depend on | diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 59da51f3e0297..166f48ce7342e 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -26,7 +26,6 @@ import java.util.concurrent.{ConcurrentHashMap, Executors, ThreadPoolExecutor} import scala.collection.JavaConversions._ import scala.collection.Map import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.SortedSet import scala.io.Source import scala.reflect.ClassTag @@ -1022,4 +1021,11 @@ private[spark] object Utils extends Logging { def getHadoopFileSystem(path: URI): FileSystem = { FileSystem.get(path, SparkHadoopUtil.get.newConfiguration()) } + + /** + * Return a Hadoop FileSystem with the scheme encoded in the given path. + */ + def getHadoopFileSystem(path: String): FileSystem = { + getHadoopFileSystem(new URI(path)) + } } diff --git a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala index beac656f573b4..8c06a2d9aa4ab 100644 --- a/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/jobs/JobProgressListenerSuite.scala @@ -20,7 +20,7 @@ package org.apache.spark.ui.jobs import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.{LocalSparkContext, SparkConf, SparkContext, Success} +import org.apache.spark.{LocalSparkContext, SparkConf, Success} import org.apache.spark.executor.{ShuffleReadMetrics, TaskMetrics} import org.apache.spark.scheduler._ import org.apache.spark.util.Utils diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 0342a8aff3c28..f75297a02dc8b 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.util -import java.util.{Properties, UUID} +import java.util.Properties import scala.collection.Map @@ -52,6 +52,8 @@ class JsonProtocolSuite extends FunSuite { val blockManagerRemoved = SparkListenerBlockManagerRemoved( BlockManagerId("Scarce", "to be counted...", 100, 200)) val unpersistRdd = SparkListenerUnpersistRDD(12345) + val applicationStart = SparkListenerApplicationStart("The winner of all", 42L, "Garfield") + val applicationEnd = SparkListenerApplicationEnd(42L) testEvent(stageSubmitted, stageSubmittedJsonString) testEvent(stageCompleted, stageCompletedJsonString) @@ -64,6 +66,8 @@ class JsonProtocolSuite extends FunSuite { testEvent(blockManagerAdded, blockManagerAddedJsonString) testEvent(blockManagerRemoved, blockManagerRemovedJsonString) testEvent(unpersistRdd, unpersistRDDJsonString) + testEvent(applicationStart, applicationStartJsonString) + testEvent(applicationEnd, applicationEndJsonString) } test("Dependent Classes") { @@ -208,7 +212,13 @@ class JsonProtocolSuite extends FunSuite { case (e1: SparkListenerBlockManagerRemoved, e2: SparkListenerBlockManagerRemoved) => assertEquals(e1.blockManagerId, e2.blockManagerId) case (e1: SparkListenerUnpersistRDD, e2: SparkListenerUnpersistRDD) => - assert(e1.rddId === e2.rddId) + assert(e1.rddId == e2.rddId) + case (e1: SparkListenerApplicationStart, e2: SparkListenerApplicationStart) => + assert(e1.appName == e2.appName) + assert(e1.time == e2.time) + assert(e1.sparkUser == e2.sparkUser) + case (e1: SparkListenerApplicationEnd, e2: SparkListenerApplicationEnd) => + assert(e1.time == e2.time) case (SparkListenerShutdown, SparkListenerShutdown) => case _ => fail("Events don't match in types!") } @@ -553,4 +563,14 @@ class JsonProtocolSuite extends FunSuite { {"Event":"SparkListenerUnpersistRDD","RDD ID":12345} """ + private val applicationStartJsonString = + """ + {"Event":"SparkListenerApplicationStart","App Name":"The winner of all","Timestamp":42, + "User":"Garfield"} + """ + + private val applicationEndJsonString = + """ + {"Event":"SparkListenerApplicationEnd","Timestamp":42} + """ } diff --git a/docs/monitoring.md b/docs/monitoring.md index 15bfb041780da..4c91c3a5929bf 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -12,17 +12,77 @@ displays useful information about the application. This includes: * A list of scheduler stages and tasks * A summary of RDD sizes and memory usage -* Information about the running executors * Environmental information. +* Information about the running executors You can access this interface by simply opening `http://:4040` in a web browser. -If multiple SparkContexts are running on the same host, they will bind to succesive ports +If multiple SparkContexts are running on the same host, they will bind to successive ports beginning with 4040 (4041, 4042, etc). -Spark's Standalone Mode cluster manager also has its own -[web UI](spark-standalone.html#monitoring-and-logging). +Note that this information is only available for the duration of the application by default. +To view the web UI after the fact, set `spark.eventLog.enabled` to true before starting the +application. This configures Spark to log Spark events that encode the information displayed +in the UI to persisted storage. -Note that in both of these UIs, the tables are sortable by clicking their headers, +## Viewing After the Fact + +Spark's Standalone Mode cluster manager also has its own +[web UI](spark-standalone.html#monitoring-and-logging). If an application has logged events over +the course of its lifetime, then the Standalone master's web UI will automatically re-render the +application's UI after the application has finished. + +If Spark is run on Mesos or YARN, it is still possible to reconstruct the UI of a finished +application through Spark's history server, provided that the application's event logs exist. +You can start a the history server by executing: + + ./sbin/start-history-server.sh + +The base logging directory must be supplied, and should contain sub-directories that each +represents an application's event logs. This creates a web interface at +`http://:18080` by default. The history server depends on the following variables: + + + + + + + + + + + +
    Environment VariableMeaning
    SPARK_DAEMON_MEMORYMemory to allocate to the history server. (default: 512m).
    SPARK_DAEMON_JAVA_OPTSJVM options for the history server (default: none).
    + +Further, the history server can be configured as follows: + + + + + + + + + + + + + + + + + + +
    Property NameDefaultMeaning
    spark.history.updateInterval10 + The period, in seconds, at which information displayed by this history server is updated. + Each update checks for any changes made to the event logs in persisted storage. +
    spark.history.retainedApplications250 + The number of application UIs to retain. If this cap is exceeded, then the oldest + applications will be removed. +
    spark.history.ui.port18080 + The port to which the web interface of the history server binds. +
    + +Note that in all of these UIs, the tables are sortable by clicking their headers, making it easy to identify slow tasks, data skew, etc. # Metrics diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala index 3ebf288130fb6..910b31d209e13 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoopInit.scala @@ -116,14 +116,14 @@ trait SparkILoopInit { } } - def initializeSpark() { + def initializeSpark() { intp.beQuietDuring { command(""" @transient val sc = org.apache.spark.repl.Main.interp.createSparkContext(); """) command("import org.apache.spark.SparkContext._") } - echo("Spark context available as sc.") + echo("Spark context available as sc.") } // code to be executed only after the interpreter is initialized diff --git a/sbin/start-history-server.sh b/sbin/start-history-server.sh new file mode 100755 index 0000000000000..4a90c68763b68 --- /dev/null +++ b/sbin/start-history-server.sh @@ -0,0 +1,37 @@ +#!/usr/bin/env 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. +# + +# Starts the history server on the machine this script is executed on. +# +# Usage: start-history-server.sh [] +# Example: ./start-history-server.sh --dir /tmp/spark-events --port 18080 +# + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +if [ $# -lt 1 ]; then + echo "Usage: ./start-history-server.sh " + echo "Example: ./start-history-server.sh /tmp/spark-events" + exit +fi + +LOG_DIR=$1 + +"$sbin"/spark-daemon.sh start org.apache.spark.deploy.history.HistoryServer 1 --dir "$LOG_DIR" diff --git a/sbin/stop-history-server.sh b/sbin/stop-history-server.sh new file mode 100755 index 0000000000000..c0034ad641cbe --- /dev/null +++ b/sbin/stop-history-server.sh @@ -0,0 +1,25 @@ +#!/usr/bin/env 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. +# + +# Stops the history server on the machine this script is executed on. + +sbin=`dirname "$0"` +sbin=`cd "$sbin"; pwd` + +"$sbin"/spark-daemon.sh stop org.apache.spark.deploy.history.HistoryServer 1 From 3bd312940e2f5250edaf3e88d6c23de25bb1d0a9 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 10 Apr 2014 11:17:41 -0700 Subject: [PATCH 251/397] SPARK-1428: MLlib should convert non-float64 NumPy arrays to float64 instead of complaining Author: Sandeep Closes #356 from techaddict/1428 and squashes the following commits: 3bdf5f6 [Sandeep] SPARK-1428: MLlib should convert non-float64 NumPy arrays to float64 instead of complaining --- python/pyspark/mllib/_common.py | 18 ++++++++++++++---- 1 file changed, 14 insertions(+), 4 deletions(-) diff --git a/python/pyspark/mllib/_common.py b/python/pyspark/mllib/_common.py index 20a0e309d1494..7ef251d24c77e 100644 --- a/python/pyspark/mllib/_common.py +++ b/python/pyspark/mllib/_common.py @@ -15,8 +15,9 @@ # limitations under the License. # -from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape +from numpy import ndarray, copyto, float64, int64, int32, ones, array_equal, array, dot, shape, complex, issubdtype from pyspark import SparkContext, RDD +import numpy as np from pyspark.serializers import Serializer import struct @@ -47,13 +48,22 @@ def _deserialize_byte_array(shape, ba, offset): return ar.copy() def _serialize_double_vector(v): - """Serialize a double vector into a mutually understood format.""" + """Serialize a double vector into a mutually understood format. + + >>> x = array([1,2,3]) + >>> y = _deserialize_double_vector(_serialize_double_vector(x)) + >>> array_equal(y, array([1.0, 2.0, 3.0])) + True + """ if type(v) != ndarray: raise TypeError("_serialize_double_vector called on a %s; " "wanted ndarray" % type(v)) + """complex is only datatype that can't be converted to float64""" + if issubdtype(v.dtype, complex): + raise TypeError("_serialize_double_vector called on a %s; " + "wanted ndarray" % type(v)) if v.dtype != float64: - raise TypeError("_serialize_double_vector called on an ndarray of %s; " - "wanted ndarray of float64" % v.dtype) + v = v.astype(float64) if v.ndim != 1: raise TypeError("_serialize_double_vector called on a %ddarray; " "wanted a 1darray" % v.ndim) From 7b52b66312994d4dbf243eadb6d27eb06350a81f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 10 Apr 2014 14:43:29 -0700 Subject: [PATCH 252/397] Revert "SPARK-1433: Upgrade Mesos dependency to 0.17.0" This reverts commit 12c077d5aa0b76a808a55db625c9677a52bd43f9. --- .../cluster/mesos/CoarseMesosSchedulerBackend.scala | 6 ++---- .../scheduler/cluster/mesos/MesosSchedulerBackend.scala | 2 +- docs/_config.yml | 2 +- pom.xml | 6 +++--- project/SparkBuild.scala | 2 +- 5 files changed, 8 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index c478e685641d7..06b041e1fd9a9 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -194,12 +194,10 @@ private[spark] class CoarseMesosSchedulerBackend( .addResources(createResource("cpus", cpusToUse)) .addResources(createResource("mem", sc.executorMemory)) .build() - d.launchTasks(Collections.singletonList(offer.getId), - Collections.singletonList(task), - filters) + d.launchTasks(offer.getId, Collections.singletonList(task), filters) } else { // Filter it out - d.declineOffer(offer.getId, filters) + d.launchTasks(offer.getId, Collections.emptyList[MesosTaskInfo](), filters) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index f878ae338fc95..dfdcafe19fb93 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -223,7 +223,7 @@ private[spark] class MesosSchedulerBackend( // Reply to the offers val filters = Filters.newBuilder().setRefuseSeconds(1).build() // TODO: lower timeout? for (i <- 0 until offers.size) { - d.launchTasks(Collections.singletonList(offers(i).getId), mesosTasks(i), filters) + d.launchTasks(offers(i).getId, mesosTasks(i), filters) } } } finally { diff --git a/docs/_config.yml b/docs/_config.yml index bd5ed6c9220d2..d585b8c5ea763 100644 --- a/docs/_config.yml +++ b/docs/_config.yml @@ -7,6 +7,6 @@ SPARK_VERSION: 1.0.0-SNAPSHOT SPARK_VERSION_SHORT: 1.0.0 SCALA_BINARY_VERSION: "2.10" SCALA_VERSION: "2.10.4" -MESOS_VERSION: 0.17.0 +MESOS_VERSION: 0.13.0 SPARK_ISSUE_TRACKER_URL: https://spark-project.atlassian.net SPARK_GITHUB_URL: https://github.com/apache/spark diff --git a/pom.xml b/pom.xml index 11511bcb9da52..c03bb35c99442 100644 --- a/pom.xml +++ b/pom.xml @@ -112,7 +112,7 @@ 2.10.4 2.10 - 0.17.0 + 0.13.0 org.spark-project.akka 2.2.3-shaded-protobuf 1.7.5 @@ -848,7 +848,7 @@ - + hadoop-provided @@ -893,6 +893,6 @@
    - + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 694f90a83ab67..21163760e6277 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -333,7 +333,7 @@ object SparkBuild extends Build { "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", - "org.apache.mesos" % "mesos" % "0.17.0", + "org.apache.mesos" % "mesos" % "0.13.0", "commons-net" % "commons-net" % "2.2", "net.java.dev.jets3t" % "jets3t" % "0.7.1" excludeAll(excludeCommonsLogging), "org.apache.derby" % "derby" % "10.4.2.0" % "test", From f0466625200842f3cc486e9aa1caa417586be533 Mon Sep 17 00:00:00 2001 From: Andrew Ash Date: Thu, 10 Apr 2014 14:59:58 -0700 Subject: [PATCH 253/397] Update tuning.md http://stackoverflow.com/questions/9699071/what-is-the-javas-internal-represention-for-string-modified-utf-8-utf-16 Author: Andrew Ash Closes #384 from ash211/patch-2 and squashes the following commits: da1b0be [Andrew Ash] Update tuning.md --- docs/tuning.md | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/docs/tuning.md b/docs/tuning.md index 093df3187a789..cc069f0e84b9c 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -90,9 +90,10 @@ than the "raw" data inside their fields. This is due to several reasons: * Each distinct Java object has an "object header", which is about 16 bytes and contains information such as a pointer to its class. For an object with very little data in it (say one `Int` field), this can be bigger than the data. -* Java Strings have about 40 bytes of overhead over the raw string data (since they store it in an +* Java `String`s have about 40 bytes of overhead over the raw string data (since they store it in an array of `Char`s and keep extra data such as the length), and store each character - as *two* bytes due to Unicode. Thus a 10-character string can easily consume 60 bytes. + as *two* bytes due to `String`'s internal usage of UTF-16 encoding. Thus a 10-character string can + easily consume 60 bytes. * Common collection classes, such as `HashMap` and `LinkedList`, use linked data structures, where there is a "wrapper" object for each entry (e.g. `Map.Entry`). This object not only has a header, but also pointers (typically 8 bytes each) to the next object in the list. From 930b70f0523e96fe01c1317ef7fad1b76b36d4d9 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 10 Apr 2014 15:04:13 -0700 Subject: [PATCH 254/397] Remove Unnecessary Whitespace's stack these together in a commit else they show up chunk by chunk in different commits. Author: Sandeep Closes #380 from techaddict/white_space and squashes the following commits: b58f294 [Sandeep] Remove Unnecessary Whitespace's --- .../org/apache/spark/bagel/BagelSuite.scala | 4 +- .../api/java/function/FlatMapFunction.java | 2 +- .../api/java/function/FlatMapFunction2.java | 2 +- .../org/apache/spark/HttpFileServer.scala | 14 +- .../scala/org/apache/spark/HttpServer.scala | 6 +- .../scala/org/apache/spark/Partition.scala | 2 +- .../org/apache/spark/SecurityManager.scala | 88 +++---- .../org/apache/spark/SparkException.scala | 2 +- .../org/apache/spark/SparkHadoopWriter.scala | 20 +- .../org/apache/spark/SparkSaslClient.scala | 10 +- .../org/apache/spark/SparkSaslServer.scala | 6 +- .../scala/org/apache/spark/TestUtils.scala | 2 +- .../spark/broadcast/TorrentBroadcast.scala | 2 +- .../apache/spark/deploy/ClientArguments.scala | 2 +- .../spark/deploy/worker/WorkerArguments.scala | 8 +- .../spark/deploy/worker/ui/IndexPage.scala | 2 +- .../CoarseGrainedExecutorBackend.scala | 2 +- .../spark/executor/ExecutorExitCode.scala | 8 +- .../executor/ExecutorURLClassLoader.scala | 2 +- .../apache/spark/metrics/sink/CsvSink.scala | 2 +- .../org/apache/spark/network/Connection.scala | 8 +- .../apache/spark/network/ConnectionId.scala | 6 +- .../spark/network/ConnectionManager.scala | 28 +-- .../spark/network/ConnectionManagerTest.scala | 24 +- .../apache/spark/network/ReceiverTest.scala | 2 +- .../spark/network/SecurityMessage.scala | 48 ++-- .../spark/network/netty/FileHeader.scala | 4 +- .../apache/spark/partial/PartialResult.scala | 4 +- .../apache/spark/rdd/DoubleRDDFunctions.scala | 8 +- .../spark/rdd/PartitionerAwareUnionRDD.scala | 2 +- .../spark/scheduler/DAGSchedulerEvent.scala | 2 +- .../spark/scheduler/LiveListenerBus.scala | 214 +++++++++--------- .../spark/storage/BlockFetcherIterator.scala | 2 +- .../apache/spark/storage/BlockManager.scala | 4 +- .../apache/spark/storage/BlockMessage.scala | 22 +- .../spark/storage/BlockMessageArray.scala | 26 +-- .../org/apache/spark/ui/JettyUtils.scala | 4 +- .../scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../apache/spark/util/ClosureCleaner.scala | 22 +- .../org/apache/spark/util/JsonProtocol.scala | 2 +- .../org/apache/spark/util/NextIterator.scala | 4 +- .../org/apache/spark/util/StatCounter.scala | 4 +- .../scala/org/apache/spark/util/Vector.scala | 2 +- .../spark/util/random/XORShiftRandom.scala | 16 +- .../org/apache/spark/AkkaUtilsSuite.scala | 20 +- .../scala/org/apache/spark/DriverSuite.scala | 2 +- .../org/apache/spark/FileServerSuite.scala | 4 +- .../scala/org/apache/spark/FileSuite.scala | 2 +- .../deploy/worker/WorkerWatcherSuite.scala | 2 +- .../WholeTextFileRecordReaderSuite.scala | 2 +- .../rdd/ParallelCollectionSplitSuite.scala | 26 +-- .../spark/scheduler/SparkListenerSuite.scala | 4 +- .../scheduler/TaskSchedulerImplSuite.scala | 2 +- .../scala/org/apache/spark/ui/UISuite.scala | 4 +- .../spark/util/ClosureCleanerSuite.scala | 2 +- .../apache/spark/util/NextIteratorSuite.scala | 4 +- .../util/random/XORShiftRandomSuite.scala | 20 +- .../streaming/mqtt/MQTTInputDStream.scala | 16 +- .../twitter/TwitterInputDStream.scala | 6 +- .../org/apache/spark/graphx/GraphOps.scala | 2 +- .../apache/spark/graphx/GraphOpsSuite.scala | 2 +- .../spark/mllib/optimization/Optimizer.scala | 2 +- .../GeneralizedLinearAlgorithm.scala | 4 +- .../spark/repl/ExecutorClassLoader.scala | 4 +- .../org/apache/spark/repl/SparkImports.scala | 2 +- .../spark/sql/catalyst/expressions/Cast.scala | 6 +- .../sql/catalyst/expressions/Expression.scala | 12 +- .../expressions/stringOperations.scala | 28 +-- .../spark/sql/catalyst/types/dataTypes.scala | 4 +- .../ExpressionEvaluationSuite.scala | 10 +- .../sql/ScalaReflectionRelationSuite.scala | 2 +- .../apache/spark/streaming/Checkpoint.scala | 14 +- .../org/apache/spark/streaming/Interval.scala | 8 +- .../org/apache/spark/streaming/Time.scala | 4 +- .../dstream/DStreamCheckpointData.scala | 2 +- .../streaming/dstream/FileInputDStream.scala | 2 +- .../streaming/dstream/QueueInputDStream.scala | 8 +- .../streaming/receivers/ActorReceiver.scala | 2 +- .../apache/spark/streaming/util/Clock.scala | 26 +-- .../spark/streaming/util/RawTextHelper.scala | 18 +- .../spark/streaming/util/RecurringTimer.scala | 8 +- .../apache/spark/streaming/JavaAPISuite.java | 2 +- 82 files changed, 467 insertions(+), 467 deletions(-) diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 9c37fadb78d2f..69144e3e657bf 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -28,9 +28,9 @@ class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializ class TestMessage(val targetId: String) extends Message[String] with Serializable class BagelSuite extends FunSuite with Assertions with BeforeAndAfter with Timeouts { - + var sc: SparkContext = _ - + after { if (sc != null) { sc.stop() diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java index fa75842047c6a..23f5fdd43631b 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction.java @@ -24,4 +24,4 @@ */ public interface FlatMapFunction extends Serializable { public Iterable call(T t) throws Exception; -} \ No newline at end of file +} diff --git a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java index d1fdec072443d..c48e92f535ff5 100644 --- a/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java +++ b/core/src/main/java/org/apache/spark/api/java/function/FlatMapFunction2.java @@ -24,4 +24,4 @@ */ public interface FlatMapFunction2 extends Serializable { public Iterable call(T1 t1, T2 t2) throws Exception; -} \ No newline at end of file +} diff --git a/core/src/main/scala/org/apache/spark/HttpFileServer.scala b/core/src/main/scala/org/apache/spark/HttpFileServer.scala index 3d7692ea8a49e..a6e300d345786 100644 --- a/core/src/main/scala/org/apache/spark/HttpFileServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpFileServer.scala @@ -24,13 +24,13 @@ import com.google.common.io.Files import org.apache.spark.util.Utils private[spark] class HttpFileServer(securityManager: SecurityManager) extends Logging { - + var baseDir : File = null var fileDir : File = null var jarDir : File = null var httpServer : HttpServer = null var serverUri : String = null - + def initialize() { baseDir = Utils.createTempDir() fileDir = new File(baseDir, "files") @@ -43,24 +43,24 @@ private[spark] class HttpFileServer(securityManager: SecurityManager) extends Lo serverUri = httpServer.uri logDebug("HTTP file server started at: " + serverUri) } - + def stop() { httpServer.stop() } - + def addFile(file: File) : String = { addFileToDir(file, fileDir) serverUri + "/files/" + file.getName } - + def addJar(file: File) : String = { addFileToDir(file, jarDir) serverUri + "/jars/" + file.getName } - + def addFileToDir(file: File, dir: File) : String = { Files.copy(file, new File(dir, file.getName)) dir + "/" + file.getName } - + } diff --git a/core/src/main/scala/org/apache/spark/HttpServer.scala b/core/src/main/scala/org/apache/spark/HttpServer.scala index cb5df25fa48df..7e9b517f901a2 100644 --- a/core/src/main/scala/org/apache/spark/HttpServer.scala +++ b/core/src/main/scala/org/apache/spark/HttpServer.scala @@ -83,19 +83,19 @@ private[spark] class HttpServer(resourceBase: File, securityManager: SecurityMan } } - /** + /** * Setup Jetty to the HashLoginService using a single user with our * shared secret. Configure it to use DIGEST-MD5 authentication so that the password * isn't passed in plaintext. */ private def setupSecurityHandler(securityMgr: SecurityManager): ConstraintSecurityHandler = { val constraint = new Constraint() - // use DIGEST-MD5 as the authentication mechanism + // use DIGEST-MD5 as the authentication mechanism constraint.setName(Constraint.__DIGEST_AUTH) constraint.setRoles(Array("user")) constraint.setAuthenticate(true) constraint.setDataConstraint(Constraint.DC_NONE) - + val cm = new ConstraintMapping() cm.setConstraint(constraint) cm.setPathSpec("/*") diff --git a/core/src/main/scala/org/apache/spark/Partition.scala b/core/src/main/scala/org/apache/spark/Partition.scala index 87914a061f5d7..27892dbd2a0bc 100644 --- a/core/src/main/scala/org/apache/spark/Partition.scala +++ b/core/src/main/scala/org/apache/spark/Partition.scala @@ -25,7 +25,7 @@ trait Partition extends Serializable { * Get the split's index within its parent RDD */ def index: Int - + // A better default implementation of HashCode override def hashCode(): Int = index } diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index 2237ee3bb7aad..b52f2d4f416b2 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -25,93 +25,93 @@ import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil -/** - * Spark class responsible for security. - * +/** + * Spark class responsible for security. + * * In general this class should be instantiated by the SparkEnv and most components - * should access it from that. There are some cases where the SparkEnv hasn't been + * should access it from that. There are some cases where the SparkEnv hasn't been * initialized yet and this class must be instantiated directly. - * + * * Spark currently supports authentication via a shared secret. * Authentication can be configured to be on via the 'spark.authenticate' configuration - * parameter. This parameter controls whether the Spark communication protocols do + * parameter. This parameter controls whether the Spark communication protocols do * authentication using the shared secret. This authentication is a basic handshake to * make sure both sides have the same shared secret and are allowed to communicate. - * If the shared secret is not identical they will not be allowed to communicate. - * - * The Spark UI can also be secured by using javax servlet filters. A user may want to - * secure the UI if it has data that other users should not be allowed to see. The javax - * servlet filter specified by the user can authenticate the user and then once the user - * is logged in, Spark can compare that user versus the view acls to make sure they are - * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' + * If the shared secret is not identical they will not be allowed to communicate. + * + * The Spark UI can also be secured by using javax servlet filters. A user may want to + * secure the UI if it has data that other users should not be allowed to see. The javax + * servlet filter specified by the user can authenticate the user and then once the user + * is logged in, Spark can compare that user versus the view acls to make sure they are + * authorized to view the UI. The configs 'spark.ui.acls.enable' and 'spark.ui.view.acls' * control the behavior of the acls. Note that the person who started the application * always has view access to the UI. * * Spark does not currently support encryption after authentication. - * + * * At this point spark has multiple communication protocols that need to be secured and * different underlying mechanisms are used depending on the protocol: * - * - Akka -> The only option here is to use the Akka Remote secure-cookie functionality. - * Akka remoting allows you to specify a secure cookie that will be exchanged - * and ensured to be identical in the connection handshake between the client - * and the server. If they are not identical then the client will be refused - * to connect to the server. There is no control of the underlying - * authentication mechanism so its not clear if the password is passed in + * - Akka -> The only option here is to use the Akka Remote secure-cookie functionality. + * Akka remoting allows you to specify a secure cookie that will be exchanged + * and ensured to be identical in the connection handshake between the client + * and the server. If they are not identical then the client will be refused + * to connect to the server. There is no control of the underlying + * authentication mechanism so its not clear if the password is passed in * plaintext or uses DIGEST-MD5 or some other mechanism. * Akka also has an option to turn on SSL, this option is not currently supported * but we could add a configuration option in the future. - * - * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty - * for the HttpServer. Jetty supports multiple authentication mechanisms - - * Basic, Digest, Form, Spengo, etc. It also supports multiple different login + * + * - HTTP for broadcast and file server (via HttpServer) -> Spark currently uses Jetty + * for the HttpServer. Jetty supports multiple authentication mechanisms - + * Basic, Digest, Form, Spengo, etc. It also supports multiple different login * services - Hash, JAAS, Spnego, JDBC, etc. Spark currently uses the HashLoginService - * to authenticate using DIGEST-MD5 via a single user and the shared secret. + * to authenticate using DIGEST-MD5 via a single user and the shared secret. * Since we are using DIGEST-MD5, the shared secret is not passed on the wire * in plaintext. * We currently do not support SSL (https), but Jetty can be configured to use it * so we could add a configuration option for this in the future. - * + * * The Spark HttpServer installs the HashLoginServer and configures it to DIGEST-MD5. - * Any clients must specify the user and password. There is a default + * Any clients must specify the user and password. There is a default * Authenticator installed in the SecurityManager to how it does the authentication * and in this case gets the user name and password from the request. * - * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously - * exchange messages. For this we use the Java SASL - * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 + * - ConnectionManager -> The Spark ConnectionManager uses java nio to asynchronously + * exchange messages. For this we use the Java SASL + * (Simple Authentication and Security Layer) API and again use DIGEST-MD5 * as the authentication mechanism. This means the shared secret is not passed * over the wire in plaintext. * Note that SASL is pluggable as to what mechanism it uses. We currently use * DIGEST-MD5 but this could be changed to use Kerberos or other in the future. * Spark currently supports "auth" for the quality of protection, which means * the connection is not supporting integrity or privacy protection (encryption) - * after authentication. SASL also supports "auth-int" and "auth-conf" which + * after authentication. SASL also supports "auth-int" and "auth-conf" which * SPARK could be support in the future to allow the user to specify the quality - * of protection they want. If we support those, the messages will also have to + * of protection they want. If we support those, the messages will also have to * be wrapped and unwrapped via the SaslServer/SaslClient.wrap/unwrap API's. - * - * Since the connectionManager does asynchronous messages passing, the SASL + * + * Since the connectionManager does asynchronous messages passing, the SASL * authentication is a bit more complex. A ConnectionManager can be both a client * and a Server, so for a particular connection is has to determine what to do. - * A ConnectionId was added to be able to track connections and is used to + * A ConnectionId was added to be able to track connections and is used to * match up incoming messages with connections waiting for authentication. * If its acting as a client and trying to send a message to another ConnectionManager, * it blocks the thread calling sendMessage until the SASL negotiation has occurred. * The ConnectionManager tracks all the sendingConnections using the ConnectionId * and waits for the response from the server and does the handshake. * - * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters + * - HTTP for the Spark UI -> the UI was changed to use servlets so that javax servlet filters * can be used. Yarn requires a specific AmIpFilter be installed for security to work * properly. For non-Yarn deployments, users can write a filter to go through a * companies normal login service. If an authentication filter is in place then the * SparkUI can be configured to check the logged in user against the list of users who * have view acls to see if that user is authorized. - * The filters can also be used for many different purposes. For instance filters + * The filters can also be used for many different purposes. For instance filters * could be used for logging, encryption, or compression. - * + * * The exact mechanisms used to generate/distributed the shared secret is deployment specific. - * + * * For Yarn deployments, the secret is automatically generated using the Akka remote * Crypt.generateSecureCookie() API. The secret is placed in the Hadoop UGI which gets passed * around via the Hadoop RPC mechanism. Hadoop RPC can be configured to support different levels @@ -121,7 +121,7 @@ import org.apache.spark.deploy.SparkHadoopUtil * to reduce the possibility of web based attacks through YARN. Hadoop can be configured to use * filters to do authentication. That authentication then happens via the ResourceManager Proxy * and Spark will use that to do authorization against the view acls. - * + * * For other Spark deployments, the shared secret must be specified via the * spark.authenticate.secret config. * All the nodes (Master and Workers) and the applications need to have the same shared secret. @@ -152,7 +152,7 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { " are ui acls enabled: " + uiAclsOn + " users with view permissions: " + viewAcls.toString()) // Set our own authenticator to properly negotiate user/password for HTTP connections. - // This is needed by the HTTP client fetching from the HttpServer. Put here so its + // This is needed by the HTTP client fetching from the HttpServer. Put here so its // only set once. if (authOn) { Authenticator.setDefault( @@ -214,12 +214,12 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { def uiAclsEnabled(): Boolean = uiAclsOn /** - * Checks the given user against the view acl list to see if they have + * Checks the given user against the view acl list to see if they have * authorization to view the UI. If the UI acls must are disabled * via spark.ui.acls.enable, all users have view access. - * + * * @param user to see if is authorized - * @return true is the user has permission, otherwise false + * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true diff --git a/core/src/main/scala/org/apache/spark/SparkException.scala b/core/src/main/scala/org/apache/spark/SparkException.scala index d34e47e8cac22..4351ed74b67fc 100644 --- a/core/src/main/scala/org/apache/spark/SparkException.scala +++ b/core/src/main/scala/org/apache/spark/SparkException.scala @@ -20,5 +20,5 @@ package org.apache.spark class SparkException(message: String, cause: Throwable) extends Exception(message, cause) { - def this(message: String) = this(message, null) + def this(message: String) = this(message, null) } diff --git a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index b92ea01a877f7..f6703986bdf11 100644 --- a/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/core/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -42,7 +42,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) private val now = new Date() private val conf = new SerializableWritable(jobConf) - + private var jobID = 0 private var splitID = 0 private var attemptID = 0 @@ -58,8 +58,8 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def preSetup() { setIDs(0, 0, 0) HadoopRDD.addLocalConfiguration("", 0, 0, 0, conf.value) - - val jCtxt = getJobContext() + + val jCtxt = getJobContext() getOutputCommitter().setupJob(jCtxt) } @@ -74,7 +74,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) val numfmt = NumberFormat.getInstance() numfmt.setMinimumIntegerDigits(5) numfmt.setGroupingUsed(false) - + val outputName = "part-" + numfmt.format(splitID) val path = FileOutputFormat.getOutputPath(conf.value) val fs: FileSystem = { @@ -85,7 +85,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } } - getOutputCommitter().setupTask(getTaskContext()) + getOutputCommitter().setupTask(getTaskContext()) writer = getOutputFormat().getRecordWriter(fs, conf.value, outputName, Reporter.NULL) } @@ -103,18 +103,18 @@ class SparkHadoopWriter(@transient jobConf: JobConf) def commit() { val taCtxt = getTaskContext() - val cmtr = getOutputCommitter() + val cmtr = getOutputCommitter() if (cmtr.needsTaskCommit(taCtxt)) { try { cmtr.commitTask(taCtxt) logInfo (taID + ": Committed") } catch { - case e: IOException => { + 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) } @@ -144,7 +144,7 @@ class SparkHadoopWriter(@transient jobConf: JobConf) } private def getJobContext(): JobContext = { - if (jobContext == null) { + if (jobContext == null) { jobContext = newJobContext(conf.value, jID.value) } jobContext @@ -175,7 +175,7 @@ object SparkHadoopWriter { val jobtrackerID = formatter.format(time) new JobID(jobtrackerID, id) } - + def createPathFromString(path: String, conf: JobConf): Path = { if (path == null) { throw new IllegalArgumentException("Output path is null") diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala index a2a871cbd3c31..5b14c4291d91a 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -44,12 +44,12 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg * configurable in the future. */ private var saslClient: SaslClient = Sasl.createSaslClient(Array[String](SparkSaslServer.DIGEST), - null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, + null, null, SparkSaslServer.SASL_DEFAULT_REALM, SparkSaslServer.SASL_PROPS, new SparkSaslClientCallbackHandler(securityMgr)) /** * Used to initiate SASL handshake with server. - * @return response to challenge if needed + * @return response to challenge if needed */ def firstToken(): Array[Byte] = { synchronized { @@ -86,7 +86,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg } /** - * Disposes of any system resources or security-sensitive information the + * Disposes of any system resources or security-sensitive information the * SaslClient might be using. */ def dispose() { @@ -110,7 +110,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg private class SparkSaslClientCallbackHandler(securityMgr: SecurityManager) extends CallbackHandler { - private val userName: String = + private val userName: String = SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) private val secretKey = securityMgr.getSecretKey() private val userPassword: Array[Char] = @@ -138,7 +138,7 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg rc.setText(rc.getDefaultText()) } case cb: RealmChoiceCallback => {} - case cb: Callback => throw + case cb: Callback => throw new UnsupportedCallbackException(cb, "handle: Unrecognized SASL client callback") } } diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala index 11fcb2ae3a5c5..6161a6fb7ae85 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -64,7 +64,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi } /** - * Disposes of any system resources or security-sensitive information the + * Disposes of any system resources or security-sensitive information the * SaslServer might be using. */ def dispose() { @@ -88,7 +88,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi private class SparkSaslDigestCallbackHandler(securityMgr: SecurityManager) extends CallbackHandler { - private val userName: String = + private val userName: String = SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) override def handle(callbacks: Array[Callback]) { @@ -123,7 +123,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi ac.setAuthorizedID(authzid) } } - case cb: Callback => throw + case cb: Callback => throw new UnsupportedCallbackException(cb, "handle: Unrecognized SASL DIGEST-MD5 Callback") } } diff --git a/core/src/main/scala/org/apache/spark/TestUtils.scala b/core/src/main/scala/org/apache/spark/TestUtils.scala index 4597595a838e3..f3f59e47c3e98 100644 --- a/core/src/main/scala/org/apache/spark/TestUtils.scala +++ b/core/src/main/scala/org/apache/spark/TestUtils.scala @@ -31,7 +31,7 @@ import com.google.common.io.Files * projects. * * TODO: See if we can move this to the test codebase by specifying - * test dependencies between projects. + * test dependencies between projects. */ private[spark] object TestUtils { diff --git a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala index 2b32546c6854d..2659274c5e98e 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala @@ -158,7 +158,7 @@ private[spark] class TorrentBroadcast[T](@transient var value_ : T, isLocal: Boo } def receiveBroadcast(): Boolean = { - // Receive meta-info about the size of broadcast data, + // Receive meta-info about the size of broadcast data, // the number of chunks it is divided into, etc. val metaId = BroadcastBlockId(id, "meta") var attemptId = 10 diff --git a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala index c07838f798799..5da9615c9e9af 100644 --- a/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/ClientArguments.scala @@ -43,7 +43,7 @@ private[spark] class ClientArguments(args: Array[String]) { // kill parameters var driverId: String = "" - + parse(args.toList) def parse(args: List[String]): Unit = args match { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala index d35d5be73ff97..3836bf219ed3e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/WorkerArguments.scala @@ -32,8 +32,8 @@ private[spark] class WorkerArguments(args: Array[String]) { var memory = inferDefaultMemory() var masters: Array[String] = null var workDir: String = null - - // Check for settings in environment variables + + // Check for settings in environment variables if (System.getenv("SPARK_WORKER_PORT") != null) { port = System.getenv("SPARK_WORKER_PORT").toInt } @@ -49,7 +49,7 @@ private[spark] class WorkerArguments(args: Array[String]) { if (System.getenv("SPARK_WORKER_DIR") != null) { workDir = System.getenv("SPARK_WORKER_DIR") } - + parse(args.toList) def parse(args: List[String]): Unit = args match { @@ -78,7 +78,7 @@ private[spark] class WorkerArguments(args: Array[String]) { case ("--work-dir" | "-d") :: value :: tail => workDir = value parse(tail) - + case "--webui-port" :: IntParam(value) :: tail => webUiPort = value parse(tail) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala index 85200ab0e102d..49c1009cac2bf 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala @@ -137,7 +137,7 @@ private[spark] class IndexPage(parent: WorkerWebUI) { .format(executor.appId, executor.execId)}>stdout stderr - + } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 16887d8892b31..6327ac01663f6 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -53,7 +53,7 @@ private[spark] class CoarseGrainedExecutorBackend( case RegisteredExecutor(sparkProperties) => logInfo("Successfully registered with driver") // Make this host instead of hostPort ? - executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, + executor = new Executor(executorId, Utils.parseHostPort(hostPort)._1, sparkProperties, false) case RegisterExecutorFailed(message) => diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala index ceff3a067d72a..38be2c58b333f 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorExitCode.scala @@ -34,7 +34,7 @@ object ExecutorExitCode { logging the exception. */ val UNCAUGHT_EXCEPTION_TWICE = 51 - /** The default uncaught exception handler was reached, and the uncaught exception was an + /** The default uncaught exception handler was reached, and the uncaught exception was an OutOfMemoryError. */ val OOM = 52 @@ -43,10 +43,10 @@ object ExecutorExitCode { /** TachyonStore failed to initialize after many attempts. */ val TACHYON_STORE_FAILED_TO_INITIALIZE = 54 - + /** TachyonStore failed to create a local temporary directory after many attempts. */ val TACHYON_STORE_FAILED_TO_CREATE_DIR = 55 - + def explainExitCode(exitCode: Int): String = { exitCode match { case UNCAUGHT_EXCEPTION => "Uncaught exception" @@ -57,7 +57,7 @@ object ExecutorExitCode { case TACHYON_STORE_FAILED_TO_INITIALIZE => "TachyonStore failed to initialize." case TACHYON_STORE_FAILED_TO_CREATE_DIR => "TachyonStore failed to create a local temporary directory." - case _ => + case _ => "Unknown executor exit code (" + exitCode + ")" + ( if (exitCode > 128) { " (died from signal " + (exitCode - 128) + "?)" diff --git a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala index 208e77073fd03..218ed7b5d2d39 100644 --- a/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala +++ b/core/src/main/scala/org/apache/spark/executor/ExecutorURLClassLoader.scala @@ -38,7 +38,7 @@ private[spark] class ChildExecutorURLClassLoader(urls: Array[URL], parent: Class override def addURL(url: URL) { super.addURL(url) } - override def findClass(name: String): Class[_] = { + override def findClass(name: String): Class[_] = { super.findClass(name) } } diff --git a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala index 42c1200926fea..542dce65366b2 100644 --- a/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala +++ b/core/src/main/scala/org/apache/spark/metrics/sink/CsvSink.scala @@ -45,7 +45,7 @@ private[spark] class CsvSink(val property: Properties, val registry: MetricRegis case Some(s) => TimeUnit.valueOf(s.toUpperCase()) case None => TimeUnit.valueOf(CSV_DEFAULT_UNIT) } - + MetricsSystem.checkMinimalPollingPeriod(pollUnit, pollPeriod) val pollDir = Option(property.getProperty(CSV_KEY_DIR)) match { diff --git a/core/src/main/scala/org/apache/spark/network/Connection.scala b/core/src/main/scala/org/apache/spark/network/Connection.scala index 2f7576c53b482..3ffaaab23d0f5 100644 --- a/core/src/main/scala/org/apache/spark/network/Connection.scala +++ b/core/src/main/scala/org/apache/spark/network/Connection.scala @@ -248,14 +248,14 @@ class SendingConnection(val address: InetSocketAddress, selector_ : Selector, } } - // outbox is used as a lock - ensure that it is always used as a leaf (since methods which + // outbox is used as a lock - ensure that it is always used as a leaf (since methods which // lock it are invoked in context of other locks) private val outbox = new Outbox() /* - This is orthogonal to whether we have pending bytes to write or not - and satisfies a slightly - different purpose. This flag is to see if we need to force reregister for write even when we + This is orthogonal to whether we have pending bytes to write or not - and satisfies a slightly + different purpose. This flag is to see if we need to force reregister for write even when we do not have any pending bytes to write to socket. - This can happen due to a race between adding pending buffers, and checking for existing of + This can happen due to a race between adding pending buffers, and checking for existing of data as detailed in https://github.com/mesos/spark/pull/791 */ private var needForceReregister = false diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala b/core/src/main/scala/org/apache/spark/network/ConnectionId.scala index ffaab677d411a..d579c165a1917 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionId.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionId.scala @@ -18,7 +18,7 @@ package org.apache.spark.network private[spark] case class ConnectionId(connectionManagerId: ConnectionManagerId, uniqId: Int) { - override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId + override def toString = connectionManagerId.host + "_" + connectionManagerId.port + "_" + uniqId } private[spark] object ConnectionId { @@ -26,9 +26,9 @@ private[spark] object ConnectionId { def createConnectionIdFromString(connectionIdString: String): ConnectionId = { val res = connectionIdString.split("_").map(_.trim()) if (res.size != 3) { - throw new Exception("Error converting ConnectionId string: " + connectionIdString + + throw new Exception("Error converting ConnectionId string: " + connectionIdString + " to a ConnectionId Object") } new ConnectionId(new ConnectionManagerId(res(0), res(1).toInt), res(2).toInt) - } + } } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index bdf586351ac14..cfee41c61362e 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -79,7 +79,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, private val serverChannel = ServerSocketChannel.open() // used to track the SendingConnections waiting to do SASL negotiation - private val connectionsAwaitingSasl = new HashMap[ConnectionId, SendingConnection] + private val connectionsAwaitingSasl = new HashMap[ConnectionId, SendingConnection] with SynchronizedMap[ConnectionId, SendingConnection] private val connectionsByKey = new HashMap[SelectionKey, Connection] with SynchronizedMap[SelectionKey, Connection] @@ -141,7 +141,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } finally { writeRunnableStarted.synchronized { writeRunnableStarted -= key - val needReregister = register || conn.resetForceReregister() + val needReregister = register || conn.resetForceReregister() if (needReregister && conn.changeInterestForWrite()) { conn.registerInterest() } @@ -509,7 +509,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, private def handleClientAuthentication( waitingConn: SendingConnection, - securityMsg: SecurityMessage, + securityMsg: SecurityMessage, connectionId : ConnectionId) { if (waitingConn.isSaslComplete()) { logDebug("Client sasl completed for id: " + waitingConn.connectionId) @@ -530,7 +530,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } return } - var securityMsgResp = SecurityMessage.fromResponse(replyToken, + var securityMsgResp = SecurityMessage.fromResponse(replyToken, securityMsg.getConnectionId.toString()) var message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security message") @@ -546,7 +546,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } private def handleServerAuthentication( - connection: Connection, + connection: Connection, securityMsg: SecurityMessage, connectionId: ConnectionId) { if (!connection.isSaslComplete()) { @@ -561,7 +561,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } replyToken = connection.sparkSaslServer.response(securityMsg.getToken) if (connection.isSaslComplete()) { - logDebug("Server sasl completed: " + connection.connectionId) + logDebug("Server sasl completed: " + connection.connectionId) } else { logDebug("Server sasl not completed: " + connection.connectionId) } @@ -571,7 +571,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, var message = securityMsgResp.toBufferMessage if (message == null) throw new Exception("Error creating security Message") sendSecurityMessage(connection.getRemoteConnectionManagerId(), message) - } + } } catch { case e: Exception => { logError("Error in server auth negotiation: " + e) @@ -581,7 +581,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } } else { - logDebug("connection already established for this connection id: " + connection.connectionId) + logDebug("connection already established for this connection id: " + connection.connectionId) } } @@ -609,8 +609,8 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, return true } else { if (!conn.isSaslComplete()) { - // We could handle this better and tell the client we need to do authentication - // negotiation, but for now just ignore them. + // We could handle this better and tell the client we need to do authentication + // negotiation, but for now just ignore them. logError("message sent that is not security negotiation message on connection " + "not authenticated yet, ignoring it!!") return true @@ -709,11 +709,11 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } } else { - logDebug("Sasl already established ") + logDebug("Sasl already established ") } } - // allow us to add messages to the inbox for doing sasl negotiating + // allow us to add messages to the inbox for doing sasl negotiating private def sendSecurityMessage(connManagerId: ConnectionManagerId, message: Message) { def startNewConnection(): SendingConnection = { val inetSocketAddress = new InetSocketAddress(connManagerId.host, connManagerId.port) @@ -772,7 +772,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, if (((clock.getTime() - startTime) >= (authTimeout * 1000)) && (!connection.isSaslComplete())) { // took to long to authenticate the connection, something probably went wrong - throw new Exception("Took to long for authentication to " + connectionManagerId + + throw new Exception("Took to long for authentication to " + connectionManagerId + ", waited " + authTimeout + "seconds, failing.") } } @@ -794,7 +794,7 @@ private[spark] class ConnectionManager(port: Int, conf: SparkConf, } } case None => { - logError("no messageStatus for failed message id: " + message.id) + logError("no messageStatus for failed message id: " + message.id) } } } diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala index 9d9b9dbdd5331..4894ecd41f6eb 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManagerTest.scala @@ -37,11 +37,11 @@ private[spark] object ConnectionManagerTest extends Logging{ "[size of msg in MB (integer)] [count] [await time in seconds)] ") System.exit(1) } - + if (args(0).startsWith("local")) { println("This runs only on a mesos cluster") } - + val sc = new SparkContext(args(0), "ConnectionManagerTest") val slavesFile = Source.fromFile(args(1)) val slaves = slavesFile.mkString.split("\n") @@ -50,7 +50,7 @@ private[spark] object ConnectionManagerTest extends Logging{ /* println("Slaves") */ /* slaves.foreach(println) */ val tasknum = if (args.length > 2) args(2).toInt else slaves.length - val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 + val size = ( if (args.length > 3) (args(3).toInt) else 10 ) * 1024 * 1024 val count = if (args.length > 4) args(4).toInt else 3 val awaitTime = (if (args.length > 5) args(5).toInt else 600 ).second println("Running " + count + " rounds of test: " + "parallel tasks = " + tasknum + ", " + @@ -64,16 +64,16 @@ private[spark] object ConnectionManagerTest extends Logging{ (0 until count).foreach(i => { val resultStrs = sc.parallelize(0 until tasknum, tasknum).map(i => { val connManager = SparkEnv.get.connectionManager - val thisConnManagerId = connManager.id - connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { + val thisConnManagerId = connManager.id + connManager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { logInfo("Received [" + msg + "] from [" + id + "]") None }) val buffer = ByteBuffer.allocate(size).put(Array.tabulate[Byte](size)(x => x.toByte)) buffer.flip - - val startTime = System.currentTimeMillis + + val startTime = System.currentTimeMillis val futures = slaveConnManagerIds.filter(_ != thisConnManagerId).map{ slaveConnManagerId => { val bufferMessage = Message.createBufferMessage(buffer.duplicate) @@ -84,7 +84,7 @@ private[spark] object ConnectionManagerTest extends Logging{ val results = futures.map(f => Await.result(f, awaitTime)) val finishTime = System.currentTimeMillis Thread.sleep(5000) - + val mb = size * results.size / 1024.0 / 1024.0 val ms = finishTime - startTime val resultStr = thisConnManagerId + " Sent " + mb + " MB in " + ms + " ms at " + (mb / ms * @@ -92,11 +92,11 @@ private[spark] object ConnectionManagerTest extends Logging{ logInfo(resultStr) resultStr }).collect() - - println("---------------------") - println("Run " + i) + + println("---------------------") + println("Run " + i) resultStrs.foreach(println) - println("---------------------") + println("---------------------") }) } } diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 2b41c403b2e0a..9dc51e0d401f8 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -18,7 +18,7 @@ package org.apache.spark.network import java.nio.ByteBuffer -import org.apache.spark.{SecurityManager, SparkConf} +import org.apache.spark.{SecurityManager, SparkConf} private[spark] object ReceiverTest { def main(args: Array[String]) { diff --git a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala index 0d9f743b3624b..a1dfc4094cca7 100644 --- a/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala +++ b/core/src/main/scala/org/apache/spark/network/SecurityMessage.scala @@ -26,33 +26,33 @@ import org.apache.spark._ import org.apache.spark.network._ /** - * SecurityMessage is class that contains the connectionId and sasl token + * SecurityMessage is class that contains the connectionId and sasl token * used in SASL negotiation. SecurityMessage has routines for converting * it to and from a BufferMessage so that it can be sent by the ConnectionManager * and easily consumed by users when received. * The api was modeled after BlockMessage. * - * The connectionId is the connectionId of the client side. Since + * The connectionId is the connectionId of the client side. Since * message passing is asynchronous and its possible for the server side (receiving) - * to get multiple different types of messages on the same connection the connectionId - * is used to know which connnection the security message is intended for. - * + * to get multiple different types of messages on the same connection the connectionId + * is used to know which connnection the security message is intended for. + * * For instance, lets say we are node_0. We need to send data to node_1. The node_0 side * is acting as a client and connecting to node_1. SASL negotiation has to occur - * between node_0 and node_1 before node_1 trusts node_0 so node_0 sends a security message. - * node_1 receives the message from node_0 but before it can process it and send a response, - * some thread on node_1 decides it needs to send data to node_0 so it connects to node_0 - * and sends a security message of its own to authenticate as a client. Now node_0 gets - * the message and it needs to decide if this message is in response to it being a client - * (from the first send) or if its just node_1 trying to connect to it to send data. This + * between node_0 and node_1 before node_1 trusts node_0 so node_0 sends a security message. + * node_1 receives the message from node_0 but before it can process it and send a response, + * some thread on node_1 decides it needs to send data to node_0 so it connects to node_0 + * and sends a security message of its own to authenticate as a client. Now node_0 gets + * the message and it needs to decide if this message is in response to it being a client + * (from the first send) or if its just node_1 trying to connect to it to send data. This * is where the connectionId field is used. node_0 can lookup the connectionId to see if * it is in response to it being a client or if its in response to someone sending other data. - * + * * The format of a SecurityMessage as its sent is: * - Length of the ConnectionId - * - ConnectionId + * - ConnectionId * - Length of the token - * - Token + * - Token */ private[spark] class SecurityMessage() extends Logging { @@ -61,13 +61,13 @@ private[spark] class SecurityMessage() extends Logging { def set(byteArr: Array[Byte], newconnectionId: String) { if (byteArr == null) { - token = new Array[Byte](0) + token = new Array[Byte](0) } else { token = byteArr } connectionId = newconnectionId } - + /** * Read the given buffer and set the members of this class. */ @@ -91,17 +91,17 @@ private[spark] class SecurityMessage() extends Logging { buffer.clear() set(buffer) } - + def getConnectionId: String = { return connectionId } - + def getToken: Array[Byte] = { return token } - + /** - * Create a BufferMessage that can be sent by the ConnectionManager containing + * Create a BufferMessage that can be sent by the ConnectionManager containing * the security information from this class. * @return BufferMessage */ @@ -110,12 +110,12 @@ private[spark] class SecurityMessage() extends Logging { val buffers = new ArrayBuffer[ByteBuffer]() // 4 bytes for the length of the connectionId - // connectionId is of type char so multiple the length by 2 to get number of bytes + // connectionId is of type char so multiple the length by 2 to get number of bytes // 4 bytes for the length of token // token is a byte buffer so just take the length var buffer = ByteBuffer.allocate(4 + connectionId.length() * 2 + 4 + token.length) buffer.putInt(connectionId.length()) - connectionId.foreach((x: Char) => buffer.putChar(x)) + connectionId.foreach((x: Char) => buffer.putChar(x)) buffer.putInt(token.length) if (token.length > 0) { @@ -123,7 +123,7 @@ private[spark] class SecurityMessage() extends Logging { } buffer.flip() buffers += buffer - + var message = Message.createBufferMessage(buffers) logDebug("message total size is : " + message.size) message.isSecurityNeg = true @@ -136,7 +136,7 @@ private[spark] class SecurityMessage() extends Logging { } private[spark] object SecurityMessage { - + /** * Convert the given BufferMessage to a SecurityMessage by parsing the contents * of the BufferMessage and populating the SecurityMessage fields. diff --git a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala index 4164e81d3a8ae..136c1912045aa 100644 --- a/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala +++ b/core/src/main/scala/org/apache/spark/network/netty/FileHeader.scala @@ -36,8 +36,8 @@ private[spark] class FileHeader ( if (FileHeader.HEADER_SIZE - buf.readableBytes > 0 ) { buf.writeZero(FileHeader.HEADER_SIZE - buf.readableBytes) } else { - throw new Exception("too long header " + buf.readableBytes) - logInfo("too long header") + throw new Exception("too long header " + buf.readableBytes) + logInfo("too long header") } buf } diff --git a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala index eade07fbcbe37..cadd0c7ed19ba 100644 --- a/core/src/main/scala/org/apache/spark/partial/PartialResult.scala +++ b/core/src/main/scala/org/apache/spark/partial/PartialResult.scala @@ -44,7 +44,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { } } - /** + /** * Set a handler to be called when this PartialResult completes. Only one completion handler * is supported per PartialResult. */ @@ -60,7 +60,7 @@ class PartialResult[R](initialVal: R, isFinal: Boolean) { return this } - /** + /** * Set a handler to be called if this PartialResult's job fails. Only one failure handler * is supported per PartialResult. */ diff --git a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala index 2306c9736b334..9ca971c8a4c27 100644 --- a/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/DoubleRDDFunctions.scala @@ -52,7 +52,7 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { /** Compute the standard deviation of this RDD's elements. */ def stdev(): Double = stats().stdev - /** + /** * Compute the sample standard deviation of this RDD's elements (which corrects for bias in * estimating the standard deviation by dividing by N-1 instead of N). */ @@ -123,13 +123,13 @@ class DoubleRDDFunctions(self: RDD[Double]) extends Logging with Serializable { * e.g. for the array * [1, 10, 20, 50] the buckets are [1, 10) [10, 20) [20, 50] * e.g 1<=x<10 , 10<=x<20, 20<=x<50 - * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 - * + * And on the input of 1 and 50 we would have a histogram of 1, 0, 0 + * * Note: if your histogram is evenly spaced (e.g. [0, 10, 20, 30]) this can be switched * from an O(log n) inseration to O(1) per element. (where n = # buckets) if you set evenBuckets * to true. * buckets must be sorted and not contain any duplicates. - * buckets array must be at least two elements + * buckets array must be at least two elements * All NaN entries are treated the same. If you have a NaN bucket it must be * the maximum value of the last position and all NaN entries will be counted * in that bucket. diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala index a84357b38414e..0c2cd7a24783b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionerAwareUnionRDD.scala @@ -33,7 +33,7 @@ class PartitionerAwareUnionRDDPartition( val idx: Int ) extends Partition { var parents = rdds.map(_.partitions(idx)).toArray - + override val index = idx override def hashCode(): Int = idx diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 04c53d468465a..293cfb65643a6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -54,7 +54,7 @@ private[scheduler] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] -case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent +case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] case class CompletionEvent( task: Task[_], diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index 76f3e327d60b8..545fa453b7ccf 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -1,107 +1,107 @@ -/* - * 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.scheduler - -import java.util.concurrent.LinkedBlockingQueue - -import org.apache.spark.Logging - -/** - * Asynchronously passes SparkListenerEvents to registered SparkListeners. - * - * Until start() is called, all posted events are only buffered. Only after this listener bus - * has started will events be actually propagated to all attached listeners. This listener bus - * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). - */ -private[spark] class LiveListenerBus extends SparkListenerBus with Logging { - - /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than - * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ - private val EVENT_QUEUE_CAPACITY = 10000 - private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) - private var queueFullErrorMessageLogged = false - private var started = false - private val listenerThread = new Thread("SparkListenerBus") { - setDaemon(true) - override def run() { - while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return - } - postToAll(event) - } - } - } - - // Exposed for testing - @volatile private[spark] var stopCalled = false - - /** - * Start sending events to attached listeners. - * - * This first sends out all buffered events posted before this listener bus has started, then - * listens for any additional events asynchronously while the listener bus is still running. - * This should only be called once. - */ - def start() { - if (started) { - throw new IllegalStateException("Listener bus already started!") - } - listenerThread.start() - started = true - } - - def post(event: SparkListenerEvent) { - val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { - logError("Dropping SparkListenerEvent because no remaining room in event queue. " + - "This likely means one of the SparkListeners is too slow and cannot keep up with the " + - "rate at which tasks are being started by the scheduler.") - queueFullErrorMessageLogged = true - } - } - - /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time - * elapsed before the queue emptied. - */ - def waitUntilEmpty(timeoutMillis: Int): Boolean = { - val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { - if (System.currentTimeMillis > finishTime) { - return false - } - /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify - * add overhead in the general case. */ - Thread.sleep(10) - } - true - } - - def stop() { - stopCalled = true - if (!started) { - throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") - } - post(SparkListenerShutdown) - listenerThread.join() - } -} +/* + * 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.scheduler + +import java.util.concurrent.LinkedBlockingQueue + +import org.apache.spark.Logging + +/** + * Asynchronously passes SparkListenerEvents to registered SparkListeners. + * + * Until start() is called, all posted events are only buffered. Only after this listener bus + * has started will events be actually propagated to all attached listeners. This listener bus + * is stopped when it receives a SparkListenerShutdown event, which is posted using stop(). + */ +private[spark] class LiveListenerBus extends SparkListenerBus with Logging { + + /* Cap the capacity of the SparkListenerEvent queue so we get an explicit error (rather than + * an OOM exception) if it's perpetually being added to more quickly than it's being drained. */ + private val EVENT_QUEUE_CAPACITY = 10000 + private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) + private var queueFullErrorMessageLogged = false + private var started = false + private val listenerThread = new Thread("SparkListenerBus") { + setDaemon(true) + override def run() { + while (true) { + val event = eventQueue.take + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + postToAll(event) + } + } + } + + // Exposed for testing + @volatile private[spark] var stopCalled = false + + /** + * Start sending events to attached listeners. + * + * This first sends out all buffered events posted before this listener bus has started, then + * listens for any additional events asynchronously while the listener bus is still running. + * This should only be called once. + */ + def start() { + if (started) { + throw new IllegalStateException("Listener bus already started!") + } + listenerThread.start() + started = true + } + + def post(event: SparkListenerEvent) { + val eventAdded = eventQueue.offer(event) + if (!eventAdded && !queueFullErrorMessageLogged) { + logError("Dropping SparkListenerEvent because no remaining room in event queue. " + + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + + "rate at which tasks are being started by the scheduler.") + queueFullErrorMessageLogged = true + } + } + + /** + * Waits until there are no more events in the queue, or until the specified time has elapsed. + * Used for testing only. Returns true if the queue has emptied and false is the specified time + * elapsed before the queue emptied. + */ + def waitUntilEmpty(timeoutMillis: Int): Boolean = { + val finishTime = System.currentTimeMillis + timeoutMillis + while (!eventQueue.isEmpty) { + if (System.currentTimeMillis > finishTime) { + return false + } + /* Sleep rather than using wait/notify, because this is used only for testing and wait/notify + * add overhead in the general case. */ + Thread.sleep(10) + } + true + } + + def stop() { + stopCalled = true + if (!started) { + throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") + } + post(SparkListenerShutdown) + listenerThread.join() + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala index 2fbbda5b76c74..ace9cd51c96b7 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockFetcherIterator.scala @@ -240,7 +240,7 @@ object BlockFetcherIterator { override def numRemoteBlocks: Int = numRemote override def fetchWaitTime: Long = _fetchWaitTime override def remoteBytesRead: Long = _remoteBytesRead - + // Implementing the Iterator methods with an iterator that reads fetched blocks off the queue // as they arrive. diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index a2a729130091f..df9bb4044e37a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -388,7 +388,7 @@ private[spark] class BlockManager( logDebug("Block " + blockId + " not found in memory") } } - + // Look for the block in Tachyon if (level.useOffHeap) { logDebug("Getting block " + blockId + " from tachyon") @@ -1031,7 +1031,7 @@ private[spark] class BlockManager( memoryStore.clear() diskStore.clear() if (tachyonInitialized) { - tachyonStore.clear() + tachyonStore.clear() } metadataCleaner.cancel() broadcastCleaner.cancel() diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala index 7168ae18c2615..337b45b727dec 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala @@ -37,7 +37,7 @@ private[spark] class BlockMessage() { private var id: BlockId = null private var data: ByteBuffer = null private var level: StorageLevel = null - + def set(getBlock: GetBlock) { typ = BlockMessage.TYPE_GET_BLOCK id = getBlock.id @@ -75,13 +75,13 @@ private[spark] class BlockMessage() { idBuilder += buffer.getChar() } id = BlockId(idBuilder.toString) - + if (typ == BlockMessage.TYPE_PUT_BLOCK) { val booleanInt = buffer.getInt() val replication = buffer.getInt() level = StorageLevel(booleanInt, replication) - + val dataLength = buffer.getInt() data = ByteBuffer.allocate(dataLength) if (dataLength != buffer.remaining) { @@ -108,12 +108,12 @@ private[spark] class BlockMessage() { buffer.clear() set(buffer) } - + def getType: Int = typ def getId: BlockId = id def getData: ByteBuffer = data def getLevel: StorageLevel = level - + def toBufferMessage: BufferMessage = { val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() @@ -127,7 +127,7 @@ private[spark] class BlockMessage() { buffer = ByteBuffer.allocate(8).putInt(level.toInt).putInt(level.replication) buffer.flip() buffers += buffer - + buffer = ByteBuffer.allocate(4).putInt(data.remaining) buffer.flip() buffers += buffer @@ -140,7 +140,7 @@ private[spark] class BlockMessage() { buffers += data } - + /* println() println("BlockMessage: ") @@ -158,7 +158,7 @@ private[spark] class BlockMessage() { } override def toString: String = { - "BlockMessage [type = " + typ + ", id = " + id + ", level = " + level + + "BlockMessage [type = " + typ + ", id = " + id + ", level = " + level + ", data = " + (if (data != null) data.remaining.toString else "null") + "]" } } @@ -168,7 +168,7 @@ private[spark] object BlockMessage { val TYPE_GET_BLOCK: Int = 1 val TYPE_GOT_BLOCK: Int = 2 val TYPE_PUT_BLOCK: Int = 3 - + def fromBufferMessage(bufferMessage: BufferMessage): BlockMessage = { val newBlockMessage = new BlockMessage() newBlockMessage.set(bufferMessage) @@ -192,7 +192,7 @@ private[spark] object BlockMessage { newBlockMessage.set(gotBlock) newBlockMessage } - + def fromPutBlock(putBlock: PutBlock): BlockMessage = { val newBlockMessage = new BlockMessage() newBlockMessage.set(putBlock) @@ -206,7 +206,7 @@ private[spark] object BlockMessage { val bMsg = B.toBufferMessage val C = new BlockMessage() C.set(bMsg) - + println(B.getId + " " + B.getLevel) println(C.getId + " " + C.getLevel) } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala index dc62b1efaa7d4..973d85c0a9b3a 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessageArray.scala @@ -27,16 +27,16 @@ import org.apache.spark.network._ private[spark] class BlockMessageArray(var blockMessages: Seq[BlockMessage]) extends Seq[BlockMessage] with Logging { - + def this(bm: BlockMessage) = this(Array(bm)) def this() = this(null.asInstanceOf[Seq[BlockMessage]]) - def apply(i: Int) = blockMessages(i) + def apply(i: Int) = blockMessages(i) def iterator = blockMessages.iterator - def length = blockMessages.length + def length = blockMessages.length def set(bufferMessage: BufferMessage) { val startTime = System.currentTimeMillis @@ -62,15 +62,15 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) logDebug("Trying to convert buffer " + newBuffer + " to block message") val newBlockMessage = BlockMessage.fromByteBuffer(newBuffer) logDebug("Created " + newBlockMessage) - newBlockMessages += newBlockMessage + newBlockMessages += newBlockMessage buffer.position(buffer.position() + size) } val finishTime = System.currentTimeMillis logDebug("Converted block message array from buffer message in " + (finishTime - startTime) / 1000.0 + " s") - this.blockMessages = newBlockMessages + this.blockMessages = newBlockMessages } - + def toBufferMessage: BufferMessage = { val buffers = new ArrayBuffer[ByteBuffer]() @@ -83,7 +83,7 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) buffers ++= bufferMessage.buffers logDebug("Added " + bufferMessage) }) - + logDebug("Buffer list:") buffers.foreach((x: ByteBuffer) => logDebug("" + x)) /* @@ -103,13 +103,13 @@ class BlockMessageArray(var blockMessages: Seq[BlockMessage]) } private[spark] object BlockMessageArray { - + def fromBufferMessage(bufferMessage: BufferMessage): BlockMessageArray = { val newBlockMessageArray = new BlockMessageArray() newBlockMessageArray.set(bufferMessage) newBlockMessageArray } - + def main(args: Array[String]) { val blockMessages = (0 until 10).map { i => @@ -124,10 +124,10 @@ private[spark] object BlockMessageArray { } val blockMessageArray = new BlockMessageArray(blockMessages) println("Block message array created") - + val bufferMessage = blockMessageArray.toBufferMessage println("Converted to buffer message") - + val totalSize = bufferMessage.size val newBuffer = ByteBuffer.allocate(totalSize) newBuffer.clear() @@ -137,7 +137,7 @@ private[spark] object BlockMessageArray { buffer.rewind() }) newBuffer.flip - val newBufferMessage = Message.createBufferMessage(newBuffer) + val newBufferMessage = Message.createBufferMessage(newBuffer) println("Copied to new buffer message, size = " + newBufferMessage.size) val newBlockMessageArray = BlockMessageArray.fromBufferMessage(newBufferMessage) @@ -147,7 +147,7 @@ private[spark] object BlockMessageArray { case BlockMessage.TYPE_PUT_BLOCK => { val pB = PutBlock(blockMessage.getId, blockMessage.getData, blockMessage.getLevel) println(pB) - } + } case BlockMessage.TYPE_GET_BLOCK => { val gB = new GetBlock(blockMessage.getId) println(gB) diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index e1a1f209c9282..9ce0398d010a8 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -136,7 +136,7 @@ private[spark] object JettyUtils extends Logging { private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) filters.foreach { - case filter : String => + case filter : String => if (!filter.isEmpty) { logInfo("Adding filter: " + filter) val holder : FilterHolder = new FilterHolder() @@ -151,7 +151,7 @@ private[spark] object JettyUtils extends Logging { if (parts.length == 2) holder.setInitParameter(parts(0), parts(1)) } } - val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, + val enumDispatcher = java.util.EnumSet.of(DispatcherType.ASYNC, DispatcherType.ERROR, DispatcherType.FORWARD, DispatcherType.INCLUDE, DispatcherType.REQUEST) handlers.foreach { case(handler) => handler.addFilter(holder, "/*", enumDispatcher) } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a487924effbff..a7cf04b3cbb86 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -48,7 +48,7 @@ private[spark] object UIUtils { case _ =>
  • Storage
  • } val environment = page match { - case Environment => + case Environment =>
  • Environment
  • case _ =>
  • Environment
  • } diff --git a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala index cdbbc65292188..2d05e09b10948 100644 --- a/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala +++ b/core/src/main/scala/org/apache/spark/util/ClosureCleaner.scala @@ -45,7 +45,7 @@ private[spark] object ClosureCleaner extends Logging { private def isClosure(cls: Class[_]): Boolean = { cls.getName.contains("$anonfun$") } - + // Get a list of the classes of the outer objects of a given closure object, obj; // the outer objects are defined as any closures that obj is nested within, plus // possibly the class that the outermost closure is in, if any. We stop searching @@ -63,7 +63,7 @@ private[spark] object ClosureCleaner extends Logging { } Nil } - + // Get a list of the outer objects for a given closure object. private def getOuterObjects(obj: AnyRef): List[AnyRef] = { for (f <- obj.getClass.getDeclaredFields if f.getName == "$outer") { @@ -76,7 +76,7 @@ private[spark] object ClosureCleaner extends Logging { } Nil } - + private def getInnerClasses(obj: AnyRef): List[Class[_]] = { val seen = Set[Class[_]](obj.getClass) var stack = List[Class[_]](obj.getClass) @@ -92,7 +92,7 @@ private[spark] object ClosureCleaner extends Logging { } return (seen - obj.getClass).toList } - + private def createNullValue(cls: Class[_]): AnyRef = { if (cls.isPrimitive) { new java.lang.Byte(0: Byte) // Should be convertible to any primitive type @@ -100,13 +100,13 @@ private[spark] object ClosureCleaner extends Logging { null } } - + def clean(func: AnyRef) { // TODO: cache outerClasses / innerClasses / accessedFields val outerClasses = getOuterClasses(func) val innerClasses = getInnerClasses(func) val outerObjects = getOuterObjects(func) - + val accessedFields = Map[Class[_], Set[String]]() for (cls <- outerClasses) accessedFields(cls) = Set[String]() @@ -143,7 +143,7 @@ private[spark] object ClosureCleaner extends Logging { field.set(outer, value) } } - + if (outer != null) { // logInfo("2: Setting $outer on " + func.getClass + " to " + outer); val field = func.getClass.getDeclaredField("$outer") @@ -151,7 +151,7 @@ private[spark] object ClosureCleaner extends Logging { field.set(func, outer) } } - + private def instantiateClass(cls: Class[_], outer: AnyRef, inInterpreter: Boolean): AnyRef = { // logInfo("Creating a " + cls + " with outer = " + outer) if (!inInterpreter) { @@ -192,7 +192,7 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor } } } - + override def visitMethodInsn(op: Int, owner: String, name: String, desc: String) { // Check for calls a getter method for a variable in an interpreter wrapper object. @@ -209,12 +209,12 @@ class FieldAccessFinder(output: Map[Class[_], Set[String]]) extends ClassVisitor private[spark] class InnerClosureFinder(output: Set[Class[_]]) extends ClassVisitor(ASM4) { var myName: String = null - + override def visit(version: Int, access: Int, name: String, sig: String, superName: String, interfaces: Array[String]) { myName = name } - + override def visitMethod(access: Int, name: String, desc: String, sig: String, exceptions: Array[String]): MethodVisitor = { new MethodVisitor(ASM4) { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index d990fd49ef834..f2396f7c80a35 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -611,7 +611,7 @@ private[spark] object JsonProtocol { val rddInfo = new RDDInfo(rddId, name, numPartitions, storageLevel) rddInfo.numCachedPartitions = numCachedPartitions rddInfo.memSize = memSize - rddInfo.tachyonSize = tachyonSize + rddInfo.tachyonSize = tachyonSize rddInfo.diskSize = diskSize rddInfo } diff --git a/core/src/main/scala/org/apache/spark/util/NextIterator.scala b/core/src/main/scala/org/apache/spark/util/NextIterator.scala index 8266e5e495efc..e5c732a5a559b 100644 --- a/core/src/main/scala/org/apache/spark/util/NextIterator.scala +++ b/core/src/main/scala/org/apache/spark/util/NextIterator.scala @@ -19,7 +19,7 @@ package org.apache.spark.util /** Provides a basic/boilerplate Iterator implementation. */ private[spark] abstract class NextIterator[U] extends Iterator[U] { - + private var gotNext = false private var nextValue: U = _ private var closed = false @@ -34,7 +34,7 @@ private[spark] abstract class NextIterator[U] extends Iterator[U] { * This convention is required because `null` may be a valid value, * and using `Option` seems like it might create unnecessary Some/None * instances, given some iterators might be called in a tight loop. - * + * * @return U, or set 'finished' when done */ protected def getNext(): U diff --git a/core/src/main/scala/org/apache/spark/util/StatCounter.scala b/core/src/main/scala/org/apache/spark/util/StatCounter.scala index 732748a7ff82b..d80eed455c427 100644 --- a/core/src/main/scala/org/apache/spark/util/StatCounter.scala +++ b/core/src/main/scala/org/apache/spark/util/StatCounter.scala @@ -62,10 +62,10 @@ class StatCounter(values: TraversableOnce[Double]) extends Serializable { if (n == 0) { mu = other.mu m2 = other.m2 - n = other.n + n = other.n maxValue = other.maxValue minValue = other.minValue - } else if (other.n != 0) { + } else if (other.n != 0) { val delta = other.mu - mu if (other.n * 10 < n) { mu = mu + (delta * other.n) / (n + other.n) diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index 3c8f94a416c65..1a647fa1c9d84 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -136,7 +136,7 @@ object Vector { def ones(length: Int) = Vector(length, _ => 1) /** - * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers + * Creates this [[org.apache.spark.util.Vector]] of given length containing random numbers * between 0.0 and 1.0. Optional scala.util.Random number generator can be provided. */ def random(length: Int, random: Random = new XORShiftRandom()) = diff --git a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala index 8a4cdea2fa7b1..7f220383f9f8b 100644 --- a/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala +++ b/core/src/main/scala/org/apache/spark/util/random/XORShiftRandom.scala @@ -25,28 +25,28 @@ import scala.util.hashing.MurmurHash3 import org.apache.spark.util.Utils.timeIt /** - * This class implements a XORShift random number generator algorithm + * This class implements a XORShift random number generator algorithm * Source: * Marsaglia, G. (2003). Xorshift RNGs. Journal of Statistical Software, Vol. 8, Issue 14. * @see Paper * This implementation is approximately 3.5 times faster than * {@link java.util.Random java.util.Random}, partly because of the algorithm, but also due - * to renouncing thread safety. JDK's implementation uses an AtomicLong seed, this class + * to renouncing thread safety. JDK's implementation uses an AtomicLong seed, this class * uses a regular Long. We can forgo thread safety since we use a new instance of the RNG * for each thread. */ private[spark] class XORShiftRandom(init: Long) extends JavaRandom(init) { - + def this() = this(System.nanoTime) private var seed = XORShiftRandom.hashSeed(init) // we need to just override next - this will be called by nextInt, nextDouble, // nextGaussian, nextLong, etc. - override protected def next(bits: Int): Int = { + override protected def next(bits: Int): Int = { var nextSeed = seed ^ (seed << 21) nextSeed ^= (nextSeed >>> 35) - nextSeed ^= (nextSeed << 4) + nextSeed ^= (nextSeed << 4) seed = nextSeed (nextSeed & ((1L << bits) -1)).asInstanceOf[Int] } @@ -89,7 +89,7 @@ private[spark] object XORShiftRandom { val million = 1e6.toInt val javaRand = new JavaRandom(seed) val xorRand = new XORShiftRandom(seed) - + // this is just to warm up the JIT - we're not timing anything timeIt(1e6.toInt) { javaRand.nextInt() @@ -97,9 +97,9 @@ private[spark] object XORShiftRandom { } val iters = timeIt(numIters)(_) - + /* Return results as a map instead of just printing to screen - in case the user wants to do something with them */ + in case the user wants to do something with them */ Map("javaTime" -> iters {javaRand.nextInt()}, "xorTime" -> iters {xorRand.nextInt()}) diff --git a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala index c5f24c66ce0c1..c645e4cbe8132 100644 --- a/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala +++ b/core/src/test/scala/org/apache/spark/AkkaUtilsSuite.scala @@ -37,7 +37,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -54,14 +54,14 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { assert(securityManagerBad.isAuthenticationEnabled() === true) - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = conf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) - intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) } actorSystem.shutdown() @@ -75,7 +75,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -91,7 +91,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { badconf.set("spark.authenticate.secret", "good") val securityManagerBad = new SecurityManager(badconf); - val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, + val (slaveSystem, _) = AkkaUtils.createActorSystem("spark-slave", hostname, 0, conf = badconf, securityManager = securityManagerBad) val slaveTracker = new MapOutputTrackerWorker(conf) val selection = slaveSystem.actorSelection( @@ -127,7 +127,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -180,7 +180,7 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val securityManager = new SecurityManager(conf); val hostname = "localhost" - val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, + val (actorSystem, boundPort) = AkkaUtils.createActorSystem("spark", hostname, 0, conf = conf, securityManager = securityManager) System.setProperty("spark.driver.port", boundPort.toString) // Will be cleared by LocalSparkContext System.setProperty("spark.hostPort", hostname + ":" + boundPort) @@ -204,8 +204,8 @@ class AkkaUtilsSuite extends FunSuite with LocalSparkContext { val selection = slaveSystem.actorSelection( s"akka.tcp://spark@localhost:$boundPort/user/MapOutputTracker") val timeout = AkkaUtils.lookupTimeout(conf) - intercept[akka.actor.ActorNotFound] { - slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) + intercept[akka.actor.ActorNotFound] { + slaveTracker.trackerActor = Await.result(selection.resolveOne(timeout), timeout) } actorSystem.shutdown() diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 9cbdfc54a3dc8..7f59bdcce4cc7 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -39,7 +39,7 @@ class DriverSuite extends FunSuite with Timeouts { failAfter(60 seconds) { Utils.executeAndGetOutput( Seq("./bin/spark-class", "org.apache.spark.DriverWithoutCleanup", master), - new File(sparkHome), + new File(sparkHome), Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) } } diff --git a/core/src/test/scala/org/apache/spark/FileServerSuite.scala b/core/src/test/scala/org/apache/spark/FileServerSuite.scala index aee9ab9091dac..d651fbbac4e97 100644 --- a/core/src/test/scala/org/apache/spark/FileServerSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileServerSuite.scala @@ -45,7 +45,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val pw = new PrintWriter(textFile) pw.println("100") pw.close() - + val jarFile = new File(tmpDir, "test.jar") val jarStream = new FileOutputStream(jarFile) val jar = new JarOutputStream(jarStream, new java.util.jar.Manifest()) @@ -53,7 +53,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext { val jarEntry = new JarEntry(textFile.getName) jar.putNextEntry(jarEntry) - + val in = new FileInputStream(textFile) val buffer = new Array[Byte](10240) var nRead = 0 diff --git a/core/src/test/scala/org/apache/spark/FileSuite.scala b/core/src/test/scala/org/apache/spark/FileSuite.scala index 01af94077144a..b9b668d3cc62a 100644 --- a/core/src/test/scala/org/apache/spark/FileSuite.scala +++ b/core/src/test/scala/org/apache/spark/FileSuite.scala @@ -106,7 +106,7 @@ class FileSuite extends FunSuite with LocalSparkContext { sc = new SparkContext("local", "test") val tempDir = Files.createTempDir() val outputDir = new File(tempDir, "output").getAbsolutePath - val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) + val nums = sc.makeRDD(1 to 3).map(x => (new IntWritable(x), "a" * x)) nums.saveAsSequenceFile(outputDir) // Try reading the output back as a SequenceFile val output = sc.sequenceFile[IntWritable, Text](outputDir) diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala index 0b5ed6d77034b..5e538d6fab2a1 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/WorkerWatcherSuite.scala @@ -45,4 +45,4 @@ class WorkerWatcherSuite extends FunSuite { actorRef.underlyingActor.receive(new DisassociatedEvent(null, otherAkkaAddress, false)) assert(!actorRef.underlyingActor.isShutDown) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index 09e35bfc8f85f..e89b296d41026 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -42,7 +42,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { override def beforeAll() { sc = new SparkContext("local", "test") - + // Set the block size of local file system to test whether files are split right or not. sc.hadoopConfiguration.setLong("fs.local.block.size", 32) } diff --git a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala index a4381a8b974df..4df36558b6d4b 100644 --- a/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/ParallelCollectionSplitSuite.scala @@ -34,14 +34,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === "2") assert(slices(2).mkString(",") === "3") } - + test("one slice") { val data = Array(1, 2, 3) val slices = ParallelCollectionRDD.slice(data, 1) assert(slices.size === 1) assert(slices(0).mkString(",") === "1,2,3") } - + test("equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9) val slices = ParallelCollectionRDD.slice(data, 3) @@ -50,7 +50,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === "4,5,6") assert(slices(2).mkString(",") === "7,8,9") } - + test("non-equal slices") { val data = Array(1, 2, 3, 4, 5, 6, 7, 8, 9, 10) val slices = ParallelCollectionRDD.slice(data, 3) @@ -77,14 +77,14 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices(1).mkString(",") === (33 to 66).mkString(",")) assert(slices(2).mkString(",") === (67 to 100).mkString(",")) } - + test("empty data") { val data = new Array[Int](0) val slices = ParallelCollectionRDD.slice(data, 5) assert(slices.size === 5) for (slice <- slices) assert(slice.size === 0) } - + test("zero slices") { val data = Array(1, 2, 3) intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, 0) } @@ -94,7 +94,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { val data = Array(1, 2, 3) intercept[IllegalArgumentException] { ParallelCollectionRDD.slice(data, -5) } } - + test("exclusive ranges sliced into ranges") { val data = 1 until 100 val slices = ParallelCollectionRDD.slice(data, 3) @@ -102,7 +102,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[Range])) } - + test("inclusive ranges sliced into ranges") { val data = 1 to 100 val slices = ParallelCollectionRDD.slice(data, 3) @@ -124,7 +124,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(range.step === 1, "slice " + i + " step") } } - + test("random array tests") { val gen = for { d <- arbitrary[List[Int]] @@ -141,7 +141,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } check(prop) } - + test("random exclusive range tests") { val gen = for { a <- Gen.choose(-100, 100) @@ -177,7 +177,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { } check(prop) } - + test("exclusive ranges of longs") { val data = 1L until 100L val slices = ParallelCollectionRDD.slice(data, 3) @@ -185,7 +185,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("inclusive ranges of longs") { val data = 1L to 100L val slices = ParallelCollectionRDD.slice(data, 3) @@ -193,7 +193,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 100) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("exclusive ranges of doubles") { val data = 1.0 until 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) @@ -201,7 +201,7 @@ class ParallelCollectionSplitSuite extends FunSuite with Checkers { assert(slices.map(_.size).reduceLeft(_+_) === 99) assert(slices.forall(_.isInstanceOf[NumericRange[_]])) } - + test("inclusive ranges of doubles") { val data = 1.0 to 100.0 by 1.0 val slices = ParallelCollectionRDD.slice(data, 3) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index dc704e07a81de..4cdccdda6f72e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -216,7 +216,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc test("onTaskGettingResult() called when result fetched remotely") { val listener = new SaveTaskEvents sc.addSparkListener(listener) - + // Make a task whose result is larger than the akka frame size System.setProperty("spark.akka.frameSize", "1") val akkaFrameSize = @@ -236,7 +236,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc test("onTaskGettingResult() not called when result sent directly") { val listener = new SaveTaskEvents sc.addSparkListener(listener) - + // Make a task whose result is larger than the akka frame size val result = sc.parallelize(Seq(1), 1).map(2 * _).reduce { case (x, y) => x } assert(result === 2) diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 356e28dd19bc5..2fb750d9ee378 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -264,7 +264,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin test("Scheduler does not always schedule tasks on the same workers") { sc = new SparkContext("local", "TaskSchedulerImplSuite") - val taskScheduler = new TaskSchedulerImpl(sc) + val taskScheduler = new TaskSchedulerImpl(sc) taskScheduler.initialize(new FakeSchedulerBackend) // Need to initialize a DAGScheduler for the taskScheduler to use for callbacks. val dagScheduler = new DAGScheduler(sc, taskScheduler) { diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 45c322427930d..2f9739f940dc6 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -33,8 +33,8 @@ class UISuite extends FunSuite { val server = new Server(startPort) Try { server.start() } match { - case Success(s) => - case Failure(e) => + case Success(s) => + case Failure(e) => // Either case server port is busy hence setup for test complete } val serverInfo1 = JettyUtils.startJettyServer( diff --git a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala index 439e5644e20a3..d7e48e633e0ee 100644 --- a/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/ClosureCleanerSuite.scala @@ -69,7 +69,7 @@ object TestObject { class TestClass extends Serializable { var x = 5 - + def getX = x def run(): Int = { diff --git a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala index e1446cbc90bdb..32d74d0500b72 100644 --- a/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/NextIteratorSuite.scala @@ -32,7 +32,7 @@ class NextIteratorSuite extends FunSuite with ShouldMatchers { i.hasNext should be === false intercept[NoSuchElementException] { i.next() } } - + test("two iterations") { val i = new StubIterator(Buffer(1, 2)) i.hasNext should be === true @@ -70,7 +70,7 @@ class NextIteratorSuite extends FunSuite with ShouldMatchers { class StubIterator(ints: Buffer[Int]) extends NextIterator[Int] { var closeCalled = 0 - + override def getNext() = { if (ints.size == 0) { finished = true diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 757476efdb789..39199a1a17ccd 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -29,12 +29,12 @@ class XORShiftRandomSuite extends FunSuite with ShouldMatchers { val xorRand = new XORShiftRandom(seed) val hundMil = 1e8.toInt } - + /* - * This test is based on a chi-squared test for randomness. The values are hard-coded + * This test is based on a chi-squared test for randomness. The values are hard-coded * so as not to create Spark's dependency on apache.commons.math3 just to call one * method for calculating the exact p-value for a given number of random numbers - * and bins. In case one would want to move to a full-fledged test based on + * and bins. In case one would want to move to a full-fledged test based on * apache.commons.math3, the relevant class is here: * org.apache.commons.math3.stat.inference.ChiSquareTest */ @@ -49,19 +49,19 @@ class XORShiftRandomSuite extends FunSuite with ShouldMatchers { // populate bins based on modulus of the random number times(f.hundMil) {bins(math.abs(f.xorRand.nextInt) % 10) += 1} - /* since the seed is deterministic, until the algorithm is changed, we know the result will be - * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, - * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) - * significance level. However, should the RNG implementation change, the test should still - * pass at the same significance level. The chi-squared test done in R gave the following + /* since the seed is deterministic, until the algorithm is changed, we know the result will be + * exactly this: Array(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, + * 10000790, 10002286, 9998699), so the test will never fail at the prespecified (5%) + * significance level. However, should the RNG implementation change, the test should still + * pass at the same significance level. The chi-squared test done in R gave the following * results: * > chisq.test(c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, * 10000790, 10002286, 9998699)) * Chi-squared test for given probabilities - * data: c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, + * data: c(10004908, 9993136, 9994600, 10000744, 10000091, 10002474, 10002272, 10000790, * 10002286, 9998699) * X-squared = 11.975, df = 9, p-value = 0.2147 - * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million + * Note that the p-value was ~0.22. The test will fail if alpha < 0.05, which for 100 million * random numbers * and 10 bins will happen at X-squared of ~16.9196. So, the test will fail if X-squared * is greater than or equal to that number. diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 41e813d48c7b8..1204cfba39f77 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -48,41 +48,41 @@ import org.apache.spark.streaming.dstream._ * @param storageLevel RDD storage level. */ -private[streaming] +private[streaming] class MQTTInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, brokerUrl: String, topic: String, storageLevel: StorageLevel ) extends NetworkInputDStream[T](ssc_) with Logging { - + def getReceiver(): NetworkReceiver[T] = { new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]] } } -private[streaming] +private[streaming] class MQTTReceiver(brokerUrl: String, topic: String, storageLevel: StorageLevel ) extends NetworkReceiver[Any] { lazy protected val blockGenerator = new BlockGenerator(storageLevel) - + def onStop() { blockGenerator.stop() } - + def onStart() { blockGenerator.start() - // Set up persistence for messages + // Set up persistence for messages var peristance: MqttClientPersistence = new MemoryPersistence() // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance var client: MqttClient = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance) - // Connect to MqttBroker + // Connect to MqttBroker client.connect() // Subscribe to Mqtt topic @@ -91,7 +91,7 @@ class MQTTReceiver(brokerUrl: String, // Callback automatically triggers as and when new message arrives on specified topic var callback: MqttCallback = new MqttCallback() { - // Handles Mqtt message + // Handles Mqtt message override def messageArrived(arg0: String, arg1: MqttMessage) { blockGenerator += new String(arg1.getPayload()) } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 3316b6dc39d6b..843a4a7a9ad72 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -31,7 +31,7 @@ import org.apache.spark.storage.StorageLevel * @constructor create a new Twitter stream using the supplied Twitter4J authentication credentials. * An optional set of string filters can be used to restrict the set of tweets. The Twitter API is * such that this may return a sampled subset of all tweets during each interval. -* +* * If no Authorization object is provided, initializes OAuth authorization using the system * properties twitter4j.oauth.consumerKey, .consumerSecret, .accessToken and .accessTokenSecret. */ @@ -42,13 +42,13 @@ class TwitterInputDStream( filters: Seq[String], storageLevel: StorageLevel ) extends NetworkInputDStream[Status](ssc_) { - + private def createOAuthAuthorization(): Authorization = { new OAuthAuthorization(new ConfigurationBuilder().build()) } private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) - + override def getReceiver(): NetworkReceiver[Status] = { new TwitterReceiver(authorization, filters, storageLevel) } diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 377d9d6bd5e72..5635287694ee2 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -172,7 +172,7 @@ class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Seriali "EdgeDirection.Either instead.") } } - + /** * Join the vertices with an RDD and then apply a function from the * the vertex and RDD entry to a new vertex value. The input table diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index 6386306c048fc..a467ca1ae715a 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -55,7 +55,7 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { } } } - + test ("filter") { withSpark { sc => val n = 5 diff --git a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala index e41d9bbe18c37..7f6d94571b5ef 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/optimization/Optimizer.scala @@ -30,7 +30,7 @@ import org.apache.spark.mllib.linalg.Vector trait Optimizer extends Serializable { /** - * Solve the provided convex optimization problem. + * Solve the provided convex optimization problem. */ def optimize(data: RDD[(Double, Vector)], initialWeights: Vector): Vector } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala index 3bd0017aa196a..d969e7aa60061 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/GeneralizedLinearAlgorithm.scala @@ -26,7 +26,7 @@ import org.apache.spark.mllib.optimization._ import org.apache.spark.mllib.linalg.{Vectors, Vector} /** - * GeneralizedLinearModel (GLM) represents a model trained using + * GeneralizedLinearModel (GLM) represents a model trained using * GeneralizedLinearAlgorithm. GLMs consist of a weight vector and * an intercept. * @@ -38,7 +38,7 @@ abstract class GeneralizedLinearModel(val weights: Vector, val intercept: Double /** * Predict the result given a data point and the weights learned. - * + * * @param dataMatrix Row vector containing the features for this data point * @param weightMatrix Column vector containing the weights of the model * @param intercept Intercept of the model. diff --git a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala index a30dcfdcecf27..687e85ca94d3c 100644 --- a/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala +++ b/repl/src/main/scala/org/apache/spark/repl/ExecutorClassLoader.scala @@ -35,7 +35,7 @@ import com.esotericsoftware.reflectasm.shaded.org.objectweb.asm.Opcodes._ * A ClassLoader that reads classes from a Hadoop FileSystem or HTTP URI, * used to load classes defined by the interpreter when the REPL is used. * Allows the user to specify if user class path should be first - */ + */ class ExecutorClassLoader(classUri: String, parent: ClassLoader, userClassPathFirst: Boolean) extends ClassLoader { val uri = new URI(classUri) @@ -94,7 +94,7 @@ class ExecutorClassLoader(classUri: String, parent: ClassLoader, case e: Exception => None } } - + def readAndTransformClass(name: String, in: InputStream): Array[Byte] = { if (name.startsWith("line") && name.endsWith("$iw$")) { // Class seems to be an interpreter "wrapper" object storing a val or var. diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala index 8f61a5e835044..419796b68b113 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkImports.scala @@ -187,7 +187,7 @@ trait SparkImports { if (currentImps contains imv) addWrapper() val objName = req.lineRep.readPath val valName = "$VAL" + newValId(); - + if(!code.toString.endsWith(".`" + imv + "`;\n")) { // Which means already imported code.append("val " + valName + " = " + objName + ".INSTANCE;\n") code.append("import " + valName + req.accessPath + ".`" + imv + "`;\n") 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 17118499d0c87..1f3fab09e9566 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 @@ -28,7 +28,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { override def toString = s"CAST($child, $dataType)" type EvaluatedType = Any - + def nullOrCast[T](a: Any, func: T => Any): Any = if(a == null) { null } else { @@ -40,7 +40,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case BinaryType => nullOrCast[Array[Byte]](_, new String(_, "UTF-8")) case _ => nullOrCast[Any](_, _.toString) } - + // BinaryConverter def castToBinary: Any => Any = child.dataType match { case StringType => nullOrCast[String](_, _.getBytes("UTF-8")) @@ -58,7 +58,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case DoubleType => nullOrCast[Double](_, _ != 0) case FloatType => nullOrCast[Float](_, _ != 0) } - + // TimestampConverter def castToTimestamp: Any => Any = child.dataType match { case StringType => nullOrCast[String](_, s => { 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 8a1db8e796816..dd9332ada80dd 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 @@ -86,7 +86,7 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Numeric children expressions. Those expressions are supposed + * Evaluation helper function for 2 Numeric children expressions. Those expressions are supposed * to be in the same data type, and also the return type. * Either one of the expressions result is null, the evaluation result should be null. */ @@ -120,7 +120,7 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Fractional children expressions. Those expressions are + * Evaluation helper function for 2 Fractional children expressions. Those expressions are * supposed to be in the same data type, and also the return type. * Either one of the expressions result is null, the evaluation result should be null. */ @@ -153,7 +153,7 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Integral children expressions. Those expressions are + * Evaluation helper function for 2 Integral children expressions. Those expressions are * supposed to be in the same data type, and also the return type. * Either one of the expressions result is null, the evaluation result should be null. */ @@ -186,12 +186,12 @@ abstract class Expression extends TreeNode[Expression] { } /** - * Evaluation helper function for 2 Comparable children expressions. Those expressions are + * Evaluation helper function for 2 Comparable children expressions. Those expressions are * supposed to be in the same data type, and the return type should be Integer: * Negative value: 1st argument less than 2nd argument * Zero: 1st argument equals 2nd argument * Positive value: 1st argument greater than 2nd argument - * + * * Either one of the expressions result is null, the evaluation result should be null. */ @inline @@ -213,7 +213,7 @@ abstract class Expression extends TreeNode[Expression] { null } else { e1.dataType match { - case i: NativeType => + case i: NativeType => f.asInstanceOf[(Ordering[i.JvmType], i.JvmType, i.JvmType) => Boolean]( i.ordering, evalE1.asInstanceOf[i.JvmType], evalE2.asInstanceOf[i.JvmType]) case other => sys.error(s"Type $other does not support ordered operations") 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 index a27c71db1b999..ddc16ce87b895 100644 --- 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 @@ -28,19 +28,19 @@ trait StringRegexExpression { self: BinaryExpression => type EvaluatedType = Any - + def escape(v: String): String def matches(regex: Pattern, str: String): Boolean - + def nullable: Boolean = true def dataType: DataType = BooleanType - - // try cache the pattern for Literal + + // try cache the pattern for Literal private lazy val cache: Pattern = right match { case x @ Literal(value: String, StringType) => compile(value) case _ => null } - + protected def compile(str: String): Pattern = if(str == null) { null } else { @@ -49,7 +49,7 @@ trait StringRegexExpression { } protected def pattern(str: String) = if(cache == null) compile(str) else cache - + override def eval(input: Row): Any = { val l = left.eval(input) if (l == null) { @@ -73,11 +73,11 @@ trait StringRegexExpression { /** * Simple RegEx pattern matching function */ -case class Like(left: Expression, right: Expression) +case class Like(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - + def symbol = "LIKE" - + // replace the _ with .{1} exactly match 1 time of any character // replace the % with .*, match 0 or more times with any character override def escape(v: String) = { @@ -98,19 +98,19 @@ case class Like(left: Expression, right: Expression) sb.append(Pattern.quote(Character.toString(n))); } } - + i += 1 } - + sb.toString() } - + override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).matches() } -case class RLike(left: Expression, right: Expression) +case class RLike(left: Expression, right: Expression) extends BinaryExpression with StringRegexExpression { - + def symbol = "RLIKE" override def escape(v: String): String = v override def matches(regex: Pattern, str: String): Boolean = regex.matcher(str).find(0) diff --git a/sql/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 index cdeb01a9656f4..da34bd3a21503 100644 --- a/sql/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 @@ -55,9 +55,9 @@ case object BooleanType extends NativeType { case object TimestampType extends NativeType { type JvmType = Timestamp - + @transient lazy val tag = typeTag[JvmType] - + val ordering = new Ordering[JvmType] { def compare(x: Timestamp, y: Timestamp) = x.compareTo(y) } diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 888a19d79f7e4..2cd0d2b0e1385 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -144,7 +144,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("abc" like "b%", false) checkEvaluation("abc" like "bc%", false) } - + test("LIKE Non-literal Regular Expression") { val regEx = 'a.string.at(0) checkEvaluation("abcd" like regEx, null, new GenericRow(Array[Any](null))) @@ -164,7 +164,7 @@ class ExpressionEvaluationSuite extends FunSuite { test("RLIKE literal Regular Expression") { checkEvaluation("abdef" rlike "abdef", true) checkEvaluation("abbbbc" rlike "a.*c", true) - + checkEvaluation("fofo" rlike "^fo", true) checkEvaluation("fo\no" rlike "^fo\no$", true) checkEvaluation("Bn" rlike "^Ba*n", true) @@ -196,9 +196,9 @@ class ExpressionEvaluationSuite extends FunSuite { evaluate("abbbbc" rlike regEx, new GenericRow(Array[Any]("**"))) } } - + test("data type casting") { - + val sts = "1970-01-01 00:00:01.0" val ts = Timestamp.valueOf(sts) @@ -236,7 +236,7 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("23" cast ShortType, 23) checkEvaluation("2012-12-11" cast DoubleType, null) checkEvaluation(Literal(123) cast IntegerType, 123) - + intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala index 65eae3357a21e..1cbf973c34917 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/ScalaReflectionRelationSuite.scala @@ -56,4 +56,4 @@ class ScalaReflectionRelationSuite extends FunSuite { val result = sql("SELECT data FROM reflectBinary").collect().head(0).asInstanceOf[Array[Byte]] assert(result.toSeq === Seq[Byte](1)) } -} \ No newline at end of file +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala index 93023e8dced57..ac56ff709c1c4 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Checkpoint.scala @@ -59,7 +59,7 @@ class Checkpoint(@transient ssc: StreamingContext, val checkpointTime: Time) } } -private[streaming] +private[streaming] object Checkpoint extends Logging { val PREFIX = "checkpoint-" val REGEX = (PREFIX + """([\d]+)([\w\.]*)""").r @@ -79,7 +79,7 @@ object Checkpoint extends Logging { def sortFunc(path1: Path, path2: Path): Boolean = { val (time1, bk1) = path1.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } val (time2, bk2) = path2.getName match { case REGEX(x, y) => (x.toLong, !y.isEmpty) } - (time1 < time2) || (time1 == time2 && bk1) + (time1 < time2) || (time1 == time2 && bk1) } val path = new Path(checkpointDir) @@ -95,7 +95,7 @@ object Checkpoint extends Logging { } } else { logInfo("Checkpoint directory " + path + " does not exist") - Seq.empty + Seq.empty } } } @@ -160,7 +160,7 @@ class CheckpointWriter( }) } - // All done, print success + // All done, print success val finishTime = System.currentTimeMillis() logInfo("Checkpoint for time " + checkpointTime + " saved to file '" + checkpointFile + "', took " + bytes.length + " bytes and " + (finishTime - startTime) + " ms") @@ -227,14 +227,14 @@ object CheckpointReader extends Logging { { val checkpointPath = new Path(checkpointDir) def fs = checkpointPath.getFileSystem(hadoopConf) - - // Try to find the checkpoint files + + // Try to find the checkpoint files val checkpointFiles = Checkpoint.getCheckpointFiles(checkpointDir, fs).reverse if (checkpointFiles.isEmpty) { return None } - // Try to read the checkpoint files in the order + // Try to read the checkpoint files in the order logInfo("Checkpoint files found: " + checkpointFiles.mkString(",")) val compressionCodec = CompressionCodec.createCodec(conf) checkpointFiles.foreach(file => { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala index 16479a01272aa..ad4f3fdd14ad6 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Interval.scala @@ -20,11 +20,11 @@ package org.apache.spark.streaming private[streaming] class Interval(val beginTime: Time, val endTime: Time) { def this(beginMs: Long, endMs: Long) = this(new Time(beginMs), new Time(endMs)) - + def duration(): Duration = endTime - beginTime def + (time: Duration): Interval = { - new Interval(beginTime + time, endTime + time) + new Interval(beginTime + time, endTime + time) } def - (time: Duration): Interval = { @@ -40,9 +40,9 @@ class Interval(val beginTime: Time, val endTime: Time) { } def <= (that: Interval) = (this < that || this == that) - + def > (that: Interval) = !(this <= that) - + def >= (that: Interval) = !(this < that) override def toString = "[" + beginTime + ", " + endTime + "]" diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index 2678334f53844..6a6b00a778b48 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -32,7 +32,7 @@ case class Time(private val millis: Long) { def <= (that: Time): Boolean = (this.millis <= that.millis) def > (that: Time): Boolean = (this.millis > that.millis) - + def >= (that: Time): Boolean = (this.millis >= that.millis) def + (that: Duration): Time = new Time(millis + that.milliseconds) @@ -43,7 +43,7 @@ case class Time(private val millis: Long) { def floor(that: Duration): Time = { val t = that.milliseconds - val m = math.floor(this.millis / t).toLong + val m = math.floor(this.millis / t).toLong new Time(m * t) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala index 903e3f3c9b713..f33c0ceafdf42 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStreamCheckpointData.scala @@ -51,7 +51,7 @@ class DStreamCheckpointData[T: ClassTag] (dstream: DStream[T]) .map(x => (x._1, x._2.getCheckpointFile.get)) logDebug("Current checkpoint files:\n" + checkpointFiles.toSeq.mkString("\n")) - // Add the checkpoint files to the data to be serialized + // Add the checkpoint files to the data to be serialized if (!checkpointFiles.isEmpty) { currentCheckpointFiles.clear() currentCheckpointFiles ++= checkpointFiles diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala index 8a6051622e2d5..e878285f6a854 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/FileInputDStream.scala @@ -232,7 +232,7 @@ class FileInputDStream[K: ClassTag, V: ClassTag, F <: NewInputFormat[K,V] : Clas } logDebug("Accepted " + path) } catch { - case fnfe: java.io.FileNotFoundException => + case fnfe: java.io.FileNotFoundException => logWarning("Error finding new files", fnfe) reset() return false diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala index 97325f8ea3117..6376cff78b78a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/QueueInputDStream.scala @@ -31,11 +31,11 @@ class QueueInputDStream[T: ClassTag]( oneAtATime: Boolean, defaultRDD: RDD[T] ) extends InputDStream[T](ssc) { - + override def start() { } - + override def stop() { } - + override def compute(validTime: Time): Option[RDD[T]] = { val buffer = new ArrayBuffer[RDD[T]]() if (oneAtATime && queue.size > 0) { @@ -55,5 +55,5 @@ class QueueInputDStream[T: ClassTag]( None } } - + } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index 44eb2750c6c7a..f5984d03c5342 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -47,7 +47,7 @@ object ReceiverSupervisorStrategy { * the API for pushing received data into Spark Streaming for being processed. * * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * + * * @example {{{ * class MyActor extends Actor with Receiver{ * def receive { diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala index c5ef2cc8c390d..39145a3ab081a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/Clock.scala @@ -19,34 +19,34 @@ package org.apache.spark.streaming.util private[streaming] trait Clock { - def currentTime(): Long + def currentTime(): Long def waitTillTime(targetTime: Long): Long } private[streaming] class SystemClock() extends Clock { - + val minPollTime = 25L - + def currentTime(): Long = { System.currentTimeMillis() - } - + } + def waitTillTime(targetTime: Long): Long = { var currentTime = 0L currentTime = System.currentTimeMillis() - + var waitTime = targetTime - currentTime if (waitTime <= 0) { return currentTime } - + val pollTime = { if (waitTime / 10.0 > minPollTime) { (waitTime / 10.0).toLong } else { - minPollTime - } + minPollTime + } } while (true) { @@ -55,7 +55,7 @@ class SystemClock() extends Clock { if (waitTime <= 0) { return currentTime } - val sleepTime = + val sleepTime = if (waitTime < pollTime) { waitTime } else { @@ -69,7 +69,7 @@ class SystemClock() extends Clock { private[streaming] class ManualClock() extends Clock { - + var time = 0L def currentTime() = time @@ -85,13 +85,13 @@ class ManualClock() extends Clock { this.synchronized { time += timeToAdd this.notifyAll() - } + } } def waitTillTime(targetTime: Long): Long = { this.synchronized { while (time < targetTime) { this.wait(100) - } + } } currentTime() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index 07021ebb5802a..bd1df55cf70f5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -25,8 +25,8 @@ import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { - /** - * Splits lines and counts the words in them using specialized object-to-long hashmap + /** + * Splits lines and counts the words in them using specialized object-to-long hashmap * (to avoid boxing-unboxing overhead of Long in java/scala HashMap) */ def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = { @@ -55,13 +55,13 @@ object RawTextHelper { map.toIterator.map{case (k, v) => (k, v)} } - /** + /** * Gets the top k words in terms of word counts. Assumes that each word exists only once * in the `data` iterator (that is, the counts have been reduced). */ def topK(data: Iterator[(String, Long)], k: Int): Iterator[(String, Long)] = { val taken = new Array[(String, Long)](k) - + var i = 0 var len = 0 var done = false @@ -93,7 +93,7 @@ object RawTextHelper { } taken.toIterator } - + /** * Warms up the SparkContext in master and slave by running tasks to force JIT kick in * before real workload starts. @@ -106,11 +106,11 @@ object RawTextHelper { .count() } } - - def add(v1: Long, v2: Long) = (v1 + v2) - def subtract(v1: Long, v2: Long) = (v1 - v2) + def add(v1: Long, v2: Long) = (v1 + v2) + + def subtract(v1: Long, v2: Long) = (v1 - v2) - def max(v1: Long, v2: Long) = math.max(v1, v2) + def max(v1: Long, v2: Long) = math.max(v1, v2) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index f71938ac55ccb..e016377c94c0d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -22,10 +22,10 @@ import org.apache.spark.Logging private[streaming] class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: String) extends Logging { - + private val thread = new Thread("RecurringTimer - " + name) { setDaemon(true) - override def run() { loop } + override def run() { loop } } @volatile private var prevTime = -1L @@ -104,11 +104,11 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: private[streaming] object RecurringTimer { - + def main(args: Array[String]) { var lastRecurTime = 0L val period = 1000 - + def onRecur(time: Long) { val currentTime = System.currentTimeMillis() println("" + currentTime + ": " + (currentTime - lastRecurTime)) diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index 13fa64894b773..a0b1bbc34fa7c 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -1673,7 +1673,7 @@ public void testSocketTextStream() { @Test public void testSocketString() { - + class Converter implements Function> { public Iterable call(InputStream in) throws IOException { BufferedReader reader = new BufferedReader(new InputStreamReader(in)); From f99401a6308d5b9a9259d7597a35ba92f927aa50 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 10 Apr 2014 16:20:33 -0700 Subject: [PATCH 255/397] [SQL] Improve column pruning in the optimizer. Author: Michael Armbrust Closes #378 from marmbrus/columnPruning and squashes the following commits: 779da56 [Michael Armbrust] More consistent naming. 1a4e9ea [Michael Armbrust] More comments. 2f4e7b9 [Michael Armbrust] Improve column pruning in the optimizer. --- .../sql/catalyst/optimizer/Optimizer.scala | 51 ++++++++++++++++++- .../plans/logical/basicOperators.scala | 2 +- 2 files changed, 51 insertions(+), 2 deletions(-) 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 37b23ba58289c..c0a09a16ac98d 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 @@ -33,7 +33,56 @@ object Optimizer extends RuleExecutor[LogicalPlan] { Batch("Filter Pushdown", Once, CombineFilters, PushPredicateThroughProject, - PushPredicateThroughInnerJoin) :: Nil + PushPredicateThroughInnerJoin, + ColumnPruning) :: Nil +} + +/** + * Attempts to eliminate the reading of unneeded columns from the query plan using the following + * transformations: + * + * - Inserting Projections beneath the following operators: + * - Aggregate + * - Project <- Join + * - Collapse adjacent projections, performing alias substitution. + */ +object ColumnPruning extends Rule[LogicalPlan] { + def apply(plan: LogicalPlan): LogicalPlan = plan transform { + case a @ Aggregate(_, _, child) if (child.outputSet -- a.references).nonEmpty => + // Project away references that are not needed to calculate the required aggregates. + a.copy(child = Project(a.references.toSeq, child)) + + case Project(projectList, Join(left, right, joinType, condition)) => + // Collect the list of off references required either above or to evaluate the condition. + val allReferences: Set[Attribute] = + projectList.flatMap(_.references).toSet ++ condition.map(_.references).getOrElse(Set.empty) + /** Applies a projection when the child is producing unnecessary attributes */ + def prunedChild(c: LogicalPlan) = + if ((allReferences.filter(c.outputSet.contains) -- c.outputSet).nonEmpty) { + Project(allReferences.filter(c.outputSet.contains).toSeq, c) + } else { + c + } + + Project(projectList, Join(prunedChild(left), prunedChild(right), joinType, condition)) + + case Project(projectList1, Project(projectList2, child)) => + // Create a map of Aliases to their values from the child projection. + // e.g., 'SELECT ... FROM (SELECT a + b AS c, d ...)' produces Map(c -> Alias(a + b, c)). + val aliasMap = projectList2.collect { + case a @ Alias(e, _) => (a.toAttribute: Expression, a) + }.toMap + + // Substitute any attributes that are produced by the child projection, so that we safely + // eliminate it. + // e.g., 'SELECT c + 1 FROM (SELECT a + b AS C ...' produces 'SELECT a + b + 1 ...' + // TODO: Fix TransformBase to avoid the cast below. + val substitutedProjection = projectList1.map(_.transform { + case a if aliasMap.contains(a) => aliasMap(a) + }).asInstanceOf[Seq[NamedExpression]] + + Project(substitutedProjection, child) + } } /** 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 cfc0b0c3a8d98..397473e178867 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 @@ -127,7 +127,7 @@ case class Aggregate( extends UnaryNode { def output = aggregateExpressions.map(_.toAttribute) - def references = child.references + def references = (groupingExpressions ++ aggregateExpressions).flatMap(_.references).toSet } case class Limit(limit: Expression, child: LogicalPlan) extends UnaryNode { From 2c557837b4a12c644cc37bd00d02be04f3807637 Mon Sep 17 00:00:00 2001 From: Sundeep Narravula Date: Thu, 10 Apr 2014 17:10:11 -0700 Subject: [PATCH 256/397] SPARK-1202 - Add a "cancel" button in the UI for stages Author: Sundeep Narravula Author: Sundeep Narravula Closes #246 from sundeepn/uikilljob and squashes the following commits: 5fdd0e2 [Sundeep Narravula] Fix test string f6fdff1 [Sundeep Narravula] Format fix; reduced line size to less than 100 chars d1daeb9 [Sundeep Narravula] Incorporating review comments. 8d97923 [Sundeep Narravula] Ability to kill jobs thru the UI. This behavior can be turned on be settings the following variable: spark.ui.killEnabled=true (default=false) Adding DAGScheduler event StageCancelled and corresponding handlers. Added cancellation reason to handlers. --- .../scala/org/apache/spark/SparkContext.scala | 10 ++++++ .../apache/spark/scheduler/DAGScheduler.scala | 32 ++++++++++++++++--- .../spark/scheduler/DAGSchedulerEvent.scala | 2 ++ .../scala/org/apache/spark/ui/SparkUI.scala | 1 + .../org/apache/spark/ui/jobs/IndexPage.scala | 14 +++++++- .../apache/spark/ui/jobs/JobProgressUI.scala | 1 + .../org/apache/spark/ui/jobs/StagePage.scala | 1 + .../org/apache/spark/ui/jobs/StageTable.scala | 29 +++++++++++++---- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- docs/configuration.md | 7 ++++ 10 files changed, 87 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e6c9b7000d819..3bcc8ce2b25a6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1138,6 +1138,16 @@ class SparkContext(config: SparkConf) extends Logging { dagScheduler.cancelAllJobs() } + /** Cancel a given job if it's scheduled or running */ + private[spark] def cancelJob(jobId: Int) { + dagScheduler.cancelJob(jobId) + } + + /** Cancel a given stage and all jobs associated with it */ + private[spark] def cancelStage(stageId: Int) { + dagScheduler.cancelStage(stageId) + } + /** * Clean a closure to make it ready to serialized and send to tasks * (removes unreferenced variables in $outer's, updates REPL variables) diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c41d6d75a1d49..c6cbf14e20069 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -511,6 +511,13 @@ class DAGScheduler( eventProcessActor ! AllJobsCancelled } + /** + * Cancel all jobs associated with a running or scheduled stage. + */ + def cancelStage(stageId: Int) { + eventProcessActor ! StageCancelled(stageId) + } + /** * Process one event retrieved from the event processing actor. * @@ -551,6 +558,9 @@ class DAGScheduler( submitStage(finalStage) } + case StageCancelled(stageId) => + handleStageCancellation(stageId) + case JobCancelled(jobId) => handleJobCancellation(jobId) @@ -560,11 +570,13 @@ class DAGScheduler( val activeInGroup = activeJobs.filter(activeJob => groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach(handleJobCancellation) + jobIds.foreach(jobId => handleJobCancellation(jobId, + "as part of cancelled job group %s".format(groupId))) case AllJobsCancelled => // Cancel all running jobs. - runningStages.map(_.jobId).foreach(handleJobCancellation) + runningStages.map(_.jobId).foreach(jobId => handleJobCancellation(jobId, + "as part of cancellation of all jobs")) activeJobs.clear() // These should already be empty by this point, jobIdToActiveJob.clear() // but just in case we lost track of some jobs... @@ -991,11 +1003,23 @@ class DAGScheduler( } } - private def handleJobCancellation(jobId: Int) { + private def handleStageCancellation(stageId: Int) { + if (stageIdToJobIds.contains(stageId)) { + val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray + jobsThatUseStage.foreach(jobId => { + handleJobCancellation(jobId, "because Stage %s was cancelled".format(stageId)) + }) + } else { + logInfo("No active jobs to kill for Stage " + stageId) + } + } + + private def handleJobCancellation(jobId: Int, reason: String = "") { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { - failJobAndIndependentStages(jobIdToActiveJob(jobId), s"Job $jobId cancelled", None) + failJobAndIndependentStages(jobIdToActiveJob(jobId), + "Job %d cancelled %s".format(jobId, reason), None) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 293cfb65643a6..7367c08b5d324 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -44,6 +44,8 @@ private[scheduler] case class JobSubmitted( properties: Properties = null) extends DAGSchedulerEvent +private[scheduler] case class StageCancelled(stageId: Int) extends DAGSchedulerEvent + private[scheduler] case class JobCancelled(jobId: Int) extends DAGSchedulerEvent private[scheduler] case class JobGroupCancelled(groupId: String) extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index b8e6e15880bf5..dac11ec1cf52f 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -46,6 +46,7 @@ private[spark] class SparkUI( val live = sc != null val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) + val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index f811aff616bcf..5da5d1f2a3f45 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -32,6 +32,7 @@ private[ui] class IndexPage(parent: JobProgressUI) { private val sc = parent.sc private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler + private val killEnabled = parent.killEnabled private def appName = parent.appName @@ -42,7 +43,18 @@ private[ui] class IndexPage(parent: JobProgressUI) { val failedStages = listener.failedStages.reverse.toSeq val now = System.currentTimeMillis() - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) + if (killEnabled) { + val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean + val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt + + if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { + sc.cancelStage(stageId) + } + } + + + val activeStagesTable = + new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) val completedStagesTable = new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index ad1a12cdc4e36..9de659d6c7393 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -32,6 +32,7 @@ private[ui] class JobProgressUI(parent: SparkUI) { val basePath = parent.basePath val live = parent.live val sc = parent.sc + val killEnabled = parent.killEnabled lazy val listener = _listener.get lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index 0bcbd7461cc5b..b6c3e3cf45163 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -30,6 +30,7 @@ import org.apache.spark.util.{Utils, Distribution} private[ui] class StagePage(parent: JobProgressUI) { private val basePath = parent.basePath private lazy val listener = parent.listener + private lazy val sc = parent.sc private def appName = parent.appName diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index ac61568af52d2..1e874ae4969f9 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -27,7 +27,11 @@ import org.apache.spark.ui.{WebUI, UIUtils} import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { +private[ui] class StageTable( + stages: Seq[StageInfo], + parent: JobProgressUI, + killEnabled: Boolean = false) { + private val basePath = parent.basePath private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler @@ -71,15 +75,28 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) {
    } - /** Render an HTML row that represents a stage */ - private def stageRow(s: StageInfo): Seq[Node] = { - val poolName = listener.stageIdToPool.get(s.stageId) + private def makeDescription(s: StageInfo): Seq[Node] = { val nameLink = {s.name} + val killLink = if (killEnabled) { + + + } val description = listener.stageIdToDescription.get(s.stageId) - .map(d =>
    {d}
    {nameLink}
    ).getOrElse(nameLink) + .map(d =>
    {d}
    {nameLink} {killLink}
    ) + .getOrElse(
    {nameLink} {killLink}
    ) + + return description + } + + /** Render an HTML row that represents a stage */ + private def stageRow(s: StageInfo): Seq[Node] = { + val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { case Some(t) => WebUI.formatDate(new Date(t)) case None => "Unknown" @@ -118,7 +135,7 @@ private[ui] class StageTable(stages: Seq[StageInfo], parent: JobProgressUI) { }} - {description} + {makeDescription(s)} {submissionTime} {formattedDuration} diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index a74724d785ad3..db4df1d1212ff 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -290,7 +290,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont val rdd = makeRdd(1, Nil) val jobId = submit(rdd, Array(0)) cancel(jobId) - assert(failure.getMessage === s"Job $jobId cancelled") + assert(failure.getMessage === s"Job $jobId cancelled ") assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(sparkListener.failedStages.contains(0)) assert(sparkListener.failedStages.size === 1) diff --git a/docs/configuration.md b/docs/configuration.md index 9c602402f0635..f3bfd036f4164 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -190,6 +190,13 @@ Apart from these, the following properties are also available, and may be useful user that started the Spark job has view access. + + spark.ui.killEnabled + true + + Allows stages and corresponding jobs to be killed from the web ui. + + spark.shuffle.compress true From 5cd11d51c19321981a6234a7765c7a5be6913433 Mon Sep 17 00:00:00 2001 From: Ivan Wick Date: Thu, 10 Apr 2014 17:49:30 -0700 Subject: [PATCH 257/397] Set spark.executor.uri from environment variable (needed by Mesos) The Mesos backend uses this property when setting up a slave process. It is similarly set in the Scala repl (org.apache.spark.repl.SparkILoop), but I couldn't find any analogous for pyspark. Author: Ivan Wick This patch had conflicts when merged, resolved by Committer: Matei Zaharia Closes #311 from ivanwick/master and squashes the following commits: da0c3e4 [Ivan Wick] Set spark.executor.uri from environment variable (needed by Mesos) --- python/pyspark/shell.py | 3 +++ 1 file changed, 3 insertions(+) diff --git a/python/pyspark/shell.py b/python/pyspark/shell.py index 35e48276e3cb9..61613dbed8dce 100644 --- a/python/pyspark/shell.py +++ b/python/pyspark/shell.py @@ -29,6 +29,9 @@ # this is the equivalent of ADD_JARS add_files = os.environ.get("ADD_FILES").split(',') if os.environ.get("ADD_FILES") != None else None +if os.environ.get("SPARK_EXECUTOR_URI"): + SparkContext.setSystemProperty("spark.executor.uri", os.environ["SPARK_EXECUTOR_URI"]) + sc = SparkContext(os.environ.get("MASTER", "local[*]"), "PySparkShell", pyFiles=add_files) print """Welcome to From 7b4203ab4c640f7875ae3536228ed4d791062017 Mon Sep 17 00:00:00 2001 From: Harvey Feng Date: Thu, 10 Apr 2014 18:25:54 -0700 Subject: [PATCH 258/397] Add Spark v0.9.1 to ec2 launch script and use it as the default Mainly ported from branch-0.9. Author: Harvey Feng Closes #385 from harveyfeng/0.9.1-ec2 and squashes the following commits: 769ac2f [Harvey Feng] Add Spark v0.9.1 to ec2 launch script and use it as the default --- ec2/spark_ec2.py | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/ec2/spark_ec2.py b/ec2/spark_ec2.py index d8840c94ac17c..31209a662bbe1 100755 --- a/ec2/spark_ec2.py +++ b/ec2/spark_ec2.py @@ -70,7 +70,7 @@ def parse_args(): "slaves across multiple (an additional $0.01/Gb for bandwidth" + "between zones applies)") parser.add_option("-a", "--ami", help="Amazon Machine Image ID to use") - parser.add_option("-v", "--spark-version", default="0.9.0", + parser.add_option("-v", "--spark-version", default="0.9.1", help="Version of Spark to use: 'X.Y.Z' or a specific git hash") parser.add_option("--spark-git-repo", default="https://github.com/apache/spark", @@ -157,7 +157,7 @@ def is_active(instance): # Return correct versions of Spark and Shark, given the supplied Spark version def get_spark_shark_version(opts): - spark_shark_map = {"0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0"} + spark_shark_map = {"0.7.3": "0.7.1", "0.8.0": "0.8.0", "0.8.1": "0.8.1", "0.9.0": "0.9.0", "0.9.1": "0.9.1"} version = opts.spark_version.replace("v", "") if version not in spark_shark_map: print >> stderr, "Don't know about Spark version: %s" % version From 44f654eecd3c181f2aeaff3871acf7f00eacc6b9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 10 Apr 2014 20:43:56 -0700 Subject: [PATCH 259/397] SPARK-1202: Improvements to task killing in the UI. 1. Adds a separate endpoint for the killing logic that is outside of a page. 2. Narrows the scope of the killingEnabled tracking. 3. Some style improvements. Author: Patrick Wendell Closes #386 from pwendell/kill-link and squashes the following commits: 8efe02b [Patrick Wendell] Improvements to task killing in the UI. --- .../org/apache/spark/ui/static/webui.css | 9 +++++++++ .../scala/org/apache/spark/ui/JettyUtils.scala | 2 ++ .../scala/org/apache/spark/ui/SparkUI.scala | 5 ++--- .../org/apache/spark/ui/jobs/IndexPage.scala | 11 ----------- .../apache/spark/ui/jobs/JobProgressUI.scala | 17 ++++++++++++++++- .../org/apache/spark/ui/jobs/StageTable.scala | 16 +++++++++------- 6 files changed, 38 insertions(+), 22 deletions(-) diff --git a/core/src/main/resources/org/apache/spark/ui/static/webui.css b/core/src/main/resources/org/apache/spark/ui/static/webui.css index fe54c34ffb1da..599c3ac9b57c0 100644 --- a/core/src/main/resources/org/apache/spark/ui/static/webui.css +++ b/core/src/main/resources/org/apache/spark/ui/static/webui.css @@ -78,3 +78,12 @@ table.sortable thead { background-repeat: repeat-x; filter: progid:dximagetransform.microsoft.gradient(startColorstr='#FFA4EDFF', endColorstr='#FF94DDFF', GradientType=0); } + +span.kill-link { + margin-right: 2px; + color: gray; +} + +span.kill-link a { + color: gray; +} diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 9ce0398d010a8..dd0818e8ab01c 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -104,10 +104,12 @@ private[spark] object JettyUtils extends Logging { def createRedirectHandler( srcPath: String, destPath: String, + beforeRedirect: HttpServletRequest => Unit = x => (), basePath: String = ""): ServletContextHandler = { val prefixedDestPath = attachPrefix(basePath, destPath) val servlet = new HttpServlet { override def doGet(request: HttpServletRequest, response: HttpServletResponse) { + beforeRedirect(request) // Make sure we don't end up with "//" in the middle val newUrl = new URL(new URL(request.getRequestURL.toString), prefixedDestPath).toString response.sendRedirect(newUrl) diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index dac11ec1cf52f..4c891d73afa87 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils /** Top level user interface for Spark */ private[spark] class SparkUI( val sc: SparkContext, - conf: SparkConf, + val conf: SparkConf, val listenerBus: SparkListenerBus, var appName: String, val basePath: String = "") @@ -46,7 +46,6 @@ private[spark] class SparkUI( val live = sc != null val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) @@ -70,7 +69,7 @@ private[spark] class SparkUI( metricsServletHandlers ++ Seq[ServletContextHandler] ( createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"), - createRedirectHandler("/", "/stages", basePath) + createRedirectHandler("/", "/stages", basePath = basePath) ) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala index 5da5d1f2a3f45..8619a31380f1e 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala @@ -32,7 +32,6 @@ private[ui] class IndexPage(parent: JobProgressUI) { private val sc = parent.sc private lazy val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler - private val killEnabled = parent.killEnabled private def appName = parent.appName @@ -43,16 +42,6 @@ private[ui] class IndexPage(parent: JobProgressUI) { val failedStages = listener.failedStages.reverse.toSeq val now = System.currentTimeMillis() - if (killEnabled) { - val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean - val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt - - if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { - sc.cancelStage(stageId) - } - } - - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) val completedStagesTable = diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala index 9de659d6c7393..30e3f35f2182b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala @@ -32,7 +32,7 @@ private[ui] class JobProgressUI(parent: SparkUI) { val basePath = parent.basePath val live = parent.live val sc = parent.sc - val killEnabled = parent.killEnabled + val killEnabled = parent.conf.getBoolean("spark.ui.killEnabled", true) lazy val listener = _listener.get lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) @@ -51,7 +51,22 @@ private[ui] class JobProgressUI(parent: SparkUI) { def formatDuration(ms: Long) = Utils.msDurationToString(ms) + private def handleKillRequest(request: HttpServletRequest) = { + if (killEnabled) { + val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean + val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt + if (stageId >= 0 && killFlag && listener.activeStages.contains(stageId)) { + sc.cancelStage(stageId) + } + // Do a quick pause here to give Spark time to kill the stage so it shows up as + // killed after the refresh. Note that this will block the serving thread so the + // time should be limited in duration. + Thread.sleep(100) + } + } + def getHandlers = Seq[ServletContextHandler]( + createRedirectHandler("/stages/stage/kill", "/stages", handleKillRequest), createServletHandler("/stages/stage", (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), createServletHandler("/stages/pool", diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 1e874ae4969f9..e419fae5a6589 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -76,20 +76,22 @@ private[ui] class StageTable( } private def makeDescription(s: StageInfo): Seq[Node] = { + // scalastyle:off + val killLink = if (killEnabled) { + + (kill) + + } + // scalastyle:on + val nameLink = {s.name} - val killLink = if (killEnabled) { - - } val description = listener.stageIdToDescription.get(s.stageId) .map(d =>
    {d}
    {nameLink} {killLink}
    ) - .getOrElse(
    {nameLink} {killLink}
    ) + .getOrElse(
    {killLink}{nameLink}
    ) return description } From 446bb3417a2855a194d49acc0ac316a021eced9d Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Fri, 11 Apr 2014 13:17:48 +0530 Subject: [PATCH 260/397] SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken Author: Thomas Graves Closes #344 from tgravescs/SPARK-1417 and squashes the following commits: c450b5f [Thomas Graves] fix test e1c1d7e [Thomas Graves] add missing $ to appUIAddress e982ddb [Thomas Graves] use appUIHostPort in appUIAddress 0803ec2 [Thomas Graves] Review comment updates - remove extra newline, simplify assert in test 658a8ec [Thomas Graves] Add a appUIHostPort routine 0614208 [Thomas Graves] Fix test 2a6b1b7 [Thomas Graves] SPARK-1417: Spark on Yarn - spark UI link from resourcemanager is broken --- .../scala/org/apache/spark/ui/SparkUI.scala | 7 +++- .../scala/org/apache/spark/SparkUISuite.scala | 35 +++++++++++++++++++ .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 2 +- 4 files changed, 43 insertions(+), 3 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/SparkUISuite.scala diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 4c891d73afa87..7fa4fd3149eb6 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -113,7 +113,12 @@ private[spark] class SparkUI( logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) } - private[spark] def appUIAddress = "http://" + publicHost + ":" + boundPort + /** + * Return the application UI host:port. This does not include the scheme (http://). + */ + private[spark] def appUIHostPort = publicHost + ":" + boundPort + + private[spark] def appUIAddress = s"http://$appUIHostPort" } diff --git a/core/src/test/scala/org/apache/spark/SparkUISuite.scala b/core/src/test/scala/org/apache/spark/SparkUISuite.scala new file mode 100644 index 0000000000000..d0d119c15081d --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SparkUISuite.scala @@ -0,0 +1,35 @@ +/* + * 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 + +import java.net.URI + +import org.scalatest.FunSuite + +class SparkUISuite extends FunSuite with SharedSparkContext { + + test("verify appUIAddress contains the scheme") { + val uiAddress = sc.ui.appUIAddress + assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + } + + test("verify appUIAddress contains the port") { + val splitUIAddress = sc.ui.appUIAddress.split(':') + assert(splitUIAddress(2).toInt == sc.ui.boundPort) + } +} diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 910484ed5432a..67ec95c8fc04f 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -234,7 +234,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, assert(sparkContext != null || count >= numTries) if (null != sparkContext) { - uiAddress = sparkContext.ui.appUIAddress + uiAddress = sparkContext.ui.appUIHostPort this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, resourceManager, diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index c8a4d2e647cbd..61af0f9ac5ca0 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -220,7 +220,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, assert(sparkContext != null || numTries >= maxNumTries) if (sparkContext != null) { - uiAddress = sparkContext.ui.appUIAddress + uiAddress = sparkContext.ui.appUIHostPort this.yarnAllocator = YarnAllocationHandler.newAllocator( yarnConf, amClient, From 98225a6effd077a1b97c7e485d45ffd89b2c5b7f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 11 Apr 2014 10:45:27 -0700 Subject: [PATCH 261/397] Some clean up in build/docs (a) Deleted an outdated line from the docs (b) Removed a work around that is no longer necessary given the mesos version bump. Author: Patrick Wendell Closes #382 from pwendell/maven-clean and squashes the following commits: f0447fa [Patrick Wendell] Minor doc clean-up --- docs/index.md | 2 -- 1 file changed, 2 deletions(-) diff --git a/docs/index.md b/docs/index.md index 7a13fa9a9a2b6..89ec5b05488a9 100644 --- a/docs/index.md +++ b/docs/index.md @@ -67,8 +67,6 @@ In addition, if you wish to run Spark on [YARN](running-on-yarn.html), set Note that on Windows, you need to set the environment variables on separate lines, e.g., `set SPARK_HADOOP_VERSION=1.2.1`. -For this version of Spark (0.8.1) Hadoop 2.2.x (or newer) users will have to build Spark and publish it locally. See [Launching Spark on YARN](running-on-yarn.html). This is needed because Hadoop 2.2 has non backwards compatible API changes. - # Where to Go from Here **Programming guides:** From f5ace8da34c58d1005c7c377cfe3df21102c1dd6 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 11 Apr 2014 12:06:13 -0700 Subject: [PATCH 262/397] [SPARK-1225, 1241] [MLLIB] Add AreaUnderCurve and BinaryClassificationMetrics This PR implements a generic version of `AreaUnderCurve` using the `RDD.sliding` implementation from https://github.com/apache/spark/pull/136 . It also contains refactoring of https://github.com/apache/spark/pull/160 for binary classification evaluation. Author: Xiangrui Meng Closes #364 from mengxr/auc and squashes the following commits: a05941d [Xiangrui Meng] replace TP/FP/TN/FN by their full names 3f42e98 [Xiangrui Meng] add (0, 0), (1, 1) to roc, and (0, 1) to pr fb4b6d2 [Xiangrui Meng] rename Evaluator to Metrics and add more metrics b1b7dab [Xiangrui Meng] fix code styles 9dc3518 [Xiangrui Meng] add tests for BinaryClassificationEvaluator ca31da5 [Xiangrui Meng] remove PredictionAndResponse 3d71525 [Xiangrui Meng] move binary evalution classes to evaluation.binary 8f78958 [Xiangrui Meng] add PredictionAndResponse dda82d5 [Xiangrui Meng] add confusion matrix aa7e278 [Xiangrui Meng] add initial version of binary classification evaluator 221ebce [Xiangrui Meng] add a new test to sliding a920865 [Xiangrui Meng] Merge branch 'sliding' into auc a9b250a [Xiangrui Meng] move sliding to mllib cab9a52 [Xiangrui Meng] use last for the last element db6cb30 [Xiangrui Meng] remove unnecessary toSeq 9916202 [Xiangrui Meng] change RDD.sliding return type to RDD[Seq[T]] 284d991 [Xiangrui Meng] change SlidedRDD to SlidingRDD c1c6c22 [Xiangrui Meng] add AreaUnderCurve 65461b2 [Xiangrui Meng] Merge branch 'sliding' into auc 5ee6001 [Xiangrui Meng] add TODO d2a600d [Xiangrui Meng] add sliding to rdd --- .../mllib/evaluation/AreaUnderCurve.scala | 62 ++++++ .../BinaryClassificationMetricComputers.scala | 57 +++++ .../binary/BinaryClassificationMetrics.scala | 204 ++++++++++++++++++ .../binary/BinaryConfusionMatrix.scala | 41 ++++ .../apache/spark/mllib/rdd/RDDFunctions.scala | 53 +++++ .../apache/spark/mllib/rdd/SlidingRDD.scala | 104 +++++++++ .../evaluation/AreaUnderCurveSuite.scala | 46 ++++ .../BinaryClassificationMetricsSuite.scala | 55 +++++ .../spark/mllib/rdd/RDDFunctionsSuite.scala | 49 +++++ 9 files changed, 671 insertions(+) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala create mode 100644 mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala new file mode 100644 index 0000000000000..7858ec602483f --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/AreaUnderCurve.scala @@ -0,0 +1,62 @@ +/* + * 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.mllib.evaluation + +import org.apache.spark.rdd.RDD +import org.apache.spark.mllib.rdd.RDDFunctions._ + +/** + * Computes the area under the curve (AUC) using the trapezoidal rule. + */ +private[evaluation] object AreaUnderCurve { + + /** + * Uses the trapezoidal rule to compute the area under the line connecting the two input points. + * @param points two 2D points stored in Seq + */ + private def trapezoid(points: Seq[(Double, Double)]): Double = { + require(points.length == 2) + val x = points.head + val y = points.last + (y._1 - x._1) * (y._2 + x._2) / 2.0 + } + + /** + * Returns the area under the given curve. + * + * @param curve a RDD of ordered 2D points stored in pairs representing a curve + */ + def of(curve: RDD[(Double, Double)]): Double = { + curve.sliding(2).aggregate(0.0)( + seqOp = (auc: Double, points: Seq[(Double, Double)]) => auc + trapezoid(points), + combOp = _ + _ + ) + } + + /** + * Returns the area under the given curve. + * + * @param curve an iterator over ordered 2D points stored in pairs representing a curve + */ + def of(curve: Iterable[(Double, Double)]): Double = { + curve.toIterator.sliding(2).withPartial(false).aggregate(0.0)( + seqop = (auc: Double, points: Seq[(Double, Double)]) => auc + trapezoid(points), + combop = _ + _ + ) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala new file mode 100644 index 0000000000000..562663ad36b40 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricComputers.scala @@ -0,0 +1,57 @@ +/* + * 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.mllib.evaluation.binary + +/** + * Trait for a binary classification evaluation metric computer. + */ +private[evaluation] trait BinaryClassificationMetricComputer extends Serializable { + def apply(c: BinaryConfusionMatrix): Double +} + +/** Precision. */ +private[evaluation] object Precision extends BinaryClassificationMetricComputer { + override def apply(c: BinaryConfusionMatrix): Double = + c.numTruePositives.toDouble / (c.numTruePositives + c.numFalsePositives) +} + +/** False positive rate. */ +private[evaluation] object FalsePositiveRate extends BinaryClassificationMetricComputer { + override def apply(c: BinaryConfusionMatrix): Double = + c.numFalsePositives.toDouble / c.numNegatives +} + +/** Recall. */ +private[evaluation] object Recall extends BinaryClassificationMetricComputer { + override def apply(c: BinaryConfusionMatrix): Double = + c.numTruePositives.toDouble / c.numPositives +} + +/** + * F-Measure. + * @param beta the beta constant in F-Measure + * @see http://en.wikipedia.org/wiki/F1_score + */ +private[evaluation] case class FMeasure(beta: Double) extends BinaryClassificationMetricComputer { + private val beta2 = beta * beta + override def apply(c: BinaryConfusionMatrix): Double = { + val precision = Precision(c) + val recall = Recall(c) + (1.0 + beta2) * (precision * recall) / (beta2 * precision + recall) + } +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala new file mode 100644 index 0000000000000..ed7b0fc943367 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetrics.scala @@ -0,0 +1,204 @@ +/* + * 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.mllib.evaluation.binary + +import org.apache.spark.rdd.{UnionRDD, RDD} +import org.apache.spark.SparkContext._ +import org.apache.spark.mllib.evaluation.AreaUnderCurve +import org.apache.spark.Logging + +/** + * Implementation of [[org.apache.spark.mllib.evaluation.binary.BinaryConfusionMatrix]]. + * + * @param count label counter for labels with scores greater than or equal to the current score + * @param totalCount label counter for all labels + */ +private case class BinaryConfusionMatrixImpl( + count: LabelCounter, + totalCount: LabelCounter) extends BinaryConfusionMatrix with Serializable { + + /** number of true positives */ + override def numTruePositives: Long = count.numPositives + + /** number of false positives */ + override def numFalsePositives: Long = count.numNegatives + + /** number of false negatives */ + override def numFalseNegatives: Long = totalCount.numPositives - count.numPositives + + /** number of true negatives */ + override def numTrueNegatives: Long = totalCount.numNegatives - count.numNegatives + + /** number of positives */ + override def numPositives: Long = totalCount.numPositives + + /** number of negatives */ + override def numNegatives: Long = totalCount.numNegatives +} + +/** + * Evaluator for binary classification. + * + * @param scoreAndLabels an RDD of (score, label) pairs. + */ +class BinaryClassificationMetrics(scoreAndLabels: RDD[(Double, Double)]) + extends Serializable with Logging { + + private lazy val ( + cumulativeCounts: RDD[(Double, LabelCounter)], + confusions: RDD[(Double, BinaryConfusionMatrix)]) = { + // Create a bin for each distinct score value, count positives and negatives within each bin, + // and then sort by score values in descending order. + val counts = scoreAndLabels.combineByKey( + createCombiner = (label: Double) => new LabelCounter(0L, 0L) += label, + mergeValue = (c: LabelCounter, label: Double) => c += label, + mergeCombiners = (c1: LabelCounter, c2: LabelCounter) => c1 += c2 + ).sortByKey(ascending = false) + val agg = counts.values.mapPartitions({ iter => + val agg = new LabelCounter() + iter.foreach(agg += _) + Iterator(agg) + }, preservesPartitioning = true).collect() + val partitionwiseCumulativeCounts = + agg.scanLeft(new LabelCounter())((agg: LabelCounter, c: LabelCounter) => agg.clone() += c) + val totalCount = partitionwiseCumulativeCounts.last + logInfo(s"Total counts: $totalCount") + val cumulativeCounts = counts.mapPartitionsWithIndex( + (index: Int, iter: Iterator[(Double, LabelCounter)]) => { + val cumCount = partitionwiseCumulativeCounts(index) + iter.map { case (score, c) => + cumCount += c + (score, cumCount.clone()) + } + }, preservesPartitioning = true) + cumulativeCounts.persist() + val confusions = cumulativeCounts.map { case (score, cumCount) => + (score, BinaryConfusionMatrixImpl(cumCount, totalCount).asInstanceOf[BinaryConfusionMatrix]) + } + (cumulativeCounts, confusions) + } + + /** Unpersist intermediate RDDs used in the computation. */ + def unpersist() { + cumulativeCounts.unpersist() + } + + /** Returns thresholds in descending order. */ + def thresholds(): RDD[Double] = cumulativeCounts.map(_._1) + + /** + * Returns the receiver operating characteristic (ROC) curve, + * which is an RDD of (false positive rate, true positive rate) + * with (0.0, 0.0) prepended and (1.0, 1.0) appended to it. + * @see http://en.wikipedia.org/wiki/Receiver_operating_characteristic + */ + def roc(): RDD[(Double, Double)] = { + val rocCurve = createCurve(FalsePositiveRate, Recall) + val sc = confusions.context + val first = sc.makeRDD(Seq((0.0, 0.0)), 1) + val last = sc.makeRDD(Seq((1.0, 1.0)), 1) + new UnionRDD[(Double, Double)](sc, Seq(first, rocCurve, last)) + } + + /** + * Computes the area under the receiver operating characteristic (ROC) curve. + */ + def areaUnderROC(): Double = AreaUnderCurve.of(roc()) + + /** + * Returns the precision-recall curve, which is an RDD of (recall, precision), + * NOT (precision, recall), with (0.0, 1.0) prepended to it. + * @see http://en.wikipedia.org/wiki/Precision_and_recall + */ + def pr(): RDD[(Double, Double)] = { + val prCurve = createCurve(Recall, Precision) + val sc = confusions.context + val first = sc.makeRDD(Seq((0.0, 1.0)), 1) + first.union(prCurve) + } + + /** + * Computes the area under the precision-recall curve. + */ + def areaUnderPR(): Double = AreaUnderCurve.of(pr()) + + /** + * Returns the (threshold, F-Measure) curve. + * @param beta the beta factor in F-Measure computation. + * @return an RDD of (threshold, F-Measure) pairs. + * @see http://en.wikipedia.org/wiki/F1_score + */ + def fMeasureByThreshold(beta: Double): RDD[(Double, Double)] = createCurve(FMeasure(beta)) + + /** Returns the (threshold, F-Measure) curve with beta = 1.0. */ + def fMeasureByThreshold(): RDD[(Double, Double)] = fMeasureByThreshold(1.0) + + /** Returns the (threshold, precision) curve. */ + def precisionByThreshold(): RDD[(Double, Double)] = createCurve(Precision) + + /** Returns the (threshold, recall) curve. */ + def recallByThreshold(): RDD[(Double, Double)] = createCurve(Recall) + + /** Creates a curve of (threshold, metric). */ + private def createCurve(y: BinaryClassificationMetricComputer): RDD[(Double, Double)] = { + confusions.map { case (s, c) => + (s, y(c)) + } + } + + /** Creates a curve of (metricX, metricY). */ + private def createCurve( + x: BinaryClassificationMetricComputer, + y: BinaryClassificationMetricComputer): RDD[(Double, Double)] = { + confusions.map { case (_, c) => + (x(c), y(c)) + } + } +} + +/** + * A counter for positives and negatives. + * + * @param numPositives number of positive labels + * @param numNegatives number of negative labels + */ +private class LabelCounter( + var numPositives: Long = 0L, + var numNegatives: Long = 0L) extends Serializable { + + /** Processes a label. */ + def +=(label: Double): LabelCounter = { + // Though we assume 1.0 for positive and 0.0 for negative, the following check will handle + // -1.0 for negative as well. + if (label > 0.5) numPositives += 1L else numNegatives += 1L + this + } + + /** Merges another counter. */ + def +=(other: LabelCounter): LabelCounter = { + numPositives += other.numPositives + numNegatives += other.numNegatives + this + } + + override def clone: LabelCounter = { + new LabelCounter(numPositives, numNegatives) + } + + override def toString: String = s"{numPos: $numPositives, numNeg: $numNegatives}" +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala new file mode 100644 index 0000000000000..75a75b216002a --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/evaluation/binary/BinaryConfusionMatrix.scala @@ -0,0 +1,41 @@ +/* + * 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.mllib.evaluation.binary + +/** + * Trait for a binary confusion matrix. + */ +private[evaluation] trait BinaryConfusionMatrix { + /** number of true positives */ + def numTruePositives: Long + + /** number of false positives */ + def numFalsePositives: Long + + /** number of false negatives */ + def numFalseNegatives: Long + + /** number of true negatives */ + def numTrueNegatives: Long + + /** number of positives */ + def numPositives: Long = numTruePositives + numFalseNegatives + + /** number of negatives */ + def numNegatives: Long = numFalsePositives + numTrueNegatives +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala new file mode 100644 index 0000000000000..873de871fd884 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -0,0 +1,53 @@ +/* + * 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.mllib.rdd + +import scala.reflect.ClassTag + +import org.apache.spark.rdd.RDD + +/** + * Machine learning specific RDD functions. + */ +private[mllib] +class RDDFunctions[T: ClassTag](self: RDD[T]) { + + /** + * Returns a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding + * window over them. The ordering is first based on the partition index and then the ordering of + * items within each partition. This is similar to sliding in Scala collections, except that it + * becomes an empty RDD if the window size is greater than the total number of items. It needs to + * trigger a Spark job if the parent RDD has more than one partitions and the window size is + * greater than 1. + */ + def sliding(windowSize: Int): RDD[Seq[T]] = { + require(windowSize > 0, s"Sliding window size must be positive, but got $windowSize.") + if (windowSize == 1) { + self.map(Seq(_)) + } else { + new SlidingRDD[T](self, windowSize) + } + } +} + +private[mllib] +object RDDFunctions { + + /** Implicit conversion from an RDD to RDDFunctions. */ + implicit def fromRDD[T: ClassTag](rdd: RDD[T]) = new RDDFunctions[T](rdd) +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala new file mode 100644 index 0000000000000..dd80782c0f001 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/SlidingRDD.scala @@ -0,0 +1,104 @@ +/* + * 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.mllib.rdd + +import scala.collection.mutable +import scala.reflect.ClassTag + +import org.apache.spark.{TaskContext, Partition} +import org.apache.spark.rdd.RDD + +private[mllib] +class SlidingRDDPartition[T](val idx: Int, val prev: Partition, val tail: Seq[T]) + extends Partition with Serializable { + override val index: Int = idx +} + +/** + * Represents a RDD from grouping items of its parent RDD in fixed size blocks by passing a sliding + * window over them. The ordering is first based on the partition index and then the ordering of + * items within each partition. This is similar to sliding in Scala collections, except that it + * becomes an empty RDD if the window size is greater than the total number of items. It needs to + * trigger a Spark job if the parent RDD has more than one partitions. To make this operation + * efficient, the number of items per partition should be larger than the window size and the + * window size should be small, e.g., 2. + * + * @param parent the parent RDD + * @param windowSize the window size, must be greater than 1 + * + * @see [[org.apache.spark.mllib.rdd.RDDFunctions#sliding]] + */ +private[mllib] +class SlidingRDD[T: ClassTag](@transient val parent: RDD[T], val windowSize: Int) + extends RDD[Seq[T]](parent) { + + require(windowSize > 1, s"Window size must be greater than 1, but got $windowSize.") + + override def compute(split: Partition, context: TaskContext): Iterator[Seq[T]] = { + val part = split.asInstanceOf[SlidingRDDPartition[T]] + (firstParent[T].iterator(part.prev, context) ++ part.tail) + .sliding(windowSize) + .withPartial(false) + } + + override def getPreferredLocations(split: Partition): Seq[String] = + firstParent[T].preferredLocations(split.asInstanceOf[SlidingRDDPartition[T]].prev) + + override def getPartitions: Array[Partition] = { + val parentPartitions = parent.partitions + val n = parentPartitions.size + if (n == 0) { + Array.empty + } else if (n == 1) { + Array(new SlidingRDDPartition[T](0, parentPartitions(0), Seq.empty)) + } else { + val n1 = n - 1 + val w1 = windowSize - 1 + // Get the first w1 items of each partition, starting from the second partition. + val nextHeads = + parent.context.runJob(parent, (iter: Iterator[T]) => iter.take(w1).toArray, 1 until n, true) + val partitions = mutable.ArrayBuffer[SlidingRDDPartition[T]]() + var i = 0 + var partitionIndex = 0 + while (i < n1) { + var j = i + val tail = mutable.ListBuffer[T]() + // Keep appending to the current tail until appended a head of size w1. + while (j < n1 && nextHeads(j).size < w1) { + tail ++= nextHeads(j) + j += 1 + } + if (j < n1) { + tail ++= nextHeads(j) + j += 1 + } + partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(i), tail) + partitionIndex += 1 + // Skip appended heads. + i = j + } + // If the head of last partition has size w1, we also need to add this partition. + if (nextHeads.last.size == w1) { + partitions += new SlidingRDDPartition[T](partitionIndex, parentPartitions(n1), Seq.empty) + } + partitions.toArray + } + } + + // TODO: Override methods such as aggregate, which only requires one Spark job. +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.scala new file mode 100644 index 0000000000000..1c9844f289fe0 --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/AreaUnderCurveSuite.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.mllib.evaluation + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext + +class AreaUnderCurveSuite extends FunSuite with LocalSparkContext { + test("auc computation") { + val curve = Seq((0.0, 0.0), (1.0, 1.0), (2.0, 3.0), (3.0, 0.0)) + val auc = 4.0 + assert(AreaUnderCurve.of(curve) === auc) + val rddCurve = sc.parallelize(curve, 2) + assert(AreaUnderCurve.of(rddCurve) == auc) + } + + test("auc of an empty curve") { + val curve = Seq.empty[(Double, Double)] + assert(AreaUnderCurve.of(curve) === 0.0) + val rddCurve = sc.parallelize(curve, 2) + assert(AreaUnderCurve.of(rddCurve) === 0.0) + } + + test("auc of a curve with a single point") { + val curve = Seq((1.0, 1.0)) + assert(AreaUnderCurve.of(curve) === 0.0) + val rddCurve = sc.parallelize(curve, 2) + assert(AreaUnderCurve.of(rddCurve) === 0.0) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala new file mode 100644 index 0000000000000..173fdaefab3da --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/evaluation/binary/BinaryClassificationMetricsSuite.scala @@ -0,0 +1,55 @@ +/* + * 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.mllib.evaluation.binary + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.evaluation.AreaUnderCurve + +class BinaryClassificationMetricsSuite extends FunSuite with LocalSparkContext { + test("binary evaluation metrics") { + val scoreAndLabels = sc.parallelize( + Seq((0.1, 0.0), (0.1, 1.0), (0.4, 0.0), (0.6, 0.0), (0.6, 1.0), (0.6, 1.0), (0.8, 1.0)), 2) + val metrics = new BinaryClassificationMetrics(scoreAndLabels) + val threshold = Seq(0.8, 0.6, 0.4, 0.1) + val numTruePositives = Seq(1, 3, 3, 4) + val numFalsePositives = Seq(0, 1, 2, 3) + val numPositives = 4 + val numNegatives = 3 + val precision = numTruePositives.zip(numFalsePositives).map { case (t, f) => + t.toDouble / (t + f) + } + val recall = numTruePositives.map(t => t.toDouble / numPositives) + val fpr = numFalsePositives.map(f => f.toDouble / numNegatives) + val rocCurve = Seq((0.0, 0.0)) ++ fpr.zip(recall) ++ Seq((1.0, 1.0)) + val pr = recall.zip(precision) + val prCurve = Seq((0.0, 1.0)) ++ pr + val f1 = pr.map { case (r, p) => 2.0 * (p * r) / (p + r) } + val f2 = pr.map { case (r, p) => 5.0 * (p * r) / (4.0 * p + r)} + assert(metrics.thresholds().collect().toSeq === threshold) + assert(metrics.roc().collect().toSeq === rocCurve) + assert(metrics.areaUnderROC() === AreaUnderCurve.of(rocCurve)) + assert(metrics.pr().collect().toSeq === prCurve) + assert(metrics.areaUnderPR() === AreaUnderCurve.of(prCurve)) + assert(metrics.fMeasureByThreshold().collect().toSeq === threshold.zip(f1)) + assert(metrics.fMeasureByThreshold(2.0).collect().toSeq === threshold.zip(f2)) + assert(metrics.precisionByThreshold().collect().toSeq === threshold.zip(precision)) + assert(metrics.recallByThreshold().collect().toSeq === threshold.zip(recall)) + } +} diff --git a/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala new file mode 100644 index 0000000000000..3f3b10dfff35e --- /dev/null +++ b/mllib/src/test/scala/org/apache/spark/mllib/rdd/RDDFunctionsSuite.scala @@ -0,0 +1,49 @@ +/* + * 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.mllib.rdd + +import org.scalatest.FunSuite + +import org.apache.spark.mllib.util.LocalSparkContext +import org.apache.spark.mllib.rdd.RDDFunctions._ + +class RDDFunctionsSuite extends FunSuite with LocalSparkContext { + + test("sliding") { + val data = 0 until 6 + for (numPartitions <- 1 to 8) { + val rdd = sc.parallelize(data, numPartitions) + for (windowSize <- 1 to 6) { + val sliding = rdd.sliding(windowSize).collect().map(_.toList).toList + val expected = data.sliding(windowSize).map(_.toList).toList + assert(sliding === expected) + } + assert(rdd.sliding(7).collect().isEmpty, + "Should return an empty RDD if the window size is greater than the number of items.") + } + } + + test("sliding with empty partitions") { + val data = Seq(Seq(1, 2, 3), Seq.empty[Int], Seq(4), Seq.empty[Int], Seq(5, 6, 7)) + val rdd = sc.parallelize(data, data.length).flatMap(s => s) + assert(rdd.partitions.size === data.length) + val sliding = rdd.sliding(3) + val expected = data.flatMap(x => x).sliding(3).toList + assert(sliding.collect().toList === expected) + } +} From 6a0f8e35ce7595c4ece11fe04133fd44ffbe5b06 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Fri, 11 Apr 2014 13:23:21 -0700 Subject: [PATCH 263/397] HOTFIX: Ignore python metastore files in RAT checks. This was causing some errors with pull request tests. Author: Patrick Wendell Closes #393 from pwendell/hotfix and squashes the following commits: 6201dd3 [Patrick Wendell] HOTFIX: Ignore python metastore files in RAT checks. --- .rat-excludes | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/.rat-excludes b/.rat-excludes index a2b5665a0be26..8954330bd10a7 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -39,4 +39,6 @@ work .*\.q golden test.out/* -.*iml \ No newline at end of file +.*iml +python/metastore/service.properties +python/metastore/db.lck From 7038b00be9c84a4d92f9d95ff3d75fae47d57d87 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Fri, 11 Apr 2014 19:41:40 -0700 Subject: [PATCH 264/397] [FIX] make coalesce test deterministic in RDDSuite Make coalesce test deterministic by setting pre-defined seeds. (Saw random failures in other PRs.) Author: Xiangrui Meng Closes #387 from mengxr/fix-random and squashes the following commits: 59bc16f [Xiangrui Meng] make coalesce test deterministic in RDDSuite --- .../scala/org/apache/spark/rdd/RDDSuite.scala | 61 ++++++++++--------- 1 file changed, 33 insertions(+), 28 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 25973348a7837..1901330d8b188 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -274,37 +274,42 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("coalesced RDDs with locality, large scale (10K partitions)") { // large scale experiment import collection.mutable - val rnd = scala.util.Random val partitions = 10000 val numMachines = 50 val machines = mutable.ListBuffer[String]() - (1 to numMachines).foreach(machines += "m"+_) - - val blocks = (1 to partitions).map(i => - { (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) } ) - - val data2 = sc.makeRDD(blocks) - val coalesced2 = data2.coalesce(numMachines*2) - - // test that you get over 90% locality in each group - val minLocality = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) - assert(minLocality >= 0.90, "Expected 90% locality but got " + (minLocality*100.0).toInt + "%") - - // test that the groups are load balanced with 100 +/- 20 elements in each - val maxImbalance = coalesced2.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) - .foldLeft(0)((dev, curr) => math.max(math.abs(100-curr),dev)) - assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) - - val data3 = sc.makeRDD(blocks).map(i => i*2) // derived RDD to test *current* pref locs - val coalesced3 = data3.coalesce(numMachines*2) - val minLocality2 = coalesced3.partitions - .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) - .foldLeft(1.0)((perc, loc) => math.min(perc,loc)) - assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + - (minLocality2*100.0).toInt + "%") + (1 to numMachines).foreach(machines += "m" + _) + val rnd = scala.util.Random + for (seed <- 1 to 5) { + rnd.setSeed(seed) + + val blocks = (1 to partitions).map { i => + (i, Array.fill(3)(machines(rnd.nextInt(machines.size))).toList) + } + + val data2 = sc.makeRDD(blocks) + val coalesced2 = data2.coalesce(numMachines * 2) + + // test that you get over 90% locality in each group + val minLocality = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.0)((perc, loc) => math.min(perc, loc)) + assert(minLocality >= 0.90, "Expected 90% locality but got " + + (minLocality * 100.0).toInt + "%") + + // test that the groups are load balanced with 100 +/- 20 elements in each + val maxImbalance = coalesced2.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].parents.size) + .foldLeft(0)((dev, curr) => math.max(math.abs(100 - curr), dev)) + assert(maxImbalance <= 20, "Expected 100 +/- 20 per partition, but got " + maxImbalance) + + val data3 = sc.makeRDD(blocks).map(i => i * 2) // derived RDD to test *current* pref locs + val coalesced3 = data3.coalesce(numMachines * 2) + val minLocality2 = coalesced3.partitions + .map(part => part.asInstanceOf[CoalescedRDDPartition].localFraction) + .foldLeft(1.0)((perc, loc) => math.min(perc, loc)) + assert(minLocality2 >= 0.90, "Expected 90% locality for derived RDD but got " + + (minLocality2 * 100.0).toInt + "%") + } } test("zipped RDDs") { From fdfb45e691946f3153d6c696bec6d7f3e391e301 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Fri, 11 Apr 2014 19:43:22 -0700 Subject: [PATCH 265/397] [WIP] [SPARK-1328] Add vector statistics As with the new vector system in MLlib, we find that it is good to add some new APIs to precess the `RDD[Vector]`. Beside, the former implementation of `computeStat` is not stable which could loss precision, and has the possibility to cause `Nan` in scientific computing, just as said in the [SPARK-1328](https://spark-project.atlassian.net/browse/SPARK-1328). APIs contain: * rowMeans(): RDD[Double] * rowNorm2(): RDD[Double] * rowSDs(): RDD[Double] * colMeans(): Vector * colMeans(size: Int): Vector * colNorm2(): Vector * colNorm2(size: Int): Vector * colSDs(): Vector * colSDs(size: Int): Vector * maxOption((Vector, Vector) => Boolean): Option[Vector] * minOption((Vector, Vector) => Boolean): Option[Vector] * rowShrink(): RDD[Vector] * colShrink(): RDD[Vector] This is working in process now, and some more APIs will add to `LabeledPoint`. Moreover, the implicit declaration will move from `MLUtils` to `MLContext` later. Author: Xusen Yin Author: Xiangrui Meng Closes #268 from yinxusen/vector-statistics and squashes the following commits: d61363f [Xusen Yin] rebase to latest master 16ae684 [Xusen Yin] fix minor error and remove useless method 10cf5d3 [Xusen Yin] refine some return type b064714 [Xusen Yin] remove computeStat in MLUtils cbbefdb [Xiangrui Meng] update multivariate statistical summary interface and clean tests 4eaf28a [Xusen Yin] merge VectorRDDStatistics into RowMatrix 48ee053 [Xusen Yin] fix minor error e624f93 [Xusen Yin] fix scala style error 1fba230 [Xusen Yin] merge while loop together 69e1f37 [Xusen Yin] remove lazy eval, and minor memory footprint 548e9de [Xusen Yin] minor revision 86522c4 [Xusen Yin] add comments on functions dc77e38 [Xusen Yin] test sparse vector RDD 18cf072 [Xusen Yin] change def to lazy val to make sure that the computations in function be evaluated only once f7a3ca2 [Xusen Yin] fix the corner case of maxmin 967d041 [Xusen Yin] full revision with Aggregator class 138300c [Xusen Yin] add new Aggregator class 1376ff4 [Xusen Yin] rename variables and adjust code 4a5c38d [Xusen Yin] add scala doc, refine code and comments 036b7a5 [Xusen Yin] fix the bug of Nan occur f6e8e9a [Xusen Yin] add sparse vectors test 4cfbadf [Xusen Yin] fix bug of min max 4e4fbd1 [Xusen Yin] separate seqop and combop out as independent functions a6d5a2e [Xusen Yin] rewrite for only computing non-zero elements 3980287 [Xusen Yin] rename variables 62a2c3e [Xusen Yin] use axpy and in-place if possible 9a75ebd [Xusen Yin] add case class to wrap return values d816ac7 [Xusen Yin] remove useless APIs c4651bb [Xusen Yin] remove row-wise APIs and refine code 1338ea1 [Xusen Yin] all-in-one version test passed cc65810 [Xusen Yin] add parallel mean and variance 9af2e95 [Xusen Yin] refine the code style ad6c82d [Xusen Yin] add shrink test e09d5d2 [Xusen Yin] add scala docs and refine shrink method 8ef3377 [Xusen Yin] pass all tests 28cf060 [Xusen Yin] fix error of column means 54b19ab [Xusen Yin] add new API to shrink RDD[Vector] 8c6c0e1 [Xusen Yin] add basic statistics --- .../mllib/linalg/distributed/RowMatrix.scala | 165 +++++++++++++++++- .../stat/MultivariateStatisticalSummary.scala | 56 ++++++ .../org/apache/spark/mllib/util/MLUtils.scala | 57 +----- .../linalg/distributed/RowMatrixSuite.scala | 15 ++ .../spark/mllib/util/MLUtilsSuite.scala | 13 -- 5 files changed, 230 insertions(+), 76 deletions(-) create mode 100644 mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala diff --git a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala index f65f43dd3007b..0c0afcd9ec0d7 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/linalg/distributed/RowMatrix.scala @@ -19,7 +19,7 @@ package org.apache.spark.mllib.linalg.distributed import java.util -import breeze.linalg.{DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} +import breeze.linalg.{Vector => BV, DenseMatrix => BDM, DenseVector => BDV, svd => brzSvd} import breeze.numerics.{sqrt => brzSqrt} import com.github.fommil.netlib.BLAS.{getInstance => blas} @@ -27,6 +27,138 @@ import org.apache.spark.annotation.Experimental import org.apache.spark.mllib.linalg._ import org.apache.spark.rdd.RDD import org.apache.spark.Logging +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary + +/** + * Column statistics aggregator implementing + * [[org.apache.spark.mllib.stat.MultivariateStatisticalSummary]] + * together with add() and merge() function. + * A numerically stable algorithm is implemented to compute sample mean and variance: + *[[http://en.wikipedia.org/wiki/Algorithms_for_calculating_variance variance-wiki]]. + * Zero elements (including explicit zero values) are skipped when calling add() and merge(), + * to have time complexity O(nnz) instead of O(n) for each column. + */ +private class ColumnStatisticsAggregator(private val n: Int) + extends MultivariateStatisticalSummary with Serializable { + + private val currMean: BDV[Double] = BDV.zeros[Double](n) + private val currM2n: BDV[Double] = BDV.zeros[Double](n) + private var totalCnt = 0.0 + private val nnz: BDV[Double] = BDV.zeros[Double](n) + private val currMax: BDV[Double] = BDV.fill(n)(Double.MinValue) + private val currMin: BDV[Double] = BDV.fill(n)(Double.MaxValue) + + override def mean: Vector = { + val realMean = BDV.zeros[Double](n) + var i = 0 + while (i < n) { + realMean(i) = currMean(i) * nnz(i) / totalCnt + i += 1 + } + Vectors.fromBreeze(realMean) + } + + override def variance: Vector = { + val realVariance = BDV.zeros[Double](n) + + val denominator = totalCnt - 1.0 + + // Sample variance is computed, if the denominator is less than 0, the variance is just 0. + if (denominator > 0.0) { + val deltaMean = currMean + var i = 0 + while (i < currM2n.size) { + realVariance(i) = + currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * (totalCnt - nnz(i)) / totalCnt + realVariance(i) /= denominator + i += 1 + } + } + + Vectors.fromBreeze(realVariance) + } + + override def count: Long = totalCnt.toLong + + override def numNonzeros: Vector = Vectors.fromBreeze(nnz) + + override def max: Vector = { + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMax(i) < 0.0)) currMax(i) = 0.0 + i += 1 + } + Vectors.fromBreeze(currMax) + } + + override def min: Vector = { + var i = 0 + while (i < n) { + if ((nnz(i) < totalCnt) && (currMin(i) > 0.0)) currMin(i) = 0.0 + i += 1 + } + Vectors.fromBreeze(currMin) + } + + /** + * Aggregates a row. + */ + def add(currData: BV[Double]): this.type = { + currData.activeIterator.foreach { + case (_, 0.0) => // Skip explicit zero elements. + case (i, value) => + if (currMax(i) < value) { + currMax(i) = value + } + if (currMin(i) > value) { + currMin(i) = value + } + + val tmpPrevMean = currMean(i) + currMean(i) = (currMean(i) * nnz(i) + value) / (nnz(i) + 1.0) + currM2n(i) += (value - currMean(i)) * (value - tmpPrevMean) + + nnz(i) += 1.0 + } + + totalCnt += 1.0 + this + } + + /** + * Merges another aggregator. + */ + def merge(other: ColumnStatisticsAggregator): this.type = { + require(n == other.n, s"Dimensions mismatch. Expecting $n but got ${other.n}.") + + totalCnt += other.totalCnt + val deltaMean = currMean - other.currMean + + var i = 0 + while (i < n) { + // merge mean together + if (other.currMean(i) != 0.0) { + currMean(i) = (currMean(i) * nnz(i) + other.currMean(i) * other.nnz(i)) / + (nnz(i) + other.nnz(i)) + } + // merge m2n together + if (nnz(i) + other.nnz(i) != 0.0) { + currM2n(i) += other.currM2n(i) + deltaMean(i) * deltaMean(i) * nnz(i) * other.nnz(i) / + (nnz(i) + other.nnz(i)) + } + if (currMax(i) < other.currMax(i)) { + currMax(i) = other.currMax(i) + } + if (currMin(i) > other.currMin(i)) { + currMin(i) = other.currMin(i) + } + i += 1 + } + + nnz += other.nnz + this + } +} /** * :: Experimental :: @@ -182,13 +314,7 @@ class RowMatrix( combOp = (s1: (Long, BDV[Double]), s2: (Long, BDV[Double])) => (s1._1 + s2._1, s1._2 += s2._2) ) - // Update _m if it is not set, or verify its value. - if (nRows <= 0L) { - nRows = m - } else { - require(nRows == m, - s"The number of rows $m is different from what specified or previously computed: ${nRows}.") - } + updateNumRows(m) mean :/= m.toDouble @@ -240,6 +366,19 @@ class RowMatrix( } } + /** + * Computes column-wise summary statistics. + */ + def computeColumnSummaryStatistics(): MultivariateStatisticalSummary = { + val zeroValue = new ColumnStatisticsAggregator(numCols().toInt) + val summary = rows.map(_.toBreeze).aggregate[ColumnStatisticsAggregator](zeroValue)( + (aggregator, data) => aggregator.add(data), + (aggregator1, aggregator2) => aggregator1.merge(aggregator2) + ) + updateNumRows(summary.count) + summary + } + /** * Multiply this matrix by a local matrix on the right. * @@ -276,6 +415,16 @@ class RowMatrix( } mat } + + /** Updates or verfires the number of rows. */ + private def updateNumRows(m: Long) { + if (nRows <= 0) { + nRows == m + } else { + require(nRows == m, + s"The number of rows $m is different from what specified or previously computed: ${nRows}.") + } + } } object RowMatrix { diff --git a/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala new file mode 100644 index 0000000000000..f9eb343da2b82 --- /dev/null +++ b/mllib/src/main/scala/org/apache/spark/mllib/stat/MultivariateStatisticalSummary.scala @@ -0,0 +1,56 @@ +/* + * 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.mllib.stat + +import org.apache.spark.mllib.linalg.Vector + +/** + * Trait for multivariate statistical summary of a data matrix. + */ +trait MultivariateStatisticalSummary { + + /** + * Sample mean vector. + */ + def mean: Vector + + /** + * Sample variance vector. Should return a zero vector if the sample size is 1. + */ + def variance: Vector + + /** + * Sample size. + */ + def count: Long + + /** + * Number of nonzero elements (including explicitly presented zero values) in each column. + */ + def numNonzeros: Vector + + /** + * Maximum value of each column. + */ + def max: Vector + + /** + * Minimum value of each column. + */ + def min: Vector +} diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala index ac2360c429e2b..901c3180eac4c 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MLUtils.scala @@ -17,14 +17,13 @@ package org.apache.spark.mllib.util -import breeze.linalg.{Vector => BV, DenseVector => BDV, SparseVector => BSV, - squaredDistance => breezeSquaredDistance} +import breeze.linalg.{Vector => BV, SparseVector => BSV, squaredDistance => breezeSquaredDistance} import org.apache.spark.annotation.Experimental import org.apache.spark.SparkContext import org.apache.spark.rdd.RDD import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.{Vector, Vectors} +import org.apache.spark.mllib.linalg.Vectors /** * Helper methods to load, save and pre-process data used in ML Lib. @@ -158,58 +157,6 @@ object MLUtils { dataStr.saveAsTextFile(dir) } - /** - * Utility function to compute mean and standard deviation on a given dataset. - * - * @param data - input data set whose statistics are computed - * @param numFeatures - number of features - * @param numExamples - number of examples in input dataset - * - * @return (yMean, xColMean, xColSd) - Tuple consisting of - * yMean - mean of the labels - * xColMean - Row vector with mean for every column (or feature) of the input data - * xColSd - Row vector standard deviation for every column (or feature) of the input data. - */ - private[mllib] def computeStats( - data: RDD[LabeledPoint], - numFeatures: Int, - numExamples: Long): (Double, Vector, Vector) = { - val brzData = data.map { case LabeledPoint(label, features) => - (label, features.toBreeze) - } - val aggStats = brzData.aggregate( - (0L, 0.0, BDV.zeros[Double](numFeatures), BDV.zeros[Double](numFeatures)) - )( - seqOp = (c, v) => (c, v) match { - case ((n, sumLabel, sum, sumSq), (label, features)) => - features.activeIterator.foreach { case (i, x) => - sumSq(i) += x * x - } - (n + 1L, sumLabel + label, sum += features, sumSq) - }, - combOp = (c1, c2) => (c1, c2) match { - case ((n1, sumLabel1, sum1, sumSq1), (n2, sumLabel2, sum2, sumSq2)) => - (n1 + n2, sumLabel1 + sumLabel2, sum1 += sum2, sumSq1 += sumSq2) - } - ) - val (nl, sumLabel, sum, sumSq) = aggStats - - require(nl > 0, "Input data is empty.") - require(nl == numExamples) - - val n = nl.toDouble - val yMean = sumLabel / n - val mean = sum / n - val std = new Array[Double](sum.length) - var i = 0 - while (i < numFeatures) { - std(i) = sumSq(i) / n - mean(i) * mean(i) - i += 1 - } - - (yMean, Vectors.fromBreeze(mean), Vectors.dense(std)) - } - /** * Returns the squared Euclidean distance between two vectors. The following formula will be used * if it does not introduce too much numerical error: diff --git a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala index 71ee8e8a4f6fd..c9f9acf4c1335 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/linalg/distributed/RowMatrixSuite.scala @@ -170,4 +170,19 @@ class RowMatrixSuite extends FunSuite with LocalSparkContext { )) } } + + test("compute column summary statistics") { + for (mat <- Seq(denseMat, sparseMat)) { + val summary = mat.computeColumnSummaryStatistics() + // Run twice to make sure no internal states are changed. + for (k <- 0 to 1) { + assert(summary.mean === Vectors.dense(4.5, 3.0, 4.0), "mean mismatch") + assert(summary.variance === Vectors.dense(15.0, 10.0, 10.0), "variance mismatch") + assert(summary.count === m, "count mismatch.") + assert(summary.numNonzeros === Vectors.dense(3.0, 3.0, 4.0), "nnz mismatch") + assert(summary.max === Vectors.dense(9.0, 7.0, 8.0), "max mismatch") + assert(summary.min === Vectors.dense(0.0, 0.0, 1.0), "column mismatch.") + } + } + } } diff --git a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala index e451c350b8d88..812a8434784be 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/util/MLUtilsSuite.scala @@ -27,7 +27,6 @@ import com.google.common.base.Charsets import com.google.common.io.Files import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.regression.LabeledPoint import org.apache.spark.mllib.util.MLUtils._ class MLUtilsSuite extends FunSuite with LocalSparkContext { @@ -56,18 +55,6 @@ class MLUtilsSuite extends FunSuite with LocalSparkContext { } } - test("compute stats") { - val data = Seq.fill(3)(Seq( - LabeledPoint(1.0, Vectors.dense(1.0, 2.0, 3.0)), - LabeledPoint(0.0, Vectors.dense(3.0, 4.0, 5.0)) - )).flatten - val rdd = sc.parallelize(data, 2) - val (meanLabel, mean, std) = MLUtils.computeStats(rdd, 3, 6) - assert(meanLabel === 0.5) - assert(mean === Vectors.dense(2.0, 3.0, 4.0)) - assert(std === Vectors.dense(1.0, 1.0, 1.0)) - } - test("loadLibSVMData") { val lines = """ From aa8bb117a3ff98420ab751ba4ddbaad88ab57f9d Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Fri, 11 Apr 2014 20:33:42 -0700 Subject: [PATCH 266/397] Update WindowedDStream.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit update the content of Exception when windowDuration is not multiple of parent.slideDuration Author: baishuo(白硕) Closes #390 from baishuo/windowdstream and squashes the following commits: 533c968 [baishuo(白硕)] Update WindowedDStream.scala --- .../org/apache/spark/streaming/dstream/WindowedDStream.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala index 24289b714f99e..775b6bfd065c0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/WindowedDStream.scala @@ -32,7 +32,7 @@ class WindowedDStream[T: ClassTag]( extends DStream[T](parent.ssc) { if (!_windowDuration.isMultipleOf(parent.slideDuration)) { - throw new Exception("The window duration of windowed DStream (" + _slideDuration + ") " + + throw new Exception("The window duration of windowed DStream (" + _windowDuration + ") " + "must be a multiple of the slide duration of parent DStream (" + parent.slideDuration + ")") } From 165e06a74c3d75e6b7341c120943add8b035b96a Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 11 Apr 2014 22:46:47 -0700 Subject: [PATCH 267/397] SPARK-1057 (alternative) Remove fastutil (This is for discussion at this point -- I'm not suggesting this should be committed.) This is what removing fastutil looks like. Much of it is straightforward, like using `java.io` buffered stream classes, and Guava for murmurhash3. Uses of the `FastByteArrayOutputStream` were a little trickier. In only one case though do I think the change to use `java.io` actually entails an extra array copy. The rest is using `OpenHashMap` and `OpenHashSet`. These are now written in terms of more scala-like operations. `OpenHashMap` is where I made three non-trivial changes to make it work, and they need review: - It is no longer private - The key must be a `ClassTag` - Unless a lot of other code changes, the key type can't enforce being a supertype of `Null` It all works and tests pass, and I think there is reason to believe it's OK from a speed perspective. But what about those last changes? Author: Sean Owen Closes #266 from srowen/SPARK-1057-alternate and squashes the following commits: 2601129 [Sean Owen] Fix Map return type error not previously caught ec65502 [Sean Owen] Updates from matei's review 00bc81e [Sean Owen] Remove use of fastutil and replace with use of java.io, spark.util and Guava classes --- core/pom.xml | 4 --- .../spark/broadcast/HttpBroadcast.scala | 9 +++-- .../spark/partial/GroupedCountEvaluator.scala | 32 +++++++---------- .../main/scala/org/apache/spark/rdd/RDD.scala | 34 +++++++++---------- .../spark/scheduler/ReplayListenerBus.scala | 5 ++- .../org/apache/spark/scheduler/Task.scala | 9 ++--- .../apache/spark/serializer/Serializer.scala | 9 ++--- .../apache/spark/storage/BlockManager.scala | 10 +++--- .../spark/storage/BlockObjectWriter.scala | 6 ++-- .../org/apache/spark/util/FileLogger.scala | 5 ++- .../org/apache/spark/util/SizeEstimator.scala | 5 ++- .../spark/util/collection/AppendOnlyMap.scala | 7 ++-- .../collection/ExternalAppendOnlyMap.scala | 5 ++- .../spark/util/collection/OpenHashMap.scala | 3 +- .../spark/util/collection/OpenHashSet.scala | 4 +-- pom.xml | 5 --- project/SparkBuild.scala | 1 - .../spark/streaming/util/RawTextHelper.scala | 15 ++++---- .../spark/streaming/util/RawTextSender.scala | 11 +++--- 19 files changed, 72 insertions(+), 107 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index 1f808380817c9..a1bdd8ec68aeb 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -157,10 +157,6 @@
    - - it.unimi.dsi - fastutil - colt colt diff --git a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala index f6a8a8af91e4b..29372f16f2cac 100644 --- a/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala +++ b/core/src/main/scala/org/apache/spark/broadcast/HttpBroadcast.scala @@ -18,11 +18,10 @@ package org.apache.spark.broadcast import java.io.{File, FileOutputStream, ObjectInputStream, ObjectOutputStream, OutputStream} -import java.net.{URI, URL, URLConnection} +import java.io.{BufferedInputStream, BufferedOutputStream} +import java.net.{URL, URLConnection, URI} import java.util.concurrent.TimeUnit -import it.unimi.dsi.fastutil.io.{FastBufferedInputStream, FastBufferedOutputStream} - import org.apache.spark.{HttpServer, Logging, SecurityManager, SparkConf, SparkEnv} import org.apache.spark.io.CompressionCodec import org.apache.spark.storage.{BroadcastBlockId, StorageLevel} @@ -164,7 +163,7 @@ private[spark] object HttpBroadcast extends Logging { if (compress) { compressionCodec.compressedOutputStream(new FileOutputStream(file)) } else { - new FastBufferedOutputStream(new FileOutputStream(file), bufferSize) + new BufferedOutputStream(new FileOutputStream(file), bufferSize) } } val ser = SparkEnv.get.serializer.newInstance() @@ -195,7 +194,7 @@ private[spark] object HttpBroadcast extends Logging { if (compress) { compressionCodec.compressedInputStream(inputStream) } else { - new FastBufferedInputStream(inputStream, bufferSize) + new BufferedInputStream(inputStream, bufferSize) } } val ser = SparkEnv.get.serializer.newInstance() diff --git a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala index 40b70baabcad9..8bb78123e3c9c 100644 --- a/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala +++ b/core/src/main/scala/org/apache/spark/partial/GroupedCountEvaluator.scala @@ -22,36 +22,33 @@ import java.util.{HashMap => JHashMap} import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.Map import scala.collection.mutable.HashMap +import scala.reflect.ClassTag import cern.jet.stat.Probability -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} + +import org.apache.spark.util.collection.OpenHashMap /** * An ApproximateEvaluator for counts by key. Returns a map of key to confidence interval. */ -private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Double) - extends ApproximateEvaluator[OLMap[T], Map[T, BoundedDouble]] { +private[spark] class GroupedCountEvaluator[T : ClassTag](totalOutputs: Int, confidence: Double) + extends ApproximateEvaluator[OpenHashMap[T,Long], Map[T, BoundedDouble]] { var outputsMerged = 0 - var sums = new OLMap[T] // Sum of counts for each key + var sums = new OpenHashMap[T,Long]() // Sum of counts for each key - override def merge(outputId: Int, taskResult: OLMap[T]) { + override def merge(outputId: Int, taskResult: OpenHashMap[T,Long]) { outputsMerged += 1 - val iter = taskResult.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - sums.put(entry.getKey, sums.getLong(entry.getKey) + entry.getLongValue) + taskResult.foreach { case (key, value) => + sums.changeValue(key, value, _ + value) } } override def currentResult(): Map[T, BoundedDouble] = { if (outputsMerged == totalOutputs) { val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getLongValue() - result(entry.getKey) = new BoundedDouble(sum, 1.0, sum, sum) + sums.foreach { case (key, sum) => + result(key) = new BoundedDouble(sum, 1.0, sum, sum) } result } else if (outputsMerged == 0) { @@ -60,16 +57,13 @@ private[spark] class GroupedCountEvaluator[T](totalOutputs: Int, confidence: Dou val p = outputsMerged.toDouble / totalOutputs val confFactor = Probability.normalInverse(1 - (1 - confidence) / 2) val result = new JHashMap[T, BoundedDouble](sums.size) - val iter = sums.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - val sum = entry.getLongValue + sums.foreach { case (key, sum) => val mean = (sum + 1 - p) / p val variance = (sum + 1) * (1 - p) / (p * p) val stdev = math.sqrt(variance) val low = mean - confFactor * stdev val high = mean + confFactor * stdev - result(entry.getKey) = new BoundedDouble(mean, confidence, low, high) + result(key) = new BoundedDouble(mean, confidence, low, high) } result } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 3437b2cac19c2..891efccf23b6a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -20,12 +20,10 @@ package org.apache.spark.rdd import java.util.Random import scala.collection.Map -import scala.collection.JavaConversions.mapAsScalaMap import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} import com.clearspring.analytics.stream.cardinality.HyperLogLog -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} import org.apache.hadoop.io.BytesWritable import org.apache.hadoop.io.compress.CompressionCodec import org.apache.hadoop.io.NullWritable @@ -43,6 +41,7 @@ import org.apache.spark.partial.GroupedCountEvaluator import org.apache.spark.partial.PartialResult import org.apache.spark.storage.StorageLevel import org.apache.spark.util.{BoundedPriorityQueue, SerializableHyperLogLog, Utils} +import org.apache.spark.util.collection.OpenHashMap import org.apache.spark.util.random.{BernoulliSampler, PoissonSampler} /** @@ -834,24 +833,24 @@ abstract class RDD[T: ClassTag]( throw new SparkException("countByValue() does not support arrays") } // TODO: This should perhaps be distributed by default. - def countPartition(iter: Iterator[T]): Iterator[OLMap[T]] = { - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) + def countPartition(iter: Iterator[T]): Iterator[OpenHashMap[T,Long]] = { + val map = new OpenHashMap[T,Long] + iter.foreach { + t => map.changeValue(t, 1L, _ + 1L) } Iterator(map) } - def mergeMaps(m1: OLMap[T], m2: OLMap[T]): OLMap[T] = { - val iter = m2.object2LongEntrySet.fastIterator() - while (iter.hasNext) { - val entry = iter.next() - m1.put(entry.getKey, m1.getLong(entry.getKey) + entry.getLongValue) + def mergeMaps(m1: OpenHashMap[T,Long], m2: OpenHashMap[T,Long]): OpenHashMap[T,Long] = { + m2.foreach { case (key, value) => + m1.changeValue(key, value, _ + value) } m1 } val myResult = mapPartitions(countPartition).reduce(mergeMaps) - myResult.asInstanceOf[java.util.Map[T, Long]] // Will be wrapped as a Scala mutable Map + // Convert to a Scala mutable map + val mutableResult = scala.collection.mutable.Map[T,Long]() + myResult.foreach { case (k, v) => mutableResult.put(k, v) } + mutableResult } /** @@ -866,11 +865,10 @@ abstract class RDD[T: ClassTag]( if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } - val countPartition: (TaskContext, Iterator[T]) => OLMap[T] = { (ctx, iter) => - val map = new OLMap[T] - while (iter.hasNext) { - val v = iter.next() - map.put(v, map.getLong(v) + 1L) + val countPartition: (TaskContext, Iterator[T]) => OpenHashMap[T,Long] = { (ctx, iter) => + val map = new OpenHashMap[T,Long] + iter.foreach { + t => map.changeValue(t, 1L, _ + 1L) } map } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index b03665fd56d33..f868e772cf58a 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -17,11 +17,10 @@ package org.apache.spark.scheduler -import java.io.InputStream +import java.io.{BufferedInputStream, InputStream} import scala.io.Source -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.hadoop.fs.{Path, FileSystem} import org.json4s.jackson.JsonMethods._ @@ -62,7 +61,7 @@ private[spark] class ReplayListenerBus( var currentLine = "" try { fileStream = Some(fileSystem.open(path)) - bufferedStream = Some(new FastBufferedInputStream(fileStream.get)) + bufferedStream = Some(new BufferedInputStream(fileStream.get)) compressStream = Some(wrapForCompression(bufferedStream.get)) // Parse each line as an event and post the event to all attached listeners diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index b85b4a50cd93a..a8bcb7dfe2f3c 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -17,13 +17,11 @@ package org.apache.spark.scheduler -import java.io.{DataInputStream, DataOutputStream} +import java.io.{ByteArrayOutputStream, DataInputStream, DataOutputStream} import java.nio.ByteBuffer import scala.collection.mutable.HashMap -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.TaskContext import org.apache.spark.executor.TaskMetrics import org.apache.spark.serializer.SerializerInstance @@ -104,7 +102,7 @@ private[spark] object Task { serializer: SerializerInstance) : ByteBuffer = { - val out = new FastByteArrayOutputStream(4096) + val out = new ByteArrayOutputStream(4096) val dataOut = new DataOutputStream(out) // Write currentFiles @@ -125,8 +123,7 @@ private[spark] object Task { dataOut.flush() val taskBytes = serializer.serialize(task).array() out.write(taskBytes) - out.trim() - ByteBuffer.wrap(out.array) + ByteBuffer.wrap(out.toByteArray) } /** diff --git a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala index 9f04dc6e427c0..f2c8f9b6218d6 100644 --- a/core/src/main/scala/org/apache/spark/serializer/Serializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/Serializer.scala @@ -17,11 +17,9 @@ package org.apache.spark.serializer -import java.io.{EOFException, InputStream, OutputStream} +import java.io.{ByteArrayOutputStream, EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.SparkEnv import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.{ByteBufferInputStream, NextIterator} @@ -73,10 +71,9 @@ trait SerializerInstance { def serializeMany[T](iterator: Iterator[T]): ByteBuffer = { // Default implementation uses serializeStream - val stream = new FastByteArrayOutputStream() + val stream = new ByteArrayOutputStream() serializeStream(stream).writeAll(iterator) - val buffer = ByteBuffer.allocate(stream.position.toInt) - buffer.put(stream.array, 0, stream.position.toInt) + val buffer = ByteBuffer.wrap(stream.toByteArray) buffer.flip() buffer } diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index df9bb4044e37a..f14017051fa07 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -17,7 +17,7 @@ package org.apache.spark.storage -import java.io.{File, InputStream, OutputStream} +import java.io.{File, InputStream, OutputStream, BufferedOutputStream, ByteArrayOutputStream} import java.nio.{ByteBuffer, MappedByteBuffer} import scala.collection.mutable.{ArrayBuffer, HashMap} @@ -26,7 +26,6 @@ import scala.concurrent.duration._ import scala.util.Random import akka.actor.{ActorSystem, Cancellable, Props} -import it.unimi.dsi.fastutil.io.{FastBufferedOutputStream, FastByteArrayOutputStream} import sun.nio.ch.DirectBuffer import org.apache.spark.{Logging, MapOutputTracker, SecurityManager, SparkConf, SparkEnv, SparkException} @@ -992,7 +991,7 @@ private[spark] class BlockManager( outputStream: OutputStream, values: Iterator[Any], serializer: Serializer = defaultSerializer) { - val byteStream = new FastBufferedOutputStream(outputStream) + val byteStream = new BufferedOutputStream(outputStream) val ser = serializer.newInstance() ser.serializeStream(wrapForCompression(blockId, byteStream)).writeAll(values).close() } @@ -1002,10 +1001,9 @@ private[spark] class BlockManager( blockId: BlockId, values: Iterator[Any], serializer: Serializer = defaultSerializer): ByteBuffer = { - val byteStream = new FastByteArrayOutputStream(4096) + val byteStream = new ByteArrayOutputStream(4096) dataSerializeStream(blockId, byteStream, values, serializer) - byteStream.trim() - ByteBuffer.wrap(byteStream.array) + ByteBuffer.wrap(byteStream.toByteArray) } /** diff --git a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala index 696b930a26b9e..a2687e6be4e34 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockObjectWriter.scala @@ -17,11 +17,9 @@ package org.apache.spark.storage -import java.io.{FileOutputStream, File, OutputStream} +import java.io.{BufferedOutputStream, FileOutputStream, File, OutputStream} import java.nio.channels.FileChannel -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream - import org.apache.spark.Logging import org.apache.spark.serializer.{SerializationStream, Serializer} @@ -119,7 +117,7 @@ private[spark] class DiskBlockObjectWriter( ts = new TimeTrackingOutputStream(fos) channel = fos.getChannel() lastValidPosition = initialPosition - bs = compressStream(new FastBufferedOutputStream(ts, bufferSize)) + bs = compressStream(new BufferedOutputStream(ts, bufferSize)) objOut = serializer.newInstance().serializeStream(bs) initialized = true this diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 0080a8b342b05..68a12e8ed67d7 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -17,12 +17,11 @@ package org.apache.spark.util -import java.io._ +import java.io.{FileOutputStream, BufferedOutputStream, PrintWriter, IOException} import java.net.URI import java.text.SimpleDateFormat import java.util.Date -import it.unimi.dsi.fastutil.io.FastBufferedOutputStream import org.apache.hadoop.fs.{FSDataOutputStream, Path} import org.apache.spark.{Logging, SparkConf} @@ -100,7 +99,7 @@ private[spark] class FileLogger( hadoopDataStream.get } - val bstream = new FastBufferedOutputStream(dstream, outputBufferSize) + val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream new PrintWriter(cstream) } diff --git a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala index b955612ca7749..08465575309c6 100644 --- a/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala +++ b/core/src/main/scala/org/apache/spark/util/SizeEstimator.scala @@ -27,9 +27,8 @@ import java.util.concurrent.ConcurrentHashMap import scala.collection.mutable.ArrayBuffer -import it.unimi.dsi.fastutil.ints.IntOpenHashSet - import org.apache.spark.Logging +import org.apache.spark.util.collection.OpenHashSet /** * Estimates the sizes of Java objects (number of bytes of memory they occupy), for use in @@ -207,7 +206,7 @@ private[spark] object SizeEstimator extends Logging { // Estimate the size of a large array by sampling elements without replacement. var size = 0.0 val rand = new Random(42) - val drawn = new IntOpenHashSet(ARRAY_SAMPLE_SIZE) + val drawn = new OpenHashSet[Int](ARRAY_SAMPLE_SIZE) for (i <- 0 until ARRAY_SAMPLE_SIZE) { var index = 0 do { diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index 025492b177a77..ad38250ad339f 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -19,6 +19,8 @@ package org.apache.spark.util.collection import java.util.{Arrays, Comparator} +import com.google.common.hash.Hashing + import org.apache.spark.annotation.DeveloperApi /** @@ -199,11 +201,8 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) /** * Re-hash a value to deal better with hash functions that don't differ in the lower bits. - * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ - private def rehash(h: Int): Int = { - it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) - } + private def rehash(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() /** Double the table's size and re-hash everything */ protected def growTable() { diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index dd01ae821f705..d615767284c0b 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -17,14 +17,13 @@ package org.apache.spark.util.collection -import java.io._ +import java.io.{InputStream, BufferedInputStream, FileInputStream, File, Serializable, EOFException} import java.util.Comparator import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import com.google.common.io.ByteStreams -import it.unimi.dsi.fastutil.io.FastBufferedInputStream import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.annotation.DeveloperApi @@ -350,7 +349,7 @@ class ExternalAppendOnlyMap[K, V, C]( private class DiskMapIterator(file: File, blockId: BlockId, batchSizes: ArrayBuffer[Long]) extends Iterator[(K, C)] { private val fileStream = new FileInputStream(file) - private val bufferedStream = new FastBufferedInputStream(fileStream, fileBufferSize) + private val bufferedStream = new BufferedInputStream(fileStream, fileBufferSize) // An intermediate stream that reads from exactly one batch // This guards against pre-fetching and other arbitrary behavior of higher level streams diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala index 62f99f3981793..b8de4ff9aa494 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashMap.scala @@ -30,7 +30,8 @@ import org.apache.spark.annotation.DeveloperApi * Under the hood, it uses our OpenHashSet implementation. */ @DeveloperApi -class OpenHashMap[K >: Null : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( +private[spark] +class OpenHashMap[K : ClassTag, @specialized(Long, Int, Double) V: ClassTag]( initialCapacity: Int) extends Iterable[(K, V)] with Serializable { diff --git a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala index 148c12e64d2ce..19af4f8cbe428 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/OpenHashSet.scala @@ -18,6 +18,7 @@ package org.apache.spark.util.collection import scala.reflect._ +import com.google.common.hash.Hashing /** * A simple, fast hash set optimized for non-null insertion-only use case, where keys are never @@ -256,9 +257,8 @@ class OpenHashSet[@specialized(Long, Int) T: ClassTag]( /** * Re-hash a value to deal better with hash functions that don't differ in the lower bits. - * We use the Murmur Hash 3 finalization step that's also used in fastutil. */ - private def hashcode(h: Int): Int = it.unimi.dsi.fastutil.HashCommon.murmurHash3(h) + private def hashcode(h: Int): Int = Hashing.murmur3_32().hashInt(h).asInt() private def nextPowerOf2(n: Int): Int = { val highBit = Integer.highestOneBit(n) diff --git a/pom.xml b/pom.xml index c03bb35c99442..5f66cbe768592 100644 --- a/pom.xml +++ b/pom.xml @@ -348,11 +348,6 @@ - - it.unimi.dsi - fastutil - 6.4.4 - colt colt diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 21163760e6277..a6058bba3d211 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -331,7 +331,6 @@ object SparkBuild extends Build { "org.spark-project.akka" %% "akka-slf4j" % akkaVersion excludeAll(excludeNetty), "org.spark-project.akka" %% "akka-testkit" % akkaVersion % "test", "org.json4s" %% "json4s-jackson" % "3.2.6" excludeAll(excludeScalap), - "it.unimi.dsi" % "fastutil" % "6.4.4", "colt" % "colt" % "1.2.0", "org.apache.mesos" % "mesos" % "0.13.0", "commons-net" % "commons-net" % "2.2", diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala index bd1df55cf70f5..bbf57ef9275c0 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextHelper.scala @@ -19,18 +19,17 @@ package org.apache.spark.streaming.util import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ -import it.unimi.dsi.fastutil.objects.{Object2LongOpenHashMap => OLMap} +import org.apache.spark.util.collection.OpenHashMap import scala.collection.JavaConversions.mapAsScalaMap private[streaming] object RawTextHelper { - /** - * Splits lines and counts the words in them using specialized object-to-long hashmap - * (to avoid boxing-unboxing overhead of Long in java/scala HashMap) + /** + * Splits lines and counts the words. */ def splitAndCountPartitions(iter: Iterator[String]): Iterator[(String, Long)] = { - val map = new OLMap[String] + val map = new OpenHashMap[String,Long] var i = 0 var j = 0 while (iter.hasNext) { @@ -43,14 +42,16 @@ object RawTextHelper { } if (j > i) { val w = s.substring(i, j) - val c = map.getLong(w) - map.put(w, c + 1) + map.changeValue(w, 1L, _ + 1L) } i = j while (i < s.length && s.charAt(i) == ' ') { i += 1 } } + map.toIterator.map { + case (k, v) => (k, v) + } } map.toIterator.map{case (k, v) => (k, v)} } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala index 684b38e8b3102..a7850812bd612 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RawTextSender.scala @@ -17,14 +17,12 @@ package org.apache.spark.streaming.util -import java.io.IOException +import java.io.{ByteArrayOutputStream, IOException} import java.net.ServerSocket import java.nio.ByteBuffer import scala.io.Source -import it.unimi.dsi.fastutil.io.FastByteArrayOutputStream - import org.apache.spark.{SparkConf, Logging} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.IntParam @@ -45,16 +43,15 @@ object RawTextSender extends Logging { // Repeat the input data multiple times to fill in a buffer val lines = Source.fromFile(file).getLines().toArray - val bufferStream = new FastByteArrayOutputStream(blockSize + 1000) + val bufferStream = new ByteArrayOutputStream(blockSize + 1000) val ser = new KryoSerializer(new SparkConf()).newInstance() val serStream = ser.serializeStream(bufferStream) var i = 0 - while (bufferStream.position < blockSize) { + while (bufferStream.size < blockSize) { serStream.writeObject(lines(i)) i = (i + 1) % lines.length } - bufferStream.trim() - val array = bufferStream.array + val array = bufferStream.toByteArray val countBuf = ByteBuffer.wrap(new Array[Byte](4)) countBuf.putInt(array.length) From 6aa08c39cf30fa5c4ed97f4fff16371b9030a2e6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Fri, 11 Apr 2014 23:33:49 -0700 Subject: [PATCH 268/397] [SPARK-1386] Web UI for Spark Streaming When debugging Spark Streaming applications it is necessary to monitor certain metrics that are not shown in the Spark application UI. For example, what is average processing time of batches? What is the scheduling delay? Is the system able to process as fast as it is receiving data? How many records I am receiving through my receivers? While the StreamingListener interface introduced in the 0.9 provided some of this information, it could only be accessed programmatically. A UI that shows information specific to the streaming applications is necessary for easier debugging. This PR introduces such a UI. It shows various statistics related to the streaming application. Here is a screenshot of the UI running on my local machine. http://i.imgur.com/1ooDGhm.png This UI is integrated into the Spark UI running at 4040. Author: Tathagata Das Author: Andrew Or Closes #290 from tdas/streaming-web-ui and squashes the following commits: fc73ca5 [Tathagata Das] Merge pull request #9 from andrewor14/ui-refactor 642dd88 [Andrew Or] Merge SparkUISuite.scala into UISuite.scala eb30517 [Andrew Or] Merge github.com:apache/spark into ui-refactor f4f4cbe [Tathagata Das] More minor fixes. 34bb364 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 252c566 [Tathagata Das] Merge pull request #8 from andrewor14/ui-refactor e038b4b [Tathagata Das] Addressed Patrick's comments. 125a054 [Andrew Or] Disable serving static resources with gzip 90feb8d [Andrew Or] Address Patrick's comments 89dae36 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 72fe256 [Tathagata Das] Merge pull request #6 from andrewor14/ui-refactor 2fc09c8 [Tathagata Das] Added binary check exclusions aa396d4 [Andrew Or] Rename tabs and pages (No more IndexPage.scala) f8e1053 [Tathagata Das] Added Spark and Streaming UI unit tests. caa5e05 [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 585cd65 [Tathagata Das] Merge pull request #5 from andrewor14/ui-refactor 914b8ff [Tathagata Das] Moved utils functions to UIUtils. 548c98c [Andrew Or] Wide refactoring of WebUI, UITab, and UIPage (see commit message) 6de06b0 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui ee6543f [Tathagata Das] Minor changes based on Andrew's comments. fa760fe [Tathagata Das] Fixed long line. 1c0bcef [Tathagata Das] Refactored streaming UI into two files. 1af239b [Tathagata Das] Changed streaming UI to attach itself as a tab with the Spark UI. 827e81a [Tathagata Das] Merge branch 'streaming-web-ui' of github.com:tdas/spark into streaming-web-ui 168fe86 [Tathagata Das] Merge pull request #2 from andrewor14/ui-refactor 3e986f8 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-web-ui c78c92d [Andrew Or] Remove outdated comment 8f7323b [Andrew Or] End of file new lines, indentation, and imports (minor) 0d61ee8 [Andrew Or] Merge branch 'streaming-web-ui' of github.com:tdas/spark into ui-refactor 9a48fa1 [Andrew Or] Allow adding tabs to SparkUI dynamically + add example 61358e3 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-web-ui 53be2c5 [Tathagata Das] Minor style updates. ed25dfc [Andrew Or] Generalize SparkUI header to display tabs dynamically a37ad4f [Andrew Or] Comments, imports and formatting (minor) cd000b0 [Andrew Or] Merge github.com:apache/spark into ui-refactor 7d57444 [Andrew Or] Refactoring the UI interface to add flexibility aef4dd5 [Tathagata Das] Added Apache licenses. db27bad [Tathagata Das] Added last batch processing time to StreamingUI. 4d86e98 [Tathagata Das] Added basic stats to the StreamingUI and refactored the UI to a Page to make it easier to transition to using SparkUI later. 93f1c69 [Tathagata Das] Added network receiver information to the Streaming UI. 56cc7fb [Tathagata Das] First cut implementation of Streaming UI. --- .../scala/org/apache/spark/SparkContext.scala | 1 - .../spark/deploy/SparkUIContainer.scala | 50 ----- .../{IndexPage.scala => HistoryPage.scala} | 12 +- .../spark/deploy/history/HistoryServer.scala | 61 +++--- .../apache/spark/deploy/master/Master.scala | 8 +- .../deploy/master/ui/ApplicationPage.scala | 13 +- .../ui/{IndexPage.scala => MasterPage.scala} | 23 ++- .../spark/deploy/master/ui/MasterWebUI.scala | 54 ++---- .../apache/spark/deploy/worker/Worker.scala | 2 +- .../spark/deploy/worker/ui/LogPage.scala | 147 ++++++++++++++ .../ui/{IndexPage.scala => WorkerPage.scala} | 6 +- .../spark/deploy/worker/ui/WorkerWebUI.scala | 180 +++--------------- .../scheduler/ApplicationEventListener.scala | 4 +- .../apache/spark/storage/StorageUtils.scala | 16 +- .../org/apache/spark/ui/JettyUtils.scala | 1 + .../main/scala/org/apache/spark/ui/Page.scala | 22 --- .../scala/org/apache/spark/ui/SparkUI.scala | 108 ++++------- .../scala/org/apache/spark/ui/UIUtils.scala | 172 +++++++++++++---- .../scala/org/apache/spark/ui/WebUI.scala | 141 +++++++++++--- ...ironmentUI.scala => EnvironmentPage.scala} | 47 +---- .../apache/spark/ui/env/EnvironmentTab.scala | 50 +++++ ...{ExecutorsUI.scala => ExecutorsPage.scala} | 84 +------- .../apache/spark/ui/exec/ExecutorsTab.scala | 86 +++++++++ .../apache/spark/ui/jobs/ExecutorTable.scala | 7 +- .../spark/ui/jobs/JobProgressListener.scala | 10 +- ...{IndexPage.scala => JobProgressPage.scala} | 16 +- ...bProgressUI.scala => JobProgressTab.scala} | 45 ++--- .../org/apache/spark/ui/jobs/PoolPage.scala | 14 +- .../org/apache/spark/ui/jobs/PoolTable.scala | 7 +- .../org/apache/spark/ui/jobs/StagePage.scala | 45 ++--- .../org/apache/spark/ui/jobs/StageTable.scala | 18 +- .../org/apache/spark/ui/storage/RDDPage.scala | 17 +- .../{IndexPage.scala => StoragePage.scala} | 13 +- ...{BlockManagerUI.scala => StorageTab.scala} | 32 +--- .../org/apache/spark/util/JsonProtocol.scala | 12 +- .../scala/org/apache/spark/ui/UISuite.scala | 81 +++++++- .../apache/spark/util/JsonProtocolSuite.scala | 4 +- project/MimaBuild.scala | 8 +- .../spark/streaming/StreamingContext.scala | 23 +-- .../spark/streaming/dstream/DStream.scala | 9 - .../dstream/NetworkInputDStream.scala | 79 +++++--- .../spark/streaming/scheduler/BatchInfo.scala | 1 + .../streaming/scheduler/JobGenerator.scala | 9 +- .../streaming/scheduler/JobScheduler.scala | 11 +- .../spark/streaming/scheduler/JobSet.scala | 7 +- .../scheduler/NetworkInputTracker.scala | 86 ++++++--- .../scheduler/StreamingListener.scala | 18 +- .../scheduler/StreamingListenerBus.scala | 4 + .../ui/StreamingJobProgressListener.scala | 148 ++++++++++++++ .../spark/streaming/ui/StreamingPage.scala | 180 ++++++++++++++++++ .../spark/streaming/ui/StreamingTab.scala | 27 +-- .../spark/streaming/InputStreamsSuite.scala | 6 +- .../streaming/StreamingContextSuite.scala | 1 - .../org/apache/spark/streaming/UISuite.scala | 46 +++++ 54 files changed, 1426 insertions(+), 846 deletions(-) delete mode 100644 core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala rename core/src/main/scala/org/apache/spark/deploy/history/{IndexPage.scala => HistoryPage.scala} (85%) rename core/src/main/scala/org/apache/spark/deploy/master/ui/{IndexPage.scala => MasterPage.scala} (91%) create mode 100644 core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala rename core/src/main/scala/org/apache/spark/deploy/worker/ui/{IndexPage.scala => WorkerPage.scala} (97%) delete mode 100644 core/src/main/scala/org/apache/spark/ui/Page.scala rename core/src/main/scala/org/apache/spark/ui/env/{EnvironmentUI.scala => EnvironmentPage.scala} (61%) create mode 100644 core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala rename core/src/main/scala/org/apache/spark/ui/exec/{ExecutorsUI.scala => ExecutorsPage.scala} (61%) create mode 100644 core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala rename core/src/main/scala/org/apache/spark/ui/jobs/{IndexPage.scala => JobProgressPage.scala} (90%) rename core/src/main/scala/org/apache/spark/ui/jobs/{JobProgressUI.scala => JobProgressTab.scala} (53%) rename core/src/main/scala/org/apache/spark/ui/storage/{IndexPage.scala => StoragePage.scala} (90%) rename core/src/main/scala/org/apache/spark/ui/storage/{BlockManagerUI.scala => StorageTab.scala} (75%) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala rename core/src/test/scala/org/apache/spark/SparkUISuite.scala => streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala (58%) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 3bcc8ce2b25a6..a764c174d562c 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -213,7 +213,6 @@ class SparkContext(config: SparkConf) extends Logging { // Initialize the Spark UI, registering all associated listeners private[spark] val ui = new SparkUI(this) ui.bind() - ui.start() // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala b/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala deleted file mode 100644 index 33fceae4ff489..0000000000000 --- a/core/src/main/scala/org/apache/spark/deploy/SparkUIContainer.scala +++ /dev/null @@ -1,50 +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.deploy - -import org.apache.spark.ui.{SparkUI, WebUI} - -private[spark] abstract class SparkUIContainer(name: String) extends WebUI(name) { - - /** Attach a SparkUI to this container. Only valid after bind(). */ - def attachUI(ui: SparkUI) { - assert(serverInfo.isDefined, - "%s must be bound to a server before attaching SparkUIs".format(name)) - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - rootHandler.addHandler(handler) - if (!handler.isStarted) { - handler.start() - } - } - } - - /** Detach a SparkUI from this container. Only valid after bind(). */ - def detachUI(ui: SparkUI) { - assert(serverInfo.isDefined, - "%s must be bound to a server before detaching SparkUIs".format(name)) - val rootHandler = serverInfo.get.rootHandler - for (handler <- ui.handlers) { - if (handler.isStarted) { - handler.stop() - } - rootHandler.removeHandler(handler) - } - } - -} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala similarity index 85% rename from core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala index 54dffffec71c5..180c853ce3096 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryPage.scala @@ -21,9 +21,9 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.{UIUtils, WebUI} +import org.apache.spark.ui.{WebUIPage, UIUtils} -private[spark] class IndexPage(parent: HistoryServer) { +private[spark] class HistoryPage(parent: HistoryServer) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val appRows = parent.appIdToInfo.values.toSeq.sortBy { app => -app.lastUpdated } @@ -62,13 +62,13 @@ private[spark] class IndexPage(parent: HistoryServer) { private def appRow(info: ApplicationHistoryInfo): Seq[Node] = { val appName = if (info.started) info.name else info.logDirPath.getName val uiAddress = parent.getAddress + info.ui.basePath - val startTime = if (info.started) WebUI.formatDate(info.startTime) else "Not started" - val endTime = if (info.completed) WebUI.formatDate(info.endTime) else "Not completed" + val startTime = if (info.started) UIUtils.formatDate(info.startTime) else "Not started" + val endTime = if (info.completed) UIUtils.formatDate(info.endTime) else "Not completed" val difference = if (info.started && info.completed) info.endTime - info.startTime else -1L - val duration = if (difference > 0) WebUI.formatDuration(difference) else "---" + val duration = if (difference > 0) UIUtils.formatDuration(difference) else "---" val sparkUser = if (info.started) info.sparkUser else "Unknown user" val logDirectory = info.logDirPath.getName - val lastUpdated = WebUI.formatDate(info.lastUpdated) + val lastUpdated = UIUtils.formatDate(info.lastUpdated) {appName} {startTime} diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index 97d2ba9deed33..cf64700f9098c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -17,17 +17,13 @@ package org.apache.spark.deploy.history -import javax.servlet.http.HttpServletRequest - import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} -import org.eclipse.jetty.servlet.ServletContextHandler import org.apache.spark.{Logging, SecurityManager, SparkConf} -import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.scheduler._ -import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.{WebUI, SparkUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.Utils @@ -46,17 +42,15 @@ import org.apache.spark.util.Utils */ class HistoryServer( val baseLogDir: String, + securityManager: SecurityManager, conf: SparkConf) - extends SparkUIContainer("History Server") with Logging { + extends WebUI(securityManager, HistoryServer.WEB_UI_PORT, conf) with Logging { import HistoryServer._ private val fileSystem = Utils.getHadoopFileSystem(baseLogDir) private val localHost = Utils.localHostName() private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) - private val port = WEB_UI_PORT - private val securityManager = new SecurityManager(conf) - private val indexPage = new IndexPage(this) // A timestamp of when the disk was last accessed to check for log updates private var lastLogCheckTime = -1L @@ -90,37 +84,23 @@ class HistoryServer( } } - private val handlers = Seq[ServletContextHandler]( - createStaticHandler(STATIC_RESOURCE_DIR, "/static"), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), securityMgr = securityManager) - ) - // A mapping of application ID to its history information, which includes the rendered UI val appIdToInfo = mutable.HashMap[String, ApplicationHistoryInfo]() + initialize() + /** - * Start the history server. + * Initialize the history server. * * This starts a background thread that periodically synchronizes information displayed on * this UI with the event logs in the provided base directory. */ - def start() { + def initialize() { + attachPage(new HistoryPage(this)) + attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) logCheckingThread.start() } - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) - logInfo("Started HistoryServer at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to bind HistoryServer", e) - System.exit(1) - } - } - /** * Check for any updates to event logs in the base directory. This is only effective once * the server has been bound. @@ -151,7 +131,7 @@ class HistoryServer( // Remove any applications that should no longer be retained appIdToInfo.foreach { case (appId, info) => if (!retainedAppIds.contains(appId)) { - detachUI(info.ui) + detachSparkUI(info.ui) appIdToInfo.remove(appId) } } @@ -186,15 +166,14 @@ class HistoryServer( val path = logDir.getPath val appId = path.getName val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) - val ui = new SparkUI(replayBus, appId, "/history/" + appId) val appListener = new ApplicationEventListener replayBus.addListener(appListener) + val ui = new SparkUI(conf, replayBus, appId, "/history/" + appId) // Do not call ui.bind() to avoid creating a new server for each application - ui.start() replayBus.replay() if (appListener.applicationStarted) { - attachUI(ui) + attachSparkUI(ui) val appName = appListener.appName val sparkUser = appListener.sparkUser val startTime = appListener.startTime @@ -213,6 +192,18 @@ class HistoryServer( fileSystem.close() } + /** Attach a reconstructed UI to this server. Only valid after bind(). */ + private def attachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") + ui.getHandlers.foreach(attachHandler) + } + + /** Detach a reconstructed UI from this server. Only valid after bind(). */ + private def detachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "HistoryServer must be bound before detaching SparkUIs") + ui.getHandlers.foreach(detachHandler) + } + /** Return the address of this server. */ def getAddress: String = "http://" + publicHost + ":" + boundPort @@ -262,9 +253,9 @@ object HistoryServer { def main(argStrings: Array[String]) { val args = new HistoryServerArguments(argStrings) - val server = new HistoryServer(args.logDir, conf) + val securityManager = new SecurityManager(conf) + val server = new HistoryServer(args.logDir, securityManager, conf) server.bind() - server.start() // Wait until the end of the world... or if the HistoryServer process is manually stopped while(true) { Thread.sleep(Int.MaxValue) } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 2446e86cb6672..6c58e741df001 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -625,7 +625,7 @@ private[spark] class Master( if (completedApps.size >= RETAINED_APPLICATIONS) { val toRemove = math.max(RETAINED_APPLICATIONS / 10, 1) completedApps.take(toRemove).foreach( a => { - appIdToUI.remove(a.id).foreach { ui => webUi.detachUI(ui) } + appIdToUI.remove(a.id).foreach { ui => webUi.detachSparkUI(ui) } applicationMetricsSystem.removeSource(a.appSource) }) completedApps.trimStart(toRemove) @@ -667,12 +667,12 @@ private[spark] class Master( if (!eventLogPaths.isEmpty) { try { val replayBus = new ReplayListenerBus(eventLogPaths, fileSystem, compressionCodec) - val ui = new SparkUI(replayBus, appName + " (completed)", "/history/" + app.id) - ui.start() + val ui = new SparkUI( + new SparkConf, replayBus, appName + " (completed)", "/history/" + app.id) replayBus.replay() app.desc.appUiUrl = ui.basePath appIdToUI(app.id) = ui - webUi.attachUI(ui) + webUi.attachSparkUI(ui) return true } catch { case t: Throwable => diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala index cb092cb5d576b..b5cd4d2ea963f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/ApplicationPage.scala @@ -28,15 +28,16 @@ import org.json4s.JValue import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.ExecutorInfo -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class ApplicationPage(parent: MasterWebUI) { - val master = parent.masterActorRef - val timeout = parent.timeout +private[spark] class ApplicationPage(parent: MasterWebUI) extends WebUIPage("app") { + + private val master = parent.masterActorRef + private val timeout = parent.timeout /** Executor details for a particular application */ - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val appId = request.getParameter("appId") val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) @@ -96,7 +97,7 @@ private[spark] class ApplicationPage(parent: MasterWebUI) { UIUtils.basicSparkPage(content, "Application: " + app.desc.name) } - def executorRow(executor: ExecutorInfo): Seq[Node] = { + private def executorRow(executor: ExecutorInfo): Seq[Node] = { {executor.id} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala similarity index 91% rename from core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala index 8c1d6c7cce450..7ca3b08a28728 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterPage.scala @@ -25,17 +25,17 @@ import scala.xml.Node import akka.pattern.ask import org.json4s.JValue -import org.apache.spark.deploy.{JsonProtocol} +import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{MasterStateResponse, RequestMasterState} import org.apache.spark.deploy.master.{ApplicationInfo, DriverInfo, WorkerInfo} -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: MasterWebUI) { - val master = parent.masterActorRef - val timeout = parent.timeout +private[spark] class MasterPage(parent: MasterWebUI) extends WebUIPage("") { + private val master = parent.masterActorRef + private val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (master ? RequestMasterState)(timeout).mapTo[MasterStateResponse] val state = Await.result(stateFuture, timeout) JsonProtocol.writeMasterState(state) @@ -139,7 +139,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { UIUtils.basicSparkPage(content, "Spark Master at " + state.uri) } - def workerRow(worker: WorkerInfo): Seq[Node] = { + private def workerRow(worker: WorkerInfo): Seq[Node] = { {worker.id} @@ -154,8 +154,7 @@ private[spark] class IndexPage(parent: MasterWebUI) { } - - def appRow(app: ApplicationInfo): Seq[Node] = { + private def appRow(app: ApplicationInfo): Seq[Node] = { {app.id} @@ -169,14 +168,14 @@ private[spark] class IndexPage(parent: MasterWebUI) { {Utils.megabytesToString(app.desc.memoryPerSlave)} - {WebUI.formatDate(app.submitDate)} + {UIUtils.formatDate(app.submitDate)} {app.desc.user} {app.state.toString} - {WebUI.formatDuration(app.duration)} + {UIUtils.formatDuration(app.duration)} } - def driverRow(driver: DriverInfo): Seq[Node] = { + private def driverRow(driver: DriverInfo): Seq[Node] = { {driver.id} {driver.submitDate} diff --git a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala index 30c8ade408a5a..a18b39fc95d64 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/ui/MasterWebUI.scala @@ -17,14 +17,9 @@ package org.apache.spark.deploy.master.ui -import javax.servlet.http.HttpServletRequest - -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.Logging -import org.apache.spark.deploy.SparkUIContainer import org.apache.spark.deploy.master.Master -import org.apache.spark.ui.SparkUI +import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ import org.apache.spark.util.{AkkaUtils, Utils} @@ -33,44 +28,33 @@ import org.apache.spark.util.{AkkaUtils, Utils} */ private[spark] class MasterWebUI(val master: Master, requestedPort: Int) - extends SparkUIContainer("MasterWebUI") with Logging { + extends WebUI(master.securityMgr, requestedPort, master.conf) with Logging { val masterActorRef = master.self val timeout = AkkaUtils.askTimeout(master.conf) - private val host = Utils.localHostName() - private val port = requestedPort - private val applicationPage = new ApplicationPage(this) - private val indexPage = new IndexPage(this) + initialize() - private val handlers: Seq[ServletContextHandler] = { - master.masterMetricsSystem.getServletHandlers ++ - master.applicationMetricsSystem.getServletHandlers ++ - Seq[ServletContextHandler]( - createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static"), - createServletHandler("/app/json", - (request: HttpServletRequest) => applicationPage.renderJson(request), master.securityMgr), - createServletHandler("/app", - (request: HttpServletRequest) => applicationPage.render(request), master.securityMgr), - createServletHandler("/json", - (request: HttpServletRequest) => indexPage.renderJson(request), master.securityMgr), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), master.securityMgr) - ) + /** Initialize all components of the server. */ + def initialize() { + attachPage(new ApplicationPage(this)) + attachPage(new MasterPage(this)) + attachHandler(createStaticHandler(MasterWebUI.STATIC_RESOURCE_DIR, "/static")) + master.masterMetricsSystem.getServletHandlers.foreach(attachHandler) + master.applicationMetricsSystem.getServletHandlers.foreach(attachHandler) } - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, master.conf)) - logInfo("Started Master web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Master web UI", e) - System.exit(1) - } + /** Attach a reconstructed UI to this Master UI. Only valid after bind(). */ + def attachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before attaching SparkUIs") + ui.getHandlers.foreach(attachHandler) } + /** Detach a reconstructed UI from this Master UI. Only valid after bind(). */ + def detachSparkUI(ui: SparkUI) { + assert(serverInfo.isDefined, "Master UI must be bound to a server before detaching SparkUIs") + ui.getHandlers.foreach(detachHandler) + } } private[spark] object MasterWebUI { diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index bf5a8d09dd2df..52c164ca3c574 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -128,8 +128,8 @@ private[spark] class Worker( host, port, cores, Utils.megabytesToString(memory))) logInfo("Spark home: " + sparkHome) createWorkDir() - webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) context.system.eventStream.subscribe(self, classOf[RemotingLifecycleEvent]) + webUi = new WorkerWebUI(this, workDir, Some(webUiPort)) webUi.bind() registerWithMaster() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala new file mode 100644 index 0000000000000..fec1207948628 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -0,0 +1,147 @@ +/* + * 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.deploy.worker.ui + +import java.io.File +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.ui.{WebUIPage, UIUtils} +import org.apache.spark.util.Utils + +private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { + private val worker = parent.worker + private val workDir = parent.workDir + + def renderLog(request: HttpServletRequest): String = { + val defaultBytes = 100 * 1024 + + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + + val path = (appId, executorId, driverId) match { + case (Some(a), Some(e), None) => + s"${workDir.getPath}/$appId/$executorId/$logType" + case (None, None, Some(d)) => + s"${workDir.getPath}/$driverId/$logType" + case _ => + throw new Exception("Request must specify either application or driver identifiers") + } + + val (startByte, endByte) = getByteRange(path, offset, byteLength) + val file = new File(path) + val logLength = file.length + + val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n" + pre + Utils.offsetBytes(path, startByte, endByte) + } + + def render(request: HttpServletRequest): Seq[Node] = { + val defaultBytes = 100 * 1024 + val appId = Option(request.getParameter("appId")) + val executorId = Option(request.getParameter("executorId")) + val driverId = Option(request.getParameter("driverId")) + val logType = request.getParameter("logType") + val offset = Option(request.getParameter("offset")).map(_.toLong) + val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) + + val (path, params) = (appId, executorId, driverId) match { + case (Some(a), Some(e), None) => + (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e") + case (None, None, Some(d)) => + (s"${workDir.getPath}/$d/$logType", s"driverId=$d") + case _ => + throw new Exception("Request must specify either application or driver identifiers") + } + + val (startByte, endByte) = getByteRange(path, offset, byteLength) + val file = new File(path) + val logLength = file.length + val logText = {Utils.offsetBytes(path, startByte, endByte)} + val linkToMaster =

    Back to Master

    + val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} + + val backButton = + if (startByte > 0) { + + + + } + else { + + } + + val nextButton = + if (endByte < logLength) { + + + + } + else { + + } + + val content = + + + {linkToMaster} +
    +
    {backButton}
    +
    {range}
    +
    {nextButton}
    +
    +
    +
    +
    {logText}
    +
    + + + UIUtils.basicSparkPage(content, logType + " log page for " + appId) + } + + /** Determine the byte range for a log or log page. */ + private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { + val defaultBytes = 100 * 1024 + val maxBytes = 1024 * 1024 + val file = new File(path) + val logLength = file.length() + val getOffset = offset.getOrElse(logLength - defaultBytes) + val startByte = + if (getOffset < 0) 0L + else if (getOffset > logLength) logLength + else getOffset + val logPageLength = math.min(byteLength, maxBytes) + val endByte = math.min(startByte + logPageLength, logLength) + (startByte, endByte) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala similarity index 97% rename from core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala rename to core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala index 49c1009cac2bf..d4513118ced05 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerPage.scala @@ -28,15 +28,15 @@ import org.apache.spark.deploy.JsonProtocol import org.apache.spark.deploy.DeployMessages.{RequestWorkerState, WorkerStateResponse} import org.apache.spark.deploy.master.DriverState import org.apache.spark.deploy.worker.{DriverRunner, ExecutorRunner} -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[spark] class IndexPage(parent: WorkerWebUI) { +private[spark] class WorkerPage(parent: WorkerWebUI) extends WebUIPage("") { val workerActor = parent.worker.self val worker = parent.worker val timeout = parent.timeout - def renderJson(request: HttpServletRequest): JValue = { + override def renderJson(request: HttpServletRequest): JValue = { val stateFuture = (workerActor ? RequestWorkerState)(timeout).mapTo[WorkerStateResponse] val workerState = Await.result(stateFuture, timeout) JsonProtocol.writeWorkerState(workerState) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala index 5625a44549aaa..0ad2edba2227f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/WorkerWebUI.scala @@ -20,174 +20,44 @@ package org.apache.spark.deploy.worker.ui import java.io.File import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkConf} import org.apache.spark.deploy.worker.Worker -import org.apache.spark.ui.{SparkUI, UIUtils, WebUI} +import org.apache.spark.ui.{SparkUI, WebUI} import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.util.{AkkaUtils, Utils} +import org.apache.spark.util.AkkaUtils /** * Web UI server for the standalone worker. */ private[spark] -class WorkerWebUI(val worker: Worker, val workDir: File, requestedPort: Option[Int] = None) - extends WebUI("WorkerWebUI") with Logging { +class WorkerWebUI( + val worker: Worker, + val workDir: File, + port: Option[Int] = None) + extends WebUI(worker.securityMgr, WorkerWebUI.getUIPort(port, worker.conf), worker.conf) + with Logging { val timeout = AkkaUtils.askTimeout(worker.conf) - private val host = Utils.localHostName() - private val port = requestedPort.getOrElse( - worker.conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) - private val indexPage = new IndexPage(this) - - private val handlers: Seq[ServletContextHandler] = { - worker.metricsSystem.getServletHandlers ++ - Seq[ServletContextHandler]( - createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static"), - createServletHandler("/log", - (request: HttpServletRequest) => log(request), worker.securityMgr), - createServletHandler("/logPage", - (request: HttpServletRequest) => logPage(request), worker.securityMgr), - createServletHandler("/json", - (request: HttpServletRequest) => indexPage.renderJson(request), worker.securityMgr), - createServletHandler("/", - (request: HttpServletRequest) => indexPage.render(request), worker.securityMgr) - ) - } - - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, worker.conf)) - logInfo("Started Worker web UI at http://%s:%d".format(host, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Worker web UI", e) - System.exit(1) - } - } - - private def log(request: HttpServletRequest): String = { - val defaultBytes = 100 * 1024 - - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - - val path = (appId, executorId, driverId) match { - case (Some(a), Some(e), None) => - s"${workDir.getPath}/$appId/$executorId/$logType" - case (None, None, Some(d)) => - s"${workDir.getPath}/$driverId/$logType" - case _ => - throw new Exception("Request must specify either application or driver identifiers") - } - - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - - val pre = s"==== Bytes $startByte-$endByte of $logLength of $path ====\n" - pre + Utils.offsetBytes(path, startByte, endByte) - } - - private def logPage(request: HttpServletRequest): Seq[scala.xml.Node] = { - val defaultBytes = 100 * 1024 - val appId = Option(request.getParameter("appId")) - val executorId = Option(request.getParameter("executorId")) - val driverId = Option(request.getParameter("driverId")) - val logType = request.getParameter("logType") - val offset = Option(request.getParameter("offset")).map(_.toLong) - val byteLength = Option(request.getParameter("byteLength")).map(_.toInt).getOrElse(defaultBytes) - - val (path, params) = (appId, executorId, driverId) match { - case (Some(a), Some(e), None) => - (s"${workDir.getPath}/$a/$e/$logType", s"appId=$a&executorId=$e") - case (None, None, Some(d)) => - (s"${workDir.getPath}/$d/$logType", s"driverId=$d") - case _ => - throw new Exception("Request must specify either application or driver identifiers") - } - - val (startByte, endByte) = getByteRange(path, offset, byteLength) - val file = new File(path) - val logLength = file.length - val logText = {Utils.offsetBytes(path, startByte, endByte)} - val linkToMaster =

    Back to Master

    - val range = Bytes {startByte.toString} - {endByte.toString} of {logLength} - - val backButton = - if (startByte > 0) { - - - - } - else { - - } - - val nextButton = - if (endByte < logLength) { - - - - } - else { - - } - - val content = - - - {linkToMaster} -
    -
    {backButton}
    -
    {range}
    -
    {nextButton}
    -
    -
    -
    -
    {logText}
    -
    - - - UIUtils.basicSparkPage(content, logType + " log page for " + appId) + initialize() + + /** Initialize all components of the server. */ + def initialize() { + val logPage = new LogPage(this) + attachPage(logPage) + attachPage(new WorkerPage(this)) + attachHandler(createStaticHandler(WorkerWebUI.STATIC_RESOURCE_BASE, "/static")) + attachHandler(createServletHandler("/log", + (request: HttpServletRequest) => logPage.renderLog(request), worker.securityMgr)) + worker.metricsSystem.getServletHandlers.foreach(attachHandler) } - - /** Determine the byte range for a log or log page. */ - private def getByteRange(path: String, offset: Option[Long], byteLength: Int): (Long, Long) = { - val defaultBytes = 100 * 1024 - val maxBytes = 1024 * 1024 - val file = new File(path) - val logLength = file.length() - val getOffset = offset.getOrElse(logLength - defaultBytes) - val startByte = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset - val logPageLength = math.min(byteLength, maxBytes) - val endByte = math.min(startByte + logPageLength, logLength) - (startByte, endByte) - } - } private[spark] object WorkerWebUI { - val DEFAULT_PORT=8081 + val DEFAULT_PORT = 8081 val STATIC_RESOURCE_BASE = SparkUI.STATIC_RESOURCE_DIR + + def getUIPort(requestedPort: Option[Int], conf: SparkConf): Int = { + requestedPort.getOrElse(conf.getInt("worker.ui.port", WorkerWebUI.DEFAULT_PORT)) + } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index affda13df6531..c1001227151a5 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -31,11 +31,11 @@ private[spark] class ApplicationEventListener extends SparkListener { def applicationStarted = startTime != -1 - def applicationFinished = endTime != -1 + def applicationCompleted = endTime != -1 def applicationDuration: Long = { val difference = endTime - startTime - if (applicationStarted && applicationFinished && difference > 0) difference else -1L + if (applicationStarted && applicationCompleted && difference > 0) difference else -1L } override def onApplicationStart(applicationStart: SparkListenerApplicationStart) { diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 07255aa366a6d..7ed371326855d 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -42,24 +42,22 @@ class StorageStatus( def memRemaining : Long = maxMem - memUsed() - def rddBlocks = blocks.flatMap { - case (rdd: RDDBlockId, status) => Some(rdd, status) - case _ => None - } + def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } } @DeveloperApi private[spark] class RDDInfo( - val id: Int, - val name: String, - val numPartitions: Int, - val storageLevel: StorageLevel) extends Ordered[RDDInfo] { + val id: Int, + val name: String, + val numPartitions: Int, + val storageLevel: StorageLevel) + extends Ordered[RDDInfo] { var numCachedPartitions = 0 var memSize = 0L var diskSize = 0L - var tachyonSize= 0L + var tachyonSize = 0L override def toString = { import Utils.bytesToString diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index dd0818e8ab01c..62a4e3d0f6a42 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -121,6 +121,7 @@ private[spark] object JettyUtils extends Logging { /** Create a handler for serving files from a static directory */ def createStaticHandler(resourceBase: String, path: String): ServletContextHandler = { val contextHandler = new ServletContextHandler + contextHandler.setInitParameter("org.eclipse.jetty.servlet.Default.gzip", "false") val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) Option(getClass.getClassLoader.getResource(resourceBase)) match { diff --git a/core/src/main/scala/org/apache/spark/ui/Page.scala b/core/src/main/scala/org/apache/spark/ui/Page.scala deleted file mode 100644 index b2a069a37552d..0000000000000 --- a/core/src/main/scala/org/apache/spark/ui/Page.scala +++ /dev/null @@ -1,22 +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.ui - -private[spark] object Page extends Enumeration { - val Stages, Storage, Environment, Executors = Value -} diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 7fa4fd3149eb6..2fef1a635427c 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -17,112 +17,86 @@ package org.apache.spark.ui -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext, SparkEnv} +import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.scheduler._ import org.apache.spark.storage.StorageStatusListener import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.env.EnvironmentUI -import org.apache.spark.ui.exec.ExecutorsUI -import org.apache.spark.ui.jobs.JobProgressUI -import org.apache.spark.ui.storage.BlockManagerUI -import org.apache.spark.util.Utils +import org.apache.spark.ui.env.EnvironmentTab +import org.apache.spark.ui.exec.ExecutorsTab +import org.apache.spark.ui.jobs.JobProgressTab +import org.apache.spark.ui.storage.StorageTab -/** Top level user interface for Spark */ +/** + * Top level user interface for a Spark application. + */ private[spark] class SparkUI( val sc: SparkContext, val conf: SparkConf, + val securityManager: SecurityManager, val listenerBus: SparkListenerBus, var appName: String, val basePath: String = "") - extends WebUI("SparkUI") with Logging { + extends WebUI(securityManager, SparkUI.getUIPort(conf), conf, basePath) + with Logging { - def this(sc: SparkContext) = this(sc, sc.conf, sc.listenerBus, sc.appName) - def this(listenerBus: SparkListenerBus, appName: String, basePath: String) = - this(null, new SparkConf, listenerBus, appName, basePath) + def this(sc: SparkContext) = this(sc, sc.conf, sc.env.securityManager, sc.listenerBus, sc.appName) + def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = + this(null, conf, new SecurityManager(conf), listenerBus, appName, basePath) // If SparkContext is not provided, assume the associated application is not live val live = sc != null - val securityManager = if (live) sc.env.securityManager else new SecurityManager(conf) - - private val localHost = Utils.localHostName() - private val publicHost = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHost) - private val port = conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + // Maintain executor storage status through Spark events + val storageStatusListener = new StorageStatusListener - private val storage = new BlockManagerUI(this) - private val jobs = new JobProgressUI(this) - private val env = new EnvironmentUI(this) - private val exec = new ExecutorsUI(this) + initialize() - val handlers: Seq[ServletContextHandler] = { - val metricsServletHandlers = if (live) { - SparkEnv.get.metricsSystem.getServletHandlers - } else { - Array[ServletContextHandler]() + /** Initialize all components of the server. */ + def initialize() { + listenerBus.addListener(storageStatusListener) + val jobProgressTab = new JobProgressTab(this) + attachTab(jobProgressTab) + attachTab(new StorageTab(this)) + attachTab(new EnvironmentTab(this)) + attachTab(new ExecutorsTab(this)) + attachHandler(createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static")) + attachHandler(createRedirectHandler("/", "/stages", basePath = basePath)) + attachHandler( + createRedirectHandler("/stages/stage/kill", "/stages", jobProgressTab.handleKillRequest)) + if (live) { + sc.env.metricsSystem.getServletHandlers.foreach(attachHandler) } - storage.getHandlers ++ - jobs.getHandlers ++ - env.getHandlers ++ - exec.getHandlers ++ - metricsServletHandlers ++ - Seq[ServletContextHandler] ( - createStaticHandler(SparkUI.STATIC_RESOURCE_DIR, "/static"), - createRedirectHandler("/", "/stages", basePath = basePath) - ) } - // Maintain executor storage status through Spark events - val storageStatusListener = new StorageStatusListener - + /** Set the app name for this UI. */ def setAppName(name: String) { appName = name } - /** Initialize all components of the server */ - def start() { - storage.start() - jobs.start() - env.start() - exec.start() - - // Storage status listener must receive events first, as other listeners depend on its state - listenerBus.addListener(storageStatusListener) - listenerBus.addListener(storage.listener) - listenerBus.addListener(jobs.listener) - listenerBus.addListener(env.listener) - listenerBus.addListener(exec.listener) - } - - /** Bind to the HTTP server behind this web interface. */ - override def bind() { - try { - serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, sc.conf)) - logInfo("Started Spark web UI at http://%s:%d".format(publicHost, boundPort)) - } catch { - case e: Exception => - logError("Failed to create Spark web UI", e) - System.exit(1) - } + /** Register the given listener with the listener bus. */ + def registerListener(listener: SparkListener) { + listenerBus.addListener(listener) } /** Stop the server behind this web interface. Only valid after bind(). */ override def stop() { super.stop() - logInfo("Stopped Spark Web UI at %s".format(appUIAddress)) + logInfo("Stopped Spark web UI at %s".format(appUIAddress)) } /** * Return the application UI host:port. This does not include the scheme (http://). */ - private[spark] def appUIHostPort = publicHost + ":" + boundPort + private[spark] def appUIHostPort = publicHostName + ":" + boundPort private[spark] def appUIAddress = s"http://$appUIHostPort" - } private[spark] object SparkUI { val DEFAULT_PORT = 4040 val STATIC_RESOURCE_DIR = "org/apache/spark/ui/static" + + def getUIPort(conf: SparkConf): Int = { + conf.getInt("spark.ui.port", SparkUI.DEFAULT_PORT) + } } diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index a7cf04b3cbb86..6a2d652528d8a 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -17,16 +17,115 @@ package org.apache.spark.ui +import java.text.SimpleDateFormat +import java.util.{Locale, Date} + import scala.xml.Node +import org.apache.spark.Logging /** Utility functions for generating XML pages with spark content. */ -private[spark] object UIUtils { +private[spark] object UIUtils extends Logging { + + // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. + private val dateFormat = new ThreadLocal[SimpleDateFormat]() { + override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") + } + + def formatDate(date: Date): String = dateFormat.get.format(date) + + def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) + + def formatDuration(milliseconds: Long): String = { + val seconds = milliseconds.toDouble / 1000 + if (seconds < 60) { + return "%.0f s".format(seconds) + } + val minutes = seconds / 60 + if (minutes < 10) { + return "%.1f min".format(minutes) + } else if (minutes < 60) { + return "%.0f min".format(minutes) + } + val hours = minutes / 60 + "%.1f h".format(hours) + } + + /** Generate a verbose human-readable string representing a duration such as "5 second 35 ms" */ + def formatDurationVerbose(ms: Long): String = { + try { + val second = 1000L + val minute = 60 * second + val hour = 60 * minute + val day = 24 * hour + val week = 7 * day + val year = 365 * day + + def toString(num: Long, unit: String): String = { + if (num == 0) { + "" + } else if (num == 1) { + s"$num $unit" + } else { + s"$num ${unit}s" + } + } + + val millisecondsString = if (ms >= second && ms % second == 0) "" else s"${ms % second} ms" + val secondString = toString((ms % minute) / second, "second") + val minuteString = toString((ms % hour) / minute, "minute") + val hourString = toString((ms % day) / hour, "hour") + val dayString = toString((ms % week) / day, "day") + val weekString = toString((ms % year) / week, "week") + val yearString = toString(ms / year, "year") - import Page._ + Seq( + second -> millisecondsString, + minute -> s"$secondString $millisecondsString", + hour -> s"$minuteString $secondString", + day -> s"$hourString $minuteString $secondString", + week -> s"$dayString $hourString $minuteString", + year -> s"$weekString $dayString $hourString" + ).foreach { case (durationLimit, durationString) => + if (ms < durationLimit) { + // if time is less than the limit (upto year) + return durationString + } + } + // if time is more than a year + return s"$yearString $weekString $dayString" + } catch { + case e: Exception => + logError("Error converting time to string", e) + // if there is some error, return blank string + return "" + } + } + + /** Generate a human-readable string representing a number (e.g. 100 K) */ + def formatNumber(records: Double): String = { + val trillion = 1e12 + val billion = 1e9 + val million = 1e6 + val thousand = 1e3 + + val (value, unit) = { + if (records >= 2*trillion) { + (records / trillion, " T") + } else if (records >= 2*billion) { + (records / billion, " B") + } else if (records >= 2*million) { + (records / million, " M") + } else if (records >= 2*thousand) { + (records / thousand, " K") + } else { + (records, "") + } + } + "%.1f%s".formatLocal(Locale.US, value, unit) + } // Yarn has to go through a proxy so the base uri is provided and has to be on all links - private[spark] val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")). - getOrElse("") + val uiRoot : String = Option(System.getenv("APPLICATION_WEB_PROXY_BASE")).getOrElse("") def prependBaseUri(basePath: String = "", resource: String = "") = uiRoot + basePath + resource @@ -36,26 +135,14 @@ private[spark] object UIUtils { basePath: String, appName: String, title: String, - page: Page.Value) : Seq[Node] = { - val jobs = page match { - case Stages => -
  • Stages
  • - case _ =>
  • Stages
  • - } - val storage = page match { - case Storage => -
  • Storage
  • - case _ =>
  • Storage
  • - } - val environment = page match { - case Environment => -
  • Environment
  • - case _ =>
  • Environment
  • - } - val executors = page match { - case Executors => -
  • Executors
  • - case _ =>
  • Executors
  • + tabs: Seq[WebUITab], + activeTab: WebUITab, + refreshInterval: Option[Int] = None): Seq[Node] = { + + val header = tabs.map { tab => +
  • + {tab.name} +
  • } @@ -74,16 +161,10 @@ private[spark] object UIUtils { - + -
    @@ -129,21 +210,36 @@ private[spark] object UIUtils { /** Returns an HTML table constructed by generating a row for each object in a sequence. */ def listingTable[T]( headers: Seq[String], - makeRow: T => Seq[Node], - rows: Seq[T], + generateDataRow: T => Seq[Node], + data: Seq[T], fixedWidth: Boolean = false): Seq[Node] = { - val colWidth = 100.toDouble / headers.size - val colWidthAttr = if (fixedWidth) colWidth + "%" else "" var tableClass = "table table-bordered table-striped table-condensed sortable" if (fixedWidth) { tableClass += " table-fixed" } - + val colWidth = 100.toDouble / headers.size + val colWidthAttr = if (fixedWidth) colWidth + "%" else "" + val headerRow: Seq[Node] = { + // if none of the headers have "\n" in them + if (headers.forall(!_.contains("\n"))) { + // represent header as simple text + headers.map(h => {h}) + } else { + // represent header text as list while respecting "\n" + headers.map { case h => + +
      + { h.split("\n").map { case t =>
    • {t}
    • } } +
    + + } + } + } - {headers.map(h => )} + {headerRow} - {rows.map(r => makeRow(r))} + {data.map(r => generateDataRow(r))}
    {h}
    } diff --git a/core/src/main/scala/org/apache/spark/ui/WebUI.scala b/core/src/main/scala/org/apache/spark/ui/WebUI.scala index 2cc7582eca8a3..b08f308fda1dd 100644 --- a/core/src/main/scala/org/apache/spark/ui/WebUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/WebUI.scala @@ -17,53 +17,134 @@ package org.apache.spark.ui -import java.text.SimpleDateFormat -import java.util.Date +import javax.servlet.http.HttpServletRequest -private[spark] abstract class WebUI(name: String) { +import scala.collection.mutable.ArrayBuffer +import scala.xml.Node + +import org.eclipse.jetty.servlet.ServletContextHandler +import org.json4s.JsonAST.{JNothing, JValue} + +import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.ui.JettyUtils._ +import org.apache.spark.util.Utils + +/** + * The top level component of the UI hierarchy that contains the server. + * + * Each WebUI represents a collection of tabs, each of which in turn represents a collection of + * pages. The use of tabs is optional, however; a WebUI may choose to include pages directly. + */ +private[spark] abstract class WebUI( + securityManager: SecurityManager, + port: Int, + conf: SparkConf, + basePath: String = "") + extends Logging { + + protected val tabs = ArrayBuffer[WebUITab]() + protected val handlers = ArrayBuffer[ServletContextHandler]() protected var serverInfo: Option[ServerInfo] = None + protected val localHostName = Utils.localHostName() + protected val publicHostName = Option(System.getenv("SPARK_PUBLIC_DNS")).getOrElse(localHostName) + private val className = Utils.getFormattedClassName(this) + + def getTabs: Seq[WebUITab] = tabs.toSeq + def getHandlers: Seq[ServletContextHandler] = handlers.toSeq + + /** Attach a tab to this UI, along with all of its attached pages. */ + def attachTab(tab: WebUITab) { + tab.pages.foreach(attachPage) + tabs += tab + } + + /** Attach a page to this UI. */ + def attachPage(page: WebUIPage) { + val pagePath = "/" + page.prefix + attachHandler(createServletHandler(pagePath, + (request: HttpServletRequest) => page.render(request), securityManager, basePath)) + attachHandler(createServletHandler(pagePath.stripSuffix("/") + "/json", + (request: HttpServletRequest) => page.renderJson(request), securityManager, basePath)) + } + + /** Attach a handler to this UI. */ + def attachHandler(handler: ServletContextHandler) { + handlers += handler + serverInfo.foreach { info => + info.rootHandler.addHandler(handler) + if (!handler.isStarted) { + handler.start() + } + } + } - /** - * Bind to the HTTP server behind this web interface. - * Overridden implementation should set serverInfo. - */ - def bind() { } + /** Detach a handler from this UI. */ + def detachHandler(handler: ServletContextHandler) { + handlers -= handler + serverInfo.foreach { info => + info.rootHandler.removeHandler(handler) + if (handler.isStarted) { + handler.stop() + } + } + } + + /** Initialize all components of the server. */ + def initialize() + + /** Bind to the HTTP server behind this web interface. */ + def bind() { + assert(!serverInfo.isDefined, "Attempted to bind %s more than once!".format(className)) + try { + serverInfo = Some(startJettyServer("0.0.0.0", port, handlers, conf)) + logInfo("Started %s at http://%s:%d".format(className, publicHostName, boundPort)) + } catch { + case e: Exception => + logError("Failed to bind %s".format(className), e) + System.exit(1) + } + } /** Return the actual port to which this server is bound. Only valid after bind(). */ def boundPort: Int = serverInfo.map(_.boundPort).getOrElse(-1) /** Stop the server behind this web interface. Only valid after bind(). */ def stop() { - assert(serverInfo.isDefined, "Attempted to stop %s before binding to a server!".format(name)) + assert(serverInfo.isDefined, + "Attempted to stop %s before binding to a server!".format(className)) serverInfo.get.server.stop() } } + /** - * Utilities used throughout the web UI. + * A tab that represents a collection of pages. + * The prefix is appended to the parent address to form a full path, and must not contain slashes. */ -private[spark] object WebUI { - // SimpleDateFormat is not thread-safe. Don't expose it to avoid improper use. - private val dateFormat = new ThreadLocal[SimpleDateFormat]() { - override def initialValue(): SimpleDateFormat = new SimpleDateFormat("yyyy/MM/dd HH:mm:ss") +private[spark] abstract class WebUITab(parent: WebUI, val prefix: String) { + val pages = ArrayBuffer[WebUIPage]() + val name = prefix.capitalize + + /** Attach a page to this tab. This prepends the page's prefix with the tab's own prefix. */ + def attachPage(page: WebUIPage) { + page.prefix = (prefix + "/" + page.prefix).stripSuffix("/") + pages += page } - def formatDate(date: Date): String = dateFormat.get.format(date) + /** Get a list of header tabs from the parent UI. */ + def headerTabs: Seq[WebUITab] = parent.getTabs +} - def formatDate(timestamp: Long): String = dateFormat.get.format(new Date(timestamp)) - def formatDuration(milliseconds: Long): String = { - val seconds = milliseconds.toDouble / 1000 - if (seconds < 60) { - return "%.0f s".format(seconds) - } - val minutes = seconds / 60 - if (minutes < 10) { - return "%.1f min".format(minutes) - } else if (minutes < 60) { - return "%.0f min".format(minutes) - } - val hours = minutes / 60 - "%.1f h".format(hours) - } +/** + * A page that represents the leaf node in the UI hierarchy. + * + * The direct parent of a WebUIPage is not specified as it can be either a WebUI or a WebUITab. + * If the parent is a WebUI, the prefix is appended to the parent's address to form a full path. + * Else, if the parent is a WebUITab, the prefix is appended to the super prefix of the parent + * to form a relative path. The prefix must not contain slashes. + */ +private[spark] abstract class WebUIPage(var prefix: String) { + def render(request: HttpServletRequest): Seq[Node] + def renderJson(request: HttpServletRequest): JValue = JNothing } diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala rename to core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala index 33df97187ea78..b347eb1b83c1f 100644 --- a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentPage.scala @@ -21,29 +21,12 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.eclipse.jetty.servlet.ServletContextHandler +import org.apache.spark.ui.{UIUtils, WebUIPage} -import org.apache.spark.scheduler._ -import org.apache.spark.ui._ -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.Page.Environment - -private[ui] class EnvironmentUI(parent: SparkUI) { +private[ui] class EnvironmentPage(parent: EnvironmentTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath - private var _listener: Option[EnvironmentListener] = None - - private def appName = parent.appName - - lazy val listener = _listener.get - - def start() { - _listener = Some(new EnvironmentListener) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/environment", - (request: HttpServletRequest) => render(request), parent.securityManager, basePath) - ) + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val runtimeInformationTable = UIUtils.listingTable( @@ -62,7 +45,7 @@ private[ui] class EnvironmentUI(parent: SparkUI) {

    Classpath Entries

    {classpathEntriesTable} - UIUtils.headerSparkPage(content, basePath, appName, "Environment", Environment) + UIUtils.headerSparkPage(content, basePath, appName, "Environment", parent.headerTabs, parent) } private def propertyHeader = Seq("Name", "Value") @@ -71,23 +54,3 @@ private[ui] class EnvironmentUI(parent: SparkUI) { private def propertyRow(kv: (String, String)) = {kv._1}{kv._2} private def classPathRow(data: (String, String)) = {data._1}{data._2} } - -/** - * A SparkListener that prepares information to be displayed on the EnvironmentUI - */ -private[ui] class EnvironmentListener extends SparkListener { - var jvmInformation = Seq[(String, String)]() - var sparkProperties = Seq[(String, String)]() - var systemProperties = Seq[(String, String)]() - var classpathEntries = Seq[(String, String)]() - - override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { - synchronized { - val environmentDetails = environmentUpdate.environmentDetails - jvmInformation = environmentDetails("JVM Information") - sparkProperties = environmentDetails("Spark Properties") - systemProperties = environmentDetails("System Properties") - classpathEntries = environmentDetails("Classpath Entries") - } - } -} diff --git a/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala new file mode 100644 index 0000000000000..03b46e1bd59af --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/env/EnvironmentTab.scala @@ -0,0 +1,50 @@ +/* + * 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.ui.env + +import org.apache.spark.scheduler._ +import org.apache.spark.ui._ + +private[ui] class EnvironmentTab(parent: SparkUI) extends WebUITab(parent, "environment") { + val appName = parent.appName + val basePath = parent.basePath + val listener = new EnvironmentListener + + attachPage(new EnvironmentPage(this)) + parent.registerListener(listener) +} + +/** + * A SparkListener that prepares information to be displayed on the EnvironmentTab + */ +private[ui] class EnvironmentListener extends SparkListener { + var jvmInformation = Seq[(String, String)]() + var sparkProperties = Seq[(String, String)]() + var systemProperties = Seq[(String, String)]() + var classpathEntries = Seq[(String, String)]() + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + jvmInformation = environmentDetails("JVM Information") + sparkProperties = environmentDetails("Spark Properties") + systemProperties = environmentDetails("System Properties") + classpathEntries = environmentDetails("Classpath Entries") + } + } +} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala similarity index 61% rename from core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala rename to core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index 77a38a1d3aa7c..c1e69f6cdaffb 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -19,35 +19,15 @@ package org.apache.spark.ui.exec import javax.servlet.http.HttpServletRequest -import scala.collection.mutable.HashMap import scala.xml.Node -import org.eclipse.jetty.servlet.ServletContextHandler - -import org.apache.spark.ExceptionFailure -import org.apache.spark.scheduler._ -import org.apache.spark.storage.StorageStatusListener -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.Page.Executors -import org.apache.spark.ui.{SparkUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils -private[ui] class ExecutorsUI(parent: SparkUI) { +private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath - private var _listener: Option[ExecutorsListener] = None - - private def appName = parent.appName - - lazy val listener = _listener.get - - def start() { - _listener = Some(new ExecutorsListener(parent.storageStatusListener)) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/executors", - (request: HttpServletRequest) => render(request), parent.securityManager, basePath) - ) + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList @@ -75,8 +55,8 @@ private[ui] class ExecutorsUI(parent: SparkUI) {
    ; - UIUtils.headerSparkPage( - content, basePath, appName, "Executors (" + execInfo.size + ")", Executors) + UIUtils.headerSparkPage(content, basePath, appName, "Executors (" + execInfo.size + ")", + parent.headerTabs, parent) } /** Header fields for the executors table */ @@ -159,55 +139,3 @@ private[ui] class ExecutorsUI(parent: SparkUI) { execFields.zip(execValues).toMap } } - -/** - * A SparkListener that prepares information to be displayed on the ExecutorsUI - */ -private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) - extends SparkListener { - - val executorToTasksActive = HashMap[String, Int]() - val executorToTasksComplete = HashMap[String, Int]() - val executorToTasksFailed = HashMap[String, Int]() - val executorToDuration = HashMap[String, Long]() - val executorToShuffleRead = HashMap[String, Long]() - val executorToShuffleWrite = HashMap[String, Long]() - - def storageStatusList = storageStatusListener.storageStatusList - - override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { - val eid = formatExecutorId(taskStart.taskInfo.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 - } - - override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { - val info = taskEnd.taskInfo - if (info != null) { - val eid = formatExecutorId(info.executorId) - executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 - executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration - taskEnd.reason match { - case e: ExceptionFailure => - executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 - case _ => - executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 - } - - // Update shuffle read/write - val metrics = taskEnd.taskMetrics - if (metrics != null) { - metrics.shuffleReadMetrics.foreach { shuffleRead => - executorToShuffleRead(eid) = - executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead - } - metrics.shuffleWriteMetrics.foreach { shuffleWrite => - executorToShuffleWrite(eid) = - executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten - } - } - } - } - - // This addresses executor ID inconsistencies in the local mode - private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) -} diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala new file mode 100644 index 0000000000000..5678bf34ac730 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsTab.scala @@ -0,0 +1,86 @@ +/* + * 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.ui.exec + +import scala.collection.mutable.HashMap + +import org.apache.spark.ExceptionFailure +import org.apache.spark.scheduler._ +import org.apache.spark.storage.StorageStatusListener +import org.apache.spark.ui.{SparkUI, WebUITab} + +private[ui] class ExecutorsTab(parent: SparkUI) extends WebUITab(parent, "executors") { + val appName = parent.appName + val basePath = parent.basePath + val listener = new ExecutorsListener(parent.storageStatusListener) + + attachPage(new ExecutorsPage(this)) + parent.registerListener(listener) +} + +/** + * A SparkListener that prepares information to be displayed on the ExecutorsTab + */ +private[ui] class ExecutorsListener(storageStatusListener: StorageStatusListener) + extends SparkListener { + + val executorToTasksActive = HashMap[String, Int]() + val executorToTasksComplete = HashMap[String, Int]() + val executorToTasksFailed = HashMap[String, Int]() + val executorToDuration = HashMap[String, Long]() + val executorToShuffleRead = HashMap[String, Long]() + val executorToShuffleWrite = HashMap[String, Long]() + + def storageStatusList = storageStatusListener.storageStatusList + + override def onTaskStart(taskStart: SparkListenerTaskStart) = synchronized { + val eid = formatExecutorId(taskStart.taskInfo.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 0) + 1 + } + + override def onTaskEnd(taskEnd: SparkListenerTaskEnd) = synchronized { + val info = taskEnd.taskInfo + if (info != null) { + val eid = formatExecutorId(info.executorId) + executorToTasksActive(eid) = executorToTasksActive.getOrElse(eid, 1) - 1 + executorToDuration(eid) = executorToDuration.getOrElse(eid, 0L) + info.duration + taskEnd.reason match { + case e: ExceptionFailure => + executorToTasksFailed(eid) = executorToTasksFailed.getOrElse(eid, 0) + 1 + case _ => + executorToTasksComplete(eid) = executorToTasksComplete.getOrElse(eid, 0) + 1 + } + + // Update shuffle read/write + val metrics = taskEnd.taskMetrics + if (metrics != null) { + metrics.shuffleReadMetrics.foreach { shuffleRead => + executorToShuffleRead(eid) = + executorToShuffleRead.getOrElse(eid, 0L) + shuffleRead.remoteBytesRead + } + metrics.shuffleWriteMetrics.foreach { shuffleWrite => + executorToShuffleWrite(eid) = + executorToShuffleWrite.getOrElse(eid, 0L) + shuffleWrite.shuffleBytesWritten + } + } + } + } + + // This addresses executor ID inconsistencies in the local mode + private def formatExecutorId(execId: String) = storageStatusListener.formatExecutorId(execId) +} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala index 73861ae6746da..c83e196c9c156 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/ExecutorTable.scala @@ -20,11 +20,12 @@ package org.apache.spark.ui.jobs import scala.collection.mutable import scala.xml.Node +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing executor summary */ -private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { - private lazy val listener = parent.listener +private[ui] class ExecutorTable(stageId: Int, parent: JobProgressTab) { + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -69,7 +70,7 @@ private[ui] class ExecutorTable(stageId: Int, parent: JobProgressUI) { {k} {executorIdToAddress.getOrElse(k, "CANNOT FIND ADDRESS")} - {parent.formatDuration(v.taskTime)} + {UIUtils.formatDuration(v.taskTime)} {v.failedTasks + v.succeededTasks} {v.failedTasks} {v.succeededTasks} diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala index 5167e20ea3d7d..0db4afa701b41 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressListener.scala @@ -222,12 +222,10 @@ private[ui] class JobProgressListener(conf: SparkConf) extends SparkListener { override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { synchronized { - val schedulingModeName = - environmentUpdate.environmentDetails("Spark Properties").toMap.get("spark.scheduler.mode") - schedulingMode = schedulingModeName match { - case Some(name) => Some(SchedulingMode.withName(name)) - case None => None - } + schedulingMode = environmentUpdate + .environmentDetails("Spark Properties").toMap + .get("spark.scheduler.mode") + .map(SchedulingMode.withName) } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala similarity index 90% rename from core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 8619a31380f1e..34ff2ac34a7ca 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -22,25 +22,23 @@ import javax.servlet.http.HttpServletRequest import scala.xml.{Node, NodeSeq} import org.apache.spark.scheduler.Schedulable -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing list of all ongoing and recently finished stages and pools */ -private[ui] class IndexPage(parent: JobProgressUI) { +private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener + private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler - private def appName = parent.appName - def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { val activeStages = listener.activeStages.values.toSeq val completedStages = listener.completedStages.reverse.toSeq val failedStages = listener.failedStages.reverse.toSeq - val now = System.currentTimeMillis() + val now = System.currentTimeMillis val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) @@ -59,7 +57,7 @@ private[ui] class IndexPage(parent: JobProgressUI) { // Total duration is not meaningful unless the UI is live
  • Total Duration: - {parent.formatDuration(now - sc.startTime)} + {UIUtils.formatDuration(now - sc.startTime)}
  • }}
  • @@ -94,7 +92,7 @@ private[ui] class IndexPage(parent: JobProgressUI) {

    Failed Stages ({failedStages.size})

    ++ failedStagesTable.toNodeSeq - UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Spark Stages", parent.headerTabs, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala similarity index 53% rename from core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala rename to core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala index 30e3f35f2182b..3308c8c8a3d37 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressTab.scala @@ -19,39 +19,28 @@ package org.apache.spark.ui.jobs import javax.servlet.http.HttpServletRequest -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.SparkConf import org.apache.spark.scheduler.SchedulingMode -import org.apache.spark.ui.JettyUtils._ -import org.apache.spark.ui.SparkUI -import org.apache.spark.util.Utils +import org.apache.spark.ui.{SparkUI, WebUITab} /** Web UI showing progress status of all jobs in the given SparkContext. */ -private[ui] class JobProgressUI(parent: SparkUI) { +private[ui] class JobProgressTab(parent: SparkUI) extends WebUITab(parent, "stages") { + val appName = parent.appName val basePath = parent.basePath val live = parent.live val sc = parent.sc - val killEnabled = parent.conf.getBoolean("spark.ui.killEnabled", true) - - lazy val listener = _listener.get - lazy val isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - - private val indexPage = new IndexPage(this) - private val stagePage = new StagePage(this) - private val poolPage = new PoolPage(this) - private var _listener: Option[JobProgressListener] = None + val conf = if (live) sc.conf else new SparkConf + val killEnabled = conf.getBoolean("spark.ui.killEnabled", true) + val listener = new JobProgressListener(conf) - def appName = parent.appName + attachPage(new JobProgressPage(this)) + attachPage(new StagePage(this)) + attachPage(new PoolPage(this)) + parent.registerListener(listener) - def start() { - val conf = if (live) sc.conf else new SparkConf - _listener = Some(new JobProgressListener(conf)) - } - - def formatDuration(ms: Long) = Utils.msDurationToString(ms) + def isFairScheduler = listener.schedulingMode.exists(_ == SchedulingMode.FAIR) - private def handleKillRequest(request: HttpServletRequest) = { + def handleKillRequest(request: HttpServletRequest) = { if (killEnabled) { val killFlag = Option(request.getParameter("terminate")).getOrElse("false").toBoolean val stageId = Option(request.getParameter("id")).getOrElse("-1").toInt @@ -64,14 +53,4 @@ private[ui] class JobProgressUI(parent: SparkUI) { Thread.sleep(100) } } - - def getHandlers = Seq[ServletContextHandler]( - createRedirectHandler("/stages/stage/kill", "/stages", handleKillRequest), - createServletHandler("/stages/stage", - (request: HttpServletRequest) => stagePage.render(request), parent.securityManager, basePath), - createServletHandler("/stages/pool", - (request: HttpServletRequest) => poolPage.render(request), parent.securityManager, basePath), - createServletHandler("/stages", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index 3638e6035ba81..fd83d37583967 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -22,17 +22,15 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.scheduler.{Schedulable, StageInfo} -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} /** Page showing specific pool details */ -private[ui] class PoolPage(parent: JobProgressUI) { +private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { + private val appName = parent.appName private val basePath = parent.basePath private val live = parent.live private val sc = parent.sc - private lazy val listener = parent.listener - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -52,8 +50,8 @@ private[ui] class PoolPage(parent: JobProgressUI) {

    Summary

    ++ poolTable.toNodeSeq ++

    {activeStages.size} Active Stages

    ++ activeStagesTable.toNodeSeq - UIUtils.headerSparkPage( - content, basePath, appName, "Fair Scheduler Pool: " + poolName, Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Fair Scheduler Pool: " + poolName, + parent.headerTabs, parent) } } } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala index c5c8d8668740b..f4b68f241966d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolTable.scala @@ -24,10 +24,9 @@ import org.apache.spark.scheduler.{Schedulable, StageInfo} import org.apache.spark.ui.UIUtils /** Table showing list of pools */ -private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { +private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressTab) { private val basePath = parent.basePath - private val poolToActiveStages = listener.poolToActiveStages - private lazy val listener = parent.listener + private val listener = parent.listener def toNodeSeq: Seq[Node] = { listener.synchronized { @@ -48,7 +47,7 @@ private[ui] class PoolTable(pools: Seq[Schedulable], parent: JobProgressUI) { SchedulingMode - {rows.map(r => makeRow(r, poolToActiveStages))} + {rows.map(r => makeRow(r, listener.poolToActiveStages))} } diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala index b6c3e3cf45163..4bce472036f7d 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StagePage.scala @@ -22,17 +22,14 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.{Utils, Distribution} /** Page showing statistics and task list for a given stage */ -private[ui] class StagePage(parent: JobProgressUI) { +private[ui] class StagePage(parent: JobProgressTab) extends WebUIPage("stage") { + private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener - private lazy val sc = parent.sc - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { listener.synchronized { @@ -44,8 +41,8 @@ private[ui] class StagePage(parent: JobProgressUI) {

    Summary Metrics

    No tasks have started yet

    Tasks

    No tasks have started yet
  • - return UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %s".format(stageId), Stages) + return UIUtils.headerSparkPage(content, basePath, appName, + "Details for Stage %s".format(stageId), parent.headerTabs, parent) } val tasks = listener.stageIdToTaskData(stageId).values.toSeq.sortBy(_.taskInfo.launchTime) @@ -60,7 +57,7 @@ private[ui] class StagePage(parent: JobProgressUI) { val hasBytesSpilled = memoryBytesSpilled > 0 && diskBytesSpilled > 0 var activeTime = 0L - val now = System.currentTimeMillis() + val now = System.currentTimeMillis val tasksActive = listener.stageIdToTasksActive(stageId).values tasksActive.foreach(activeTime += _.timeRunning(now)) @@ -70,7 +67,7 @@ private[ui] class StagePage(parent: JobProgressUI) {
    • Total task time across all tasks: - {parent.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)} + {UIUtils.formatDuration(listener.stageIdToTime.getOrElse(stageId, 0L) + activeTime)}
    • {if (hasShuffleRead)
    • @@ -121,13 +118,13 @@ private[ui] class StagePage(parent: JobProgressUI) { } val serializationQuantiles = "Result serialization time" +: Distribution(serializationTimes). - get.getQuantiles().map(ms => parent.formatDuration(ms.toLong)) + get.getQuantiles().map(ms => UIUtils.formatDuration(ms.toLong)) val serviceTimes = validTasks.map { case TaskUIData(_, metrics, _) => metrics.get.executorRunTime.toDouble } val serviceQuantiles = "Duration" +: Distribution(serviceTimes).get.getQuantiles() - .map(ms => parent.formatDuration(ms.toLong)) + .map(ms => UIUtils.formatDuration(ms.toLong)) val gettingResultTimes = validTasks.map { case TaskUIData(info, _, _) => if (info.gettingResultTime > 0) { @@ -138,7 +135,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val gettingResultQuantiles = "Time spent fetching task results" +: Distribution(gettingResultTimes).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } // The scheduler delay includes the network delay to send the task to the worker // machine and to send back the result (but not the time to fetch the task result, @@ -155,7 +152,7 @@ private[ui] class StagePage(parent: JobProgressUI) { } val schedulerDelayQuantiles = "Scheduler delay" +: Distribution(schedulerDelays).get.getQuantiles().map { millis => - parent.formatDuration(millis.toLong) + UIUtils.formatDuration(millis.toLong) } def getQuantileCols(data: Seq[Double]) = @@ -206,8 +203,8 @@ private[ui] class StagePage(parent: JobProgressUI) {

      Aggregated Metrics by Executor

      ++ executorTable.toNodeSeq ++

      Tasks

      ++ taskTable - UIUtils.headerSparkPage( - content, basePath, appName, "Details for Stage %d".format(stageId), Stages) + UIUtils.headerSparkPage(content, basePath, appName, "Details for Stage %d".format(stageId), + parent.headerTabs, parent) } } @@ -219,8 +216,8 @@ private[ui] class StagePage(parent: JobProgressUI) { taskData match { case TaskUIData(info, metrics, exception) => val duration = if (info.status == "RUNNING") info.timeRunning(System.currentTimeMillis()) else metrics.map(_.executorRunTime).getOrElse(1L) - val formatDuration = if (info.status == "RUNNING") parent.formatDuration(duration) - else metrics.map(m => parent.formatDuration(m.executorRunTime)).getOrElse("") + val formatDuration = if (info.status == "RUNNING") UIUtils.formatDuration(duration) + else metrics.map(m => UIUtils.formatDuration(m.executorRunTime)).getOrElse("") val gcTime = metrics.map(_.jvmGCTime).getOrElse(0L) val serializationTime = metrics.map(_.resultSerializationTime).getOrElse(0L) @@ -235,8 +232,8 @@ private[ui] class StagePage(parent: JobProgressUI) { val maybeWriteTime = metrics.flatMap(_.shuffleWriteMetrics).map(_.shuffleWriteTime) val writeTimeSortable = maybeWriteTime.map(_.toString).getOrElse("") - val writeTimeReadable = maybeWriteTime.map( t => t / (1000 * 1000)).map { ms => - if (ms == 0) "" else parent.formatDuration(ms) + val writeTimeReadable = maybeWriteTime.map(t => t / (1000 * 1000)).map { ms => + if (ms == 0) "" else UIUtils.formatDuration(ms) }.getOrElse("") val maybeMemoryBytesSpilled = metrics.map(_.memoryBytesSpilled) @@ -254,15 +251,15 @@ private[ui] class StagePage(parent: JobProgressUI) { {info.status} {info.taskLocality} {info.host} - {WebUI.formatDate(new Date(info.launchTime))} + {UIUtils.formatDate(new Date(info.launchTime))} {formatDuration} - {if (gcTime > 0) parent.formatDuration(gcTime) else ""} + {if (gcTime > 0) UIUtils.formatDuration(gcTime) else ""} - {if (serializationTime > 0) parent.formatDuration(serializationTime) else ""} + {if (serializationTime > 0) UIUtils.formatDuration(serializationTime) else ""} {if (shuffleRead) { diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index e419fae5a6589..8c5b1f55fd2dc 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -23,17 +23,17 @@ import scala.collection.mutable.HashMap import scala.xml.Node import org.apache.spark.scheduler.{StageInfo, TaskInfo} -import org.apache.spark.ui.{WebUI, UIUtils} +import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ private[ui] class StageTable( - stages: Seq[StageInfo], - parent: JobProgressUI, - killEnabled: Boolean = false) { + stages: Seq[StageInfo], + parent: JobProgressTab, + killEnabled: Boolean = false) { private val basePath = parent.basePath - private lazy val listener = parent.listener + private val listener = parent.listener private lazy val isFairScheduler = parent.isFairScheduler def toNodeSeq: Seq[Node] = { @@ -89,25 +89,23 @@ private[ui] class StageTable( {s.name} - val description = listener.stageIdToDescription.get(s.stageId) + listener.stageIdToDescription.get(s.stageId) .map(d =>
      {d}
      {nameLink} {killLink}
      ) .getOrElse(
      {killLink}{nameLink}
      ) - - return description } /** Render an HTML row that represents a stage */ private def stageRow(s: StageInfo): Seq[Node] = { val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { - case Some(t) => WebUI.formatDate(new Date(t)) + case Some(t) => UIUtils.formatDate(new Date(t)) case None => "Unknown" } val finishTime = s.completionTime.getOrElse(System.currentTimeMillis) val duration = s.submissionTime.map { t => if (finishTime > t) finishTime - t else System.currentTimeMillis - t } - val formattedDuration = duration.map(d => parent.formatDuration(d)).getOrElse("Unknown") + val formattedDuration = duration.map(d => UIUtils.formatDuration(d)).getOrElse("Unknown") val startedTasks = listener.stageIdToTasksActive.getOrElse(s.stageId, HashMap[Long, TaskInfo]()).size val completedTasks = listener.stageIdToTasksComplete.getOrElse(s.stageId, 0) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index 75ee9976d7b5f..d07f1c9b20fcf 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -22,23 +22,22 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.{BlockId, BlockStatus, StorageStatus, StorageUtils} -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RDDPage(parent: BlockManagerUI) { +private[ui] class RddPage(parent: StorageTab) extends WebUIPage("rdd") { + private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { val rddId = request.getParameter("id").toInt val storageStatusList = listener.storageStatusList val rddInfo = listener.rddInfoList.find(_.id == rddId).getOrElse { // Rather than crashing, render an "RDD Not Found" page - return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", Storage) + return UIUtils.headerSparkPage(Seq[Node](), basePath, appName, "RDD Not Found", + parent.headerTabs, parent) } // Worker table @@ -96,8 +95,8 @@ private[ui] class RDDPage(parent: BlockManagerUI) { ; - UIUtils.headerSparkPage( - content, basePath, appName, "RDD Storage Info for " + rddInfo.name, Storage) + UIUtils.headerSparkPage(content, basePath, appName, "RDD Storage Info for " + rddInfo.name, + parent.headerTabs, parent) } /** Header fields for the worker table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala similarity index 90% rename from core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala index 4f6acc30a88c4..b66edd91f56c0 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/IndexPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StoragePage.scala @@ -22,22 +22,19 @@ import javax.servlet.http.HttpServletRequest import scala.xml.Node import org.apache.spark.storage.RDDInfo -import org.apache.spark.ui.Page._ -import org.apache.spark.ui.UIUtils +import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing list of RDD's currently stored in the cluster */ -private[ui] class IndexPage(parent: BlockManagerUI) { +private[ui] class StoragePage(parent: StorageTab) extends WebUIPage("") { + private val appName = parent.appName private val basePath = parent.basePath - private lazy val listener = parent.listener - - private def appName = parent.appName + private val listener = parent.listener def render(request: HttpServletRequest): Seq[Node] = { - val rdds = listener.rddInfoList val content = UIUtils.listingTable(rddHeader, rddRow, rdds) - UIUtils.headerSparkPage(content, basePath, appName, "Storage ", Storage) + UIUtils.headerSparkPage(content, basePath, appName, "Storage ", parent.headerTabs, parent) } /** Header fields for the RDD table */ diff --git a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala similarity index 75% rename from core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala rename to core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 16996a2da1e72..56429f6c07fcd 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/BlockManagerUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -17,45 +17,27 @@ package org.apache.spark.ui.storage -import javax.servlet.http.HttpServletRequest - import scala.collection.mutable -import org.eclipse.jetty.servlet.ServletContextHandler - import org.apache.spark.ui._ -import org.apache.spark.ui.JettyUtils._ import org.apache.spark.scheduler._ import org.apache.spark.storage.{RDDInfo, StorageStatusListener, StorageUtils} /** Web UI showing storage status of all RDD's in the given SparkContext. */ -private[ui] class BlockManagerUI(parent: SparkUI) { +private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage") { + val appName = parent.appName val basePath = parent.basePath + val listener = new StorageListener(parent.storageStatusListener) - private val indexPage = new IndexPage(this) - private val rddPage = new RDDPage(this) - private var _listener: Option[BlockManagerListener] = None - - lazy val listener = _listener.get - - def appName = parent.appName - - def start() { - _listener = Some(new BlockManagerListener(parent.storageStatusListener)) - } - - def getHandlers = Seq[ServletContextHandler]( - createServletHandler("/storage/rdd", - (request: HttpServletRequest) => rddPage.render(request), parent.securityManager, basePath), - createServletHandler("/storage", - (request: HttpServletRequest) => indexPage.render(request), parent.securityManager, basePath) - ) + attachPage(new StoragePage(this)) + attachPage(new RddPage(this)) + parent.registerListener(listener) } /** * A SparkListener that prepares information to be displayed on the BlockManagerUI */ -private[ui] class BlockManagerListener(storageStatusListener: StorageStatusListener) +private[ui] class StorageListener(storageStatusListener: StorageStatusListener) extends SparkListener { private val _rddInfoMap = mutable.Map[Int, RDDInfo]() diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index f2396f7c80a35..465835ea7fe29 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -88,30 +88,27 @@ private[spark] object JsonProtocol { def taskStartToJson(taskStart: SparkListenerTaskStart): JValue = { val taskInfo = taskStart.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskStart)) ~ ("Stage ID" -> taskStart.stageId) ~ - ("Task Info" -> taskInfoJson) + ("Task Info" -> taskInfoToJson(taskInfo)) } def taskGettingResultToJson(taskGettingResult: SparkListenerTaskGettingResult): JValue = { val taskInfo = taskGettingResult.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing ("Event" -> Utils.getFormattedClassName(taskGettingResult)) ~ - ("Task Info" -> taskInfoJson) + ("Task Info" -> taskInfoToJson(taskInfo)) } def taskEndToJson(taskEnd: SparkListenerTaskEnd): JValue = { val taskEndReason = taskEndReasonToJson(taskEnd.reason) val taskInfo = taskEnd.taskInfo - val taskInfoJson = if (taskInfo != null) taskInfoToJson(taskInfo) else JNothing val taskMetrics = taskEnd.taskMetrics val taskMetricsJson = if (taskMetrics != null) taskMetricsToJson(taskMetrics) else JNothing ("Event" -> Utils.getFormattedClassName(taskEnd)) ~ ("Stage ID" -> taskEnd.stageId) ~ ("Task Type" -> taskEnd.taskType) ~ ("Task End Reason" -> taskEndReason) ~ - ("Task Info" -> taskInfoJson) ~ + ("Task Info" -> taskInfoToJson(taskInfo)) ~ ("Task Metrics" -> taskMetricsJson) } @@ -505,6 +502,9 @@ private[spark] object JsonProtocol { } def taskMetricsFromJson(json: JValue): TaskMetrics = { + if (json == JNothing) { + return TaskMetrics.empty + } val metrics = new TaskMetrics metrics.hostname = (json \ "Host Name").extract[String] metrics.executorDeserializeTime = (json \ "Executor Deserialize Time").extract[Long] diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index 2f9739f940dc6..b85c483ca2a08 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -18,16 +18,81 @@ package org.apache.spark.ui import java.net.ServerSocket +import javax.servlet.http.HttpServletRequest +import scala.io.Source import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server import org.eclipse.jetty.servlet.ServletContextHandler import org.scalatest.FunSuite +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ -import org.apache.spark.SparkConf +import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.LocalSparkContext._ +import scala.xml.Node class UISuite extends FunSuite { + + test("basic ui visibility") { + withSpark(new SparkContext("local", "test")) { sc => + // test if the ui is visible, and all the expected tabs are visible + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + } + } + + test("visibility at localhost:4040") { + withSpark(new SparkContext("local", "test")) { sc => + // test if visible from http://localhost:4040 + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL("http://localhost:4040").mkString + assert(html.toLowerCase.contains("stages")) + } + } + } + + test("attaching a new tab") { + withSpark(new SparkContext("local", "test")) { sc => + val sparkUI = sc.ui + + val newTab = new WebUITab(sparkUI, "foo") { + attachPage(new WebUIPage("") { + def render(request: HttpServletRequest): Seq[Node] = { + "html magic" + } + }) + } + sparkUI.attachTab(newTab) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + + // check whether new page exists + assert(html.toLowerCase.contains("foo")) + + // check whether other pages still exist + assert(html.toLowerCase.contains("stages")) + assert(html.toLowerCase.contains("storage")) + assert(html.toLowerCase.contains("environment")) + assert(html.toLowerCase.contains("executors")) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(sc.ui.appUIAddress.stripSuffix("/") + "/foo").mkString + // check whether new page exists + assert(html.contains("magic")) + } + } + } + test("jetty port increases under contention") { val startPort = 4040 val server = new Server(startPort) @@ -60,4 +125,18 @@ class UISuite extends FunSuite { case Failure(e) => } } + + test("verify appUIAddress contains the scheme") { + withSpark(new SparkContext("local", "test")) { sc => + val uiAddress = sc.ui.appUIAddress + assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) + } + } + + test("verify appUIAddress contains the port") { + withSpark(new SparkContext("local", "test")) { sc => + val splitUIAddress = sc.ui.appUIAddress.split(':') + assert(splitUIAddress(2).toInt == sc.ui.boundPort) + } + } } diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index f75297a02dc8b..16470bb7bf60d 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -523,8 +523,8 @@ class JsonProtocolSuite extends FunSuite { 700,"Fetch Wait Time":900,"Remote Bytes Read":1000},"Shuffle Write Metrics": {"Shuffle Bytes Written":1200,"Shuffle Write Time":1500},"Updated Blocks": [{"Block ID":{"Type":"RDDBlockId","RDD ID":0,"Split Index":0},"Status": - {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false,"Deserialized":false, - "Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} + {"Storage Level":{"Use Disk":true,"Use Memory":true,"Use Tachyon":false, + "Deserialized":false,"Replication":2},"Memory Size":0,"Disk Size":0,"Tachyon Size":0}}]}} """ private val jobStartJsonString = diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 5ea4817bfde18..9cb31d70444ff 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -60,6 +60,7 @@ object MimaBuild { Seq( excludePackage("org.apache.spark.api.java"), excludePackage("org.apache.spark.streaming.api.java"), + excludePackage("org.apache.spark.streaming.scheduler"), excludePackage("org.apache.spark.mllib") ) ++ excludeSparkClass("rdd.ClassTags") ++ @@ -70,7 +71,12 @@ object MimaBuild { excludeSparkClass("mllib.regression.LassoWithSGD") ++ excludeSparkClass("mllib.regression.LinearRegressionWithSGD") ++ excludeSparkClass("streaming.dstream.NetworkReceiver") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") + excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator") ++ + excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator#Block") ++ + excludeSparkClass("streaming.dstream.ReportError") ++ + excludeSparkClass("streaming.dstream.ReportBlock") ++ + excludeSparkClass("streaming.dstream.DStream") case _ => Seq() } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index a4e236c65ff86..ff5d0aaa3d0bd 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -17,29 +17,28 @@ package org.apache.spark.streaming -import scala.collection.mutable.Queue -import scala.collection.Map -import scala.reflect.ClassTag - import java.io.InputStream import java.util.concurrent.atomic.AtomicInteger -import akka.actor.Props -import akka.actor.SupervisorStrategy -import org.apache.hadoop.io.LongWritable -import org.apache.hadoop.io.Text +import scala.collection.Map +import scala.collection.mutable.Queue +import scala.reflect.ClassTag + +import akka.actor.{Props, SupervisorStrategy} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.Path +import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat -import org.apache.hadoop.fs.Path import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.MetadataCleaner import org.apache.spark.streaming.dstream._ import org.apache.spark.streaming.receivers._ import org.apache.spark.streaming.scheduler._ -import org.apache.hadoop.conf.Configuration +import org.apache.spark.streaming.ui.StreamingTab +import org.apache.spark.util.MetadataCleaner /** * Main entry point for Spark Streaming functionality. It provides methods used to create @@ -158,6 +157,8 @@ class StreamingContext private[streaming] ( private[streaming] val waiter = new ContextWaiter + private[streaming] val uiTab = new StreamingTab(this) + /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { type CheckpointState = Value diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d043200f71a0b..a7e5215437e54 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -353,15 +353,6 @@ abstract class DStream[T: ClassTag] ( dependencies.foreach(_.clearMetadata(time)) } - /* Adds metadata to the Stream while it is running. - * This method should be overwritten by sublcasses of InputDStream. - */ - private[streaming] def addMetadata(metadata: Any) { - if (metadata != null) { - logInfo("Dropping Metadata: " + metadata.toString) - } - } - /** * Refresh the list of checkpointed RDDs that will be saved along with checkpoint of * this stream. This is an internal method that should not be called directly. This is diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala index d19a635fe8eca..5a249706b4d2f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala @@ -17,24 +17,23 @@ package org.apache.spark.streaming.dstream -import java.util.concurrent.{TimeUnit, ArrayBlockingQueue} import java.nio.ByteBuffer +import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} -import scala.collection.mutable.ArrayBuffer +import scala.collection.mutable.{ArrayBuffer, HashMap} import scala.concurrent.Await -import scala.concurrent.duration._ import scala.reflect.ClassTag -import akka.actor.{Props, Actor} +import akka.actor.{Actor, Props} import akka.pattern.ask -import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} -import org.apache.spark.streaming._ import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.rdd.{RDD, BlockRDD} +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} -import org.apache.spark.streaming.scheduler.{DeregisterReceiver, AddBlocks, RegisterReceiver} -import org.apache.spark.util.AkkaUtils +import org.apache.spark.streaming._ +import org.apache.spark.streaming.scheduler.{AddBlock, DeregisterReceiver, ReceivedBlockInfo, RegisterReceiver} +import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} +import org.apache.spark.util.{AkkaUtils, Utils} /** * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] @@ -49,8 +48,10 @@ import org.apache.spark.util.AkkaUtils abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) extends InputDStream[T](ssc_) { - // This is an unique identifier that is used to match the network receiver with the - // corresponding network input stream. + /** Keeps all received blocks information */ + private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] + + /** This is an unique identifier for the network input stream. */ val id = ssc.getNewNetworkStreamId() /** @@ -65,25 +66,44 @@ abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingConte def stop() {} + /** Ask NetworkInputTracker for received data blocks and generates RDDs with them. */ override def compute(validTime: Time): Option[RDD[T]] = { // If this is called for any time before the start time of the context, // then this returns an empty RDD. This may happen when recovering from a // master failure if (validTime >= graph.startTime) { - val blockIds = ssc.scheduler.networkInputTracker.getBlocks(id, validTime) + val blockInfo = ssc.scheduler.networkInputTracker.getReceivedBlockInfo(id) + receivedBlockInfo(validTime) = blockInfo + val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) Some(new BlockRDD[T](ssc.sc, blockIds)) } else { Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) } } + + /** Get information on received blocks. */ + private[streaming] def getReceivedBlockInfo(time: Time) = { + receivedBlockInfo(time) + } + + /** + * Clear metadata that are older than `rememberDuration` of this DStream. + * This is an internal method that should not be called directly. This + * implementation overrides the default implementation to clear received + * block information. + */ + private[streaming] override def clearMetadata(time: Time) { + super.clearMetadata(time) + val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) + receivedBlockInfo --= oldReceivedBlocks.keys + logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + + (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) + } } private[streaming] sealed trait NetworkReceiverMessage -private[streaming] case class StopReceiver() extends NetworkReceiverMessage -private[streaming] case class ReportBlock(blockId: BlockId, metadata: Any) - extends NetworkReceiverMessage -private[streaming] case class ReportError(msg: String) extends NetworkReceiverMessage +private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage /** * Abstract class of a receiver that can be run on worker nodes to receive external data. See @@ -177,6 +197,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging case (e, i) => "Exception " + i + ": " + e.getMessage + "\n" + e.getStackTraceString }.mkString("\n") } + logInfo("Deregistering receiver " + streamId) val future = trackerActor.ask(DeregisterReceiver(streamId, message))(askTimeout) Await.result(future, askTimeout) @@ -209,18 +230,28 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging /** * Push a block (as an ArrayBuffer filled with data) into the block manager. */ - def pushBlock(blockId: BlockId, arrayBuffer: ArrayBuffer[T], metadata: Any, level: StorageLevel) { + def pushBlock( + blockId: StreamBlockId, + arrayBuffer: ArrayBuffer[T], + metadata: Any, + level: StorageLevel + ) { env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) - trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, arrayBuffer.size, metadata)) logDebug("Pushed block " + blockId) } /** * Push a block (as bytes) into the block manager. */ - def pushBlock(blockId: BlockId, bytes: ByteBuffer, metadata: Any, level: StorageLevel) { + def pushBlock( + blockId: StreamBlockId, + bytes: ByteBuffer, + metadata: Any, + level: StorageLevel + ) { env.blockManager.putBytes(blockId, bytes, level) - trackerActor ! AddBlocks(streamId, Array(blockId), metadata) + trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, -1, metadata)) } /** Set the ID of the DStream that this receiver is associated with */ @@ -232,9 +263,11 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging private class NetworkReceiverActor extends Actor { override def preStart() { - logInfo("Registered receiver " + streamId) - val future = trackerActor.ask(RegisterReceiver(streamId, self))(askTimeout) + val msg = RegisterReceiver( + streamId, NetworkReceiver.this.getClass.getSimpleName, Utils.localHostName(), self) + val future = trackerActor.ask(msg)(askTimeout) Await.result(future, askTimeout) + logInfo("Registered receiver " + streamId) } override def receive() = { @@ -253,7 +286,7 @@ abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging class BlockGenerator(storageLevel: StorageLevel) extends Serializable with Logging { - case class Block(id: BlockId, buffer: ArrayBuffer[T], metadata: Any = null) + case class Block(id: StreamBlockId, buffer: ArrayBuffer[T], metadata: Any = null) val clock = new SystemClock() val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 7f3cd2f8eb1fd..9c69a2a4e21f5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -29,6 +29,7 @@ import org.apache.spark.streaming.Time */ case class BatchInfo( batchTime: Time, + receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]], submissionTime: Long, processingStartTime: Option[Long], processingEndTime: Option[Long] diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index 92d885c4bc5a5..e564eccba2df5 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -201,7 +201,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { logInfo("Batches to reschedule (" + timesToReschedule.size + " batches): " + timesToReschedule.mkString(", ")) timesToReschedule.foreach(time => - jobScheduler.runJobs(time, graph.generateJobs(time)) + jobScheduler.submitJobSet(JobSet(time, graph.generateJobs(time))) ) // Restart the timer @@ -214,7 +214,12 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { case Success(jobs) => - jobScheduler.runJobs(time, jobs) + val receivedBlockInfo = graph.getNetworkInputStreams.map { stream => + val streamId = stream.id + val receivedBlockInfo = stream.getReceivedBlockInfo(time) + (streamId, receivedBlockInfo) + }.toMap + jobScheduler.submitJobSet(JobSet(time, jobs, receivedBlockInfo)) case Failure(e) => jobScheduler.reportError("Error generating jobs for time " + time, e) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index 04e0a6a283cfb..d9ada99b472ac 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -100,14 +100,13 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { logInfo("Stopped JobScheduler") } - def runJobs(time: Time, jobs: Seq[Job]) { - if (jobs.isEmpty) { - logInfo("No jobs added for time " + time) + def submitJobSet(jobSet: JobSet) { + if (jobSet.jobs.isEmpty) { + logInfo("No jobs added for time " + jobSet.time) } else { - val jobSet = new JobSet(time, jobs) - jobSets.put(time, jobSet) + jobSets.put(jobSet.time, jobSet) jobSet.jobs.foreach(job => jobExecutor.execute(new JobHandler(job))) - logInfo("Added jobs for time " + time) + logInfo("Added jobs for time " + jobSet.time) } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala index fcf303aee6cd7..a69d74362173e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobSet.scala @@ -24,7 +24,11 @@ import org.apache.spark.streaming.Time * belong to the same batch. */ private[streaming] -case class JobSet(time: Time, jobs: Seq[Job]) { +case class JobSet( + time: Time, + jobs: Seq[Job], + receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]] = Map.empty + ) { private val incompleteJobs = new HashSet[Job]() private val submissionTime = System.currentTimeMillis() // when this jobset was submitted @@ -60,6 +64,7 @@ case class JobSet(time: Time, jobs: Seq[Job]) { def toBatchInfo: BatchInfo = { new BatchInfo( time, + receivedBlockInfo, submissionTime, if (processingStartTime >= 0 ) Some(processingStartTime) else None, if (processingEndTime >= 0 ) Some(processingEndTime) else None diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index 067e804202236..a1e6f5176825a 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -17,20 +17,42 @@ package org.apache.spark.streaming.scheduler -import scala.collection.mutable.{HashMap, Queue, SynchronizedMap} +import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} import akka.actor._ + import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.SparkContext._ -import org.apache.spark.storage.BlockId +import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.dstream.{NetworkReceiver, StopReceiver} import org.apache.spark.util.AkkaUtils +/** Information about receiver */ +case class ReceiverInfo(streamId: Int, typ: String, location: String) { + override def toString = s"$typ-$streamId" +} + +/** Information about blocks received by the network receiver */ +case class ReceivedBlockInfo( + streamId: Int, + blockId: StreamBlockId, + numRecords: Long, + metadata: Any + ) + +/** + * Messages used by the NetworkReceiver and the NetworkInputTracker to communicate + * with each other. + */ private[streaming] sealed trait NetworkInputTrackerMessage -private[streaming] case class RegisterReceiver(streamId: Int, receiverActor: ActorRef) - extends NetworkInputTrackerMessage -private[streaming] case class AddBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) +private[streaming] case class RegisterReceiver( + streamId: Int, + typ: String, + host: String, + receiverActor: ActorRef + ) extends NetworkInputTrackerMessage +private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) extends NetworkInputTrackerMessage private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) extends NetworkInputTrackerMessage @@ -47,9 +69,10 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) val receiverExecutor = new ReceiverExecutor() val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] - val receivedBlockIds = new HashMap[Int, Queue[BlockId]] with SynchronizedMap[Int, Queue[BlockId]] + val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] + with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] val timeout = AkkaUtils.askTimeout(ssc.conf) - + val listenerBus = ssc.scheduler.listenerBus // actor is created when generator starts. // This not being null means the tracker has been started and not stopped @@ -83,12 +106,32 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } } + /** Return all the blocks received from a receiver. */ + def getReceivedBlockInfo(streamId: Int): Array[ReceivedBlockInfo] = { + val receivedBlockInfo = getReceivedBlockInfoQueue(streamId).dequeueAll(x => true) + logInfo("Stream " + streamId + " received " + receivedBlockInfo.size + " blocks") + receivedBlockInfo.toArray + } + + private def getReceivedBlockInfoQueue(streamId: Int) = { + receivedBlockInfo.getOrElseUpdate(streamId, new SynchronizedQueue[ReceivedBlockInfo]) + } + /** Register a receiver */ - def registerReceiver(streamId: Int, receiverActor: ActorRef, sender: ActorRef) { + def registerReceiver( + streamId: Int, + typ: String, + host: String, + receiverActor: ActorRef, + sender: ActorRef + ) { if (!networkInputStreamMap.contains(streamId)) { throw new Exception("Register received for unexpected id " + streamId) } receiverInfo += ((streamId, receiverActor)) + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted( + ReceiverInfo(streamId, typ, host) + )) logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) } @@ -98,35 +141,26 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { logError("Deregistered receiver for network stream " + streamId + " with message:\n" + message) } - /** Get all the received blocks for the given stream. */ - def getBlocks(streamId: Int, time: Time): Array[BlockId] = { - val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]()) - val result = queue.dequeueAll(x => true).toArray - logInfo("Stream " + streamId + " received " + result.size + " blocks") - result - } - /** Add new blocks for the given stream */ - def addBlocks(streamId: Int, blockIds: Seq[BlockId], metadata: Any) = { - val queue = receivedBlockIds.getOrElseUpdate(streamId, new Queue[BlockId]) - queue ++= blockIds - networkInputStreamMap(streamId).addMetadata(metadata) - logDebug("Stream " + streamId + " received new blocks: " + blockIds.mkString("[", ", ", "]")) + def addBlocks(receivedBlockInfo: ReceivedBlockInfo) { + getReceivedBlockInfoQueue(receivedBlockInfo.streamId) += receivedBlockInfo + logDebug("Stream " + receivedBlockInfo.streamId + " received new blocks: " + + receivedBlockInfo.blockId) } /** Check if any blocks are left to be processed */ def hasMoreReceivedBlockIds: Boolean = { - !receivedBlockIds.forall(_._2.isEmpty) + !receivedBlockInfo.values.forall(_.isEmpty) } /** Actor to receive messages from the receivers. */ private class NetworkInputTrackerActor extends Actor { def receive = { - case RegisterReceiver(streamId, receiverActor) => - registerReceiver(streamId, receiverActor, sender) + case RegisterReceiver(streamId, typ, host, receiverActor) => + registerReceiver(streamId, typ, host, receiverActor, sender) sender ! true - case AddBlocks(streamId, blockIds, metadata) => - addBlocks(streamId, blockIds, metadata) + case AddBlock(receivedBlockInfo) => + addBlocks(receivedBlockInfo) case DeregisterReceiver(streamId, message) => deregisterReceiver(streamId, message) sender ! true diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 461ea3506477f..5db40ebbeb1de 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -23,8 +23,11 @@ import org.apache.spark.util.Distribution /** Base trait for events related to StreamingListener */ sealed trait StreamingListenerEvent +case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent +case class StreamingListenerReceiverStarted(receiverInfo: ReceiverInfo) + extends StreamingListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object StreamingListenerShutdown extends StreamingListenerEvent @@ -34,14 +37,17 @@ private[scheduler] case object StreamingListenerShutdown extends StreamingListen * computation. */ trait StreamingListener { - /** - * Called when processing of a batch has completed - */ + + /** Called when a receiver has been started */ + def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { } + + /** Called when a batch of jobs has been submitted for processing. */ + def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { } + + /** Called when processing of a batch of jobs has completed. */ def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } - /** - * Called when processing of a batch has started - */ + /** Called when processing of a batch of jobs has started. */ def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala index 18811fc2b01d8..ea03dfc7bfeea 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala @@ -38,6 +38,10 @@ private[spark] class StreamingListenerBus() extends Logging { while (true) { val event = eventQueue.take event match { + case receiverStarted: StreamingListenerReceiverStarted => + listeners.foreach(_.onReceiverStarted(receiverStarted)) + case batchSubmitted: StreamingListenerBatchSubmitted => + listeners.foreach(_.onBatchSubmitted(batchSubmitted)) case batchStarted: StreamingListenerBatchStarted => listeners.foreach(_.onBatchStarted(batchStarted)) case batchCompleted: StreamingListenerBatchCompleted => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala new file mode 100644 index 0000000000000..8b025b09ed34d --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -0,0 +1,148 @@ +/* + * 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.streaming.ui + +import org.apache.spark.streaming.{Time, StreamingContext} +import org.apache.spark.streaming.scheduler._ +import scala.collection.mutable.{Queue, HashMap} +import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted +import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted +import org.apache.spark.streaming.scheduler.BatchInfo +import org.apache.spark.streaming.scheduler.ReceiverInfo +import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted +import org.apache.spark.util.Distribution + + +private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends StreamingListener { + + private val waitingBatchInfos = new HashMap[Time, BatchInfo] + private val runningBatchInfos = new HashMap[Time, BatchInfo] + private val completedaBatchInfos = new Queue[BatchInfo] + private val batchInfoLimit = ssc.conf.getInt("spark.streaming.ui.retainedBatches", 100) + private var totalCompletedBatches = 0L + private val receiverInfos = new HashMap[Int, ReceiverInfo] + + val batchDuration = ssc.graph.batchDuration.milliseconds + + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) = { + synchronized { + receiverInfos.put(receiverStarted.receiverInfo.streamId, receiverStarted.receiverInfo) + } + } + + override def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) = synchronized { + runningBatchInfos(batchSubmitted.batchInfo.batchTime) = batchSubmitted.batchInfo + } + + override def onBatchStarted(batchStarted: StreamingListenerBatchStarted) = synchronized { + runningBatchInfos(batchStarted.batchInfo.batchTime) = batchStarted.batchInfo + waitingBatchInfos.remove(batchStarted.batchInfo.batchTime) + } + + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) = synchronized { + waitingBatchInfos.remove(batchCompleted.batchInfo.batchTime) + runningBatchInfos.remove(batchCompleted.batchInfo.batchTime) + completedaBatchInfos.enqueue(batchCompleted.batchInfo) + if (completedaBatchInfos.size > batchInfoLimit) completedaBatchInfos.dequeue() + totalCompletedBatches += 1L + } + + def numNetworkReceivers = synchronized { + ssc.graph.getNetworkInputStreams().size + } + + def numTotalCompletedBatches: Long = synchronized { + totalCompletedBatches + } + + def numUnprocessedBatches: Long = synchronized { + waitingBatchInfos.size + runningBatchInfos.size + } + + def waitingBatches: Seq[BatchInfo] = synchronized { + waitingBatchInfos.values.toSeq + } + + def runningBatches: Seq[BatchInfo] = synchronized { + runningBatchInfos.values.toSeq + } + + def retainedCompletedBatches: Seq[BatchInfo] = synchronized { + completedaBatchInfos.toSeq + } + + def processingDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.processingDelay) + } + + def schedulingDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.schedulingDelay) + } + + def totalDelayDistribution: Option[Distribution] = synchronized { + extractDistribution(_.totalDelay) + } + + def receivedRecordsDistributions: Map[Int, Option[Distribution]] = synchronized { + val latestBatchInfos = retainedBatches.reverse.take(batchInfoLimit) + val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) + (0 until numNetworkReceivers).map { receiverId => + val blockInfoOfParticularReceiver = latestBlockInfos.map { batchInfo => + batchInfo.get(receiverId).getOrElse(Array.empty) + } + val recordsOfParticularReceiver = blockInfoOfParticularReceiver.map { blockInfo => + // calculate records per second for each batch + blockInfo.map(_.numRecords).sum.toDouble * 1000 / batchDuration + } + val distributionOption = Distribution(recordsOfParticularReceiver) + (receiverId, distributionOption) + }.toMap + } + + def lastReceivedBatchRecords: Map[Int, Long] = { + val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo) + lastReceivedBlockInfoOption.map { lastReceivedBlockInfo => + (0 until numNetworkReceivers).map { receiverId => + (receiverId, lastReceivedBlockInfo(receiverId).map(_.numRecords).sum) + }.toMap + }.getOrElse { + (0 until numNetworkReceivers).map(receiverId => (receiverId, 0L)).toMap + } + } + + def receiverInfo(receiverId: Int): Option[ReceiverInfo] = { + receiverInfos.get(receiverId) + } + + def lastCompletedBatch: Option[BatchInfo] = { + completedaBatchInfos.sortBy(_.batchTime)(Time.ordering).lastOption + } + + def lastReceivedBatch: Option[BatchInfo] = { + retainedBatches.lastOption + } + + private def retainedBatches: Seq[BatchInfo] = synchronized { + (waitingBatchInfos.values.toSeq ++ + runningBatchInfos.values.toSeq ++ completedaBatchInfos).sortBy(_.batchTime)(Time.ordering) + } + + private def extractDistribution(getMetric: BatchInfo => Option[Long]): Option[Distribution] = { + Distribution(completedaBatchInfos.flatMap(getMetric(_)).map(_.toDouble)) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala new file mode 100644 index 0000000000000..6607437db560a --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -0,0 +1,180 @@ +/* + * 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.streaming.ui + +import java.util.Calendar +import javax.servlet.http.HttpServletRequest + +import scala.xml.Node + +import org.apache.spark.Logging +import org.apache.spark.ui._ +import org.apache.spark.ui.UIUtils._ +import org.apache.spark.util.Distribution + +/** Page for Spark Web UI that shows statistics of a streaming job */ +private[ui] class StreamingPage(parent: StreamingTab) + extends WebUIPage("") with Logging { + + private val listener = parent.listener + private val startTime = Calendar.getInstance().getTime() + private val emptyCell = "-" + + /** Render the page */ + def render(request: HttpServletRequest): Seq[Node] = { + val content = + generateBasicStats() ++

      ++ +

      Statistics over last {listener.retainedCompletedBatches.size} processed batches

      ++ + generateNetworkStatsTable() ++ + generateBatchStatsTable() + UIUtils.headerSparkPage( + content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000)) + } + + /** Generate basic stats of the streaming program */ + private def generateBasicStats(): Seq[Node] = { + val timeSinceStart = System.currentTimeMillis() - startTime.getTime +
        +
      • + Started at: {startTime.toString} +
      • +
      • + Time since start: {formatDurationVerbose(timeSinceStart)} +
      • +
      • + Network receivers: {listener.numNetworkReceivers} +
      • +
      • + Batch interval: {formatDurationVerbose(listener.batchDuration)} +
      • +
      • + Processed batches: {listener.numTotalCompletedBatches} +
      • +
      • + Waiting batches: {listener.numUnprocessedBatches} +
      • +
      + } + + /** Generate stats of data received over the network the streaming program */ + private def generateNetworkStatsTable(): Seq[Node] = { + val receivedRecordDistributions = listener.receivedRecordsDistributions + val lastBatchReceivedRecord = listener.lastReceivedBatchRecords + val table = if (receivedRecordDistributions.size > 0) { + val headerRow = Seq( + "Receiver", + "Location", + "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", + "Minimum rate\n[records/sec]", + "25th percentile rate\n[records/sec]", + "Median rate\n[records/sec]", + "75th percentile rate\n[records/sec]", + "Maximum rate\n[records/sec]" + ) + val dataRows = (0 until listener.numNetworkReceivers).map { receiverId => + val receiverInfo = listener.receiverInfo(receiverId) + val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") + val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) + val receiverLastBatchRecords = formatDurationVerbose(lastBatchReceivedRecord(receiverId)) + val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => + d.getQuantiles().map(r => formatDurationVerbose(r.toLong)) + }.getOrElse { + Seq(emptyCell, emptyCell, emptyCell, emptyCell, emptyCell) + } + Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats + } + Some(listingTable(headerRow, dataRows)) + } else { + None + } + + val content = +
      Network Input Statistics
      ++ +
      {table.getOrElse("No network receivers")}
      + + content + } + + /** Generate stats of batch jobs of the streaming program */ + private def generateBatchStatsTable(): Seq[Node] = { + val numBatches = listener.retainedCompletedBatches.size + val lastCompletedBatch = listener.lastCompletedBatch + val table = if (numBatches > 0) { + val processingDelayQuantilesRow = { + Seq( + "Processing Time", + formatDurationOption(lastCompletedBatch.flatMap(_.processingDelay)) + ) ++ getQuantiles(listener.processingDelayDistribution) + } + val schedulingDelayQuantilesRow = { + Seq( + "Scheduling Delay", + formatDurationOption(lastCompletedBatch.flatMap(_.schedulingDelay)) + ) ++ getQuantiles(listener.schedulingDelayDistribution) + } + val totalDelayQuantilesRow = { + Seq( + "Total Delay", + formatDurationOption(lastCompletedBatch.flatMap(_.totalDelay)) + ) ++ getQuantiles(listener.totalDelayDistribution) + } + val headerRow = Seq("Metric", "Last batch", "Minimum", "25th percentile", + "Median", "75th percentile", "Maximum") + val dataRows: Seq[Seq[String]] = Seq( + processingDelayQuantilesRow, + schedulingDelayQuantilesRow, + totalDelayQuantilesRow + ) + Some(listingTable(headerRow, dataRows)) + } else { + None + } + + val content = +
      Batch Processing Statistics
      ++ +
      +
        + {table.getOrElse("No statistics have been generated yet.")} +
      +
      + + content + } + + + /** + * Returns a human-readable string representing a duration such as "5 second 35 ms" + */ + private def formatDurationOption(msOption: Option[Long]): String = { + msOption.map(formatDurationVerbose).getOrElse(emptyCell) + } + + /** Get quantiles for any time distribution */ + private def getQuantiles(timeDistributionOption: Option[Distribution]) = { + timeDistributionOption.get.getQuantiles().map { ms => formatDurationVerbose(ms.toLong) } + } + + /** Generate HTML table from string data */ + private def listingTable(headers: Seq[String], data: Seq[Seq[String]]) = { + def generateDataRow(data: Seq[String]): Seq[Node] = { + {data.map(d => {d})} + } + UIUtils.listingTable(headers, generateDataRow, data, fixedWidth = true) + } +} + diff --git a/core/src/test/scala/org/apache/spark/SparkUISuite.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala similarity index 58% rename from core/src/test/scala/org/apache/spark/SparkUISuite.scala rename to streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala index d0d119c15081d..51448d15c6516 100644 --- a/core/src/test/scala/org/apache/spark/SparkUISuite.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingTab.scala @@ -15,21 +15,22 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.streaming.ui -import java.net.URI +import org.apache.spark.Logging +import org.apache.spark.streaming.StreamingContext +import org.apache.spark.ui.WebUITab -import org.scalatest.FunSuite +/** Spark Web UI tab that shows statistics of a streaming job */ +private[spark] class StreamingTab(ssc: StreamingContext) + extends WebUITab(ssc.sc.ui, "streaming") with Logging { -class SparkUISuite extends FunSuite with SharedSparkContext { + val parent = ssc.sc.ui + val appName = parent.appName + val basePath = parent.basePath + val listener = new StreamingJobProgressListener(ssc) - test("verify appUIAddress contains the scheme") { - val uiAddress = sc.ui.appUIAddress - assert(uiAddress.equals("http://" + sc.ui.appUIHostPort)) - } - - test("verify appUIAddress contains the port") { - val splitUIAddress = sc.ui.appUIAddress.split(':') - assert(splitUIAddress(2).toInt == sc.ui.boundPort) - } + ssc.addStreamingListener(listener) + attachPage(new StreamingPage(this)) + parent.attachTab(this) } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 389b23d4d5e4b..952511d411a8e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -239,11 +239,11 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { /** This is a server to test the network input stream */ -class TestServer() extends Logging { +class TestServer(portToBind: Int = 0) extends Logging { val queue = new ArrayBlockingQueue[String](100) - val serverSocket = new ServerSocket(0) + val serverSocket = new ServerSocket(portToBind) val servingThread = new Thread() { override def run() { @@ -282,7 +282,7 @@ class TestServer() extends Logging { def start() { servingThread.start() } - def send(msg: String) { queue.add(msg) } + def send(msg: String) { queue.put(msg) } def stop() { servingThread.interrupt() } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 9cc27ef7f03b5..efd0d22ecb57a 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -161,7 +161,6 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w } } - test("stop only streaming context") { ssc = new StreamingContext(master, appName, batchDuration) sc = ssc.sparkContext diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala new file mode 100644 index 0000000000000..35538ec188f67 --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.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.streaming + +import scala.io.Source + +import org.scalatest.FunSuite +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +class UISuite extends FunSuite { + + test("streaming tab in spark UI") { + val ssc = new StreamingContext("local", "test", Seconds(1)) + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL(ssc.sparkContext.ui.appUIAddress).mkString + assert(!html.contains("random data that should not be present")) + // test if streaming tab exist + assert(html.toLowerCase.contains("streaming")) + // test if other Spark tabs still exist + assert(html.toLowerCase.contains("stages")) + } + + eventually(timeout(10 seconds), interval(50 milliseconds)) { + val html = Source.fromURL( + ssc.sparkContext.ui.appUIAddress.stripSuffix("/") + "/streaming").mkString + assert(html.toLowerCase.contains("batch")) + assert(html.toLowerCase.contains("network")) + } + } +} From c2d160fbee2ef90a7683d9771f2f632b68d74aef Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Sat, 12 Apr 2014 16:33:38 -0700 Subject: [PATCH 269/397] [Fix #204] Update out-dated comments This PR is self-explanatory. Author: Andrew Or Closes #381 from andrewor14/master and squashes the following commits: 3e8dde2 [Andrew Or] Fix comments for #204 --- core/src/main/scala/org/apache/spark/SparkContext.scala | 6 +----- .../org/apache/spark/scheduler/ReplayListenerBus.scala | 4 ++-- 2 files changed, 3 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index a764c174d562c..5a36e6f5c19a9 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1191,11 +1191,7 @@ class SparkContext(config: SparkConf) extends Logging { listenerBus.post(SparkListenerApplicationStart(appName, startTime, sparkUser)) } - /** - * Post the application end event to all listeners immediately, rather than adding it - * to the event queue for it to be asynchronously processed eventually. Otherwise, a race - * condition exists in which the listeners may stop before this event has been propagated. - */ + /** Post the application end event */ private def postApplicationEnd() { listenerBus.post(SparkListenerApplicationEnd(System.currentTimeMillis)) } diff --git a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala index f868e772cf58a..f89724d4ea196 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ReplayListenerBus.scala @@ -31,8 +31,8 @@ import org.apache.spark.util.JsonProtocol /** * A SparkListenerBus that replays logged events from persisted storage. * - * This class expects files to be appropriately prefixed as specified in EventLoggingListener. - * There exists a one-to-one mapping between ReplayListenerBus and event logging applications. + * This assumes the given paths are valid log files, where each line can be deserialized into + * exactly one SparkListenerEvent. */ private[spark] class ReplayListenerBus( logPaths: Seq[Path], From ca11919e6e97a62eb3e3ce882ffa29eae36f50f7 Mon Sep 17 00:00:00 2001 From: Bharath Bhushan Date: Sat, 12 Apr 2014 20:52:29 -0700 Subject: [PATCH 270/397] [SPARK-1403] Move the class loader creation back to where it was in 0.9.0 [SPARK-1403] I investigated why spark 0.9.0 loads fine on mesos while spark 1.0.0 fails. What I found was that in SparkEnv.scala, while creating the SparkEnv object, the current thread's classloader is null. But in 0.9.0, at the same place, it is set to org.apache.spark.repl.ExecutorClassLoader . I saw that https://github.com/apache/spark/commit/7edbea41b43e0dc11a2de156be220db8b7952d01 moved it to it current place. I moved it back and saw that 1.0.0 started working fine on mesos. I just created a minimal patch that allows me to run spark on mesos correctly. It seems like SecurityManager's creation needs to be taken into account for a correct fix. Also moving the creation of the serializer out of SparkEnv might be a part of the right solution. PTAL. Author: Bharath Bhushan Closes #322 from manku-timma/spark-1403 and squashes the following commits: 606c2b9 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 ec8f870 [Bharath Bhushan] revert the logger change for java 6 compatibility as PR 334 is doing it 728beca [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 044027d [Bharath Bhushan] fix compile error 6f260a4 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 b3a053f [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 04b9662 [Bharath Bhushan] add missing line 4803c19 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 f3c9a14 [Bharath Bhushan] Merge remote-tracking branch 'upstream/master' into spark-1403 42d3d6a [Bharath Bhushan] used code fragment from @ueshin to fix the problem in a better way 89109d7 [Bharath Bhushan] move the class loader creation back to where it was in 0.9.0 --- .../spark/executor/MesosExecutorBackend.scala | 22 +++++++++++++------ 1 file changed, 15 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 6fc702fdb1512..df36a06485c77 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -50,13 +50,21 @@ private[spark] class MesosExecutorBackend executorInfo: ExecutorInfo, frameworkInfo: FrameworkInfo, slaveInfo: SlaveInfo) { - logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) - this.driver = driver - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) - executor = new Executor( - executorInfo.getExecutorId.getValue, - slaveInfo.getHostname, - properties) + val cl = Thread.currentThread.getContextClassLoader + try { + // Work around for SPARK-1480 + Thread.currentThread.setContextClassLoader(getClass.getClassLoader) + logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) + this.driver = driver + val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) + executor = new Executor( + executorInfo.getExecutorId.getValue, + slaveInfo.getHostname, + properties) + } finally { + // Work around for SPARK-1480 + Thread.currentThread.setContextClassLoader(cl) + } } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { From 4bc07eebbf5e2ea0c0b6f1642049515025d88d07 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 13 Apr 2014 08:58:37 -0700 Subject: [PATCH 271/397] SPARK-1480: Clean up use of classloaders The Spark codebase is a bit fast-and-loose when accessing classloaders and this has caused a few bugs to surface in master. This patch defines some utility methods for accessing classloaders. This makes the intention when accessing a classloader much more explicit in the code and fixes a few cases where the wrong one was chosen. case (a) -> We want the classloader that loaded Spark case (b) -> We want the context class loader, or if not present, we want (a) This patch provides a better fix for SPARK-1403 (https://issues.apache.org/jira/browse/SPARK-1403) than the current work around, which it reverts. It also fixes a previously unreported bug that the `./spark-submit` script did not work for running with `local` master. It didn't work because the executor classloader did not properly delegate to the context class loader (if it is defined) and in local mode the context class loader is set by the `./spark-submit` script. A unit test is added for that case. Author: Patrick Wendell Closes #398 from pwendell/class-loaders and squashes the following commits: b4a1a58 [Patrick Wendell] Minor clean up 14f1272 [Patrick Wendell] SPARK-1480: Clean up use of classloaders --- .../main/scala/org/apache/spark/Logging.scala | 4 +-- .../org/apache/spark/executor/Executor.scala | 6 ++-- .../spark/executor/MesosExecutorBackend.scala | 22 ++++--------- .../apache/spark/metrics/MetricsConfig.scala | 3 +- .../apache/spark/scheduler/ResultTask.scala | 1 - .../spark/scheduler/SchedulableBuilder.scala | 3 +- .../spark/scheduler/TaskResultGetter.scala | 4 +-- .../spark/serializer/JavaSerializer.scala | 3 +- .../org/apache/spark/ui/JettyUtils.scala | 3 +- .../scala/org/apache/spark/util/Utils.scala | 15 +++++++++ .../ExecutorURLClassLoaderSuite.scala | 32 +++++++++++++++++-- .../org/apache/spark/repl/SparkILoop.scala | 7 ++-- .../spark/sql/catalyst/util/package.scala | 4 ++- .../compression/compressionSchemes.scala | 3 +- .../sql/execution/SparkSqlSerializer.scala | 3 +- 15 files changed, 78 insertions(+), 35 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Logging.scala b/core/src/main/scala/org/apache/spark/Logging.scala index 9d429dceeb858..50d8e93e1f0d7 100644 --- a/core/src/main/scala/org/apache/spark/Logging.scala +++ b/core/src/main/scala/org/apache/spark/Logging.scala @@ -22,6 +22,7 @@ import org.slf4j.{Logger, LoggerFactory} import org.slf4j.impl.StaticLoggerBinder import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.util.Utils /** * :: DeveloperApi :: @@ -115,8 +116,7 @@ trait Logging { val log4jInitialized = LogManager.getRootLogger.getAllAppenders.hasMoreElements if (!log4jInitialized && usingLog4j) { val defaultLogProps = "org/apache/spark/log4j-defaults.properties" - val classLoader = this.getClass.getClassLoader - Option(classLoader.getResource(defaultLogProps)) match { + Option(Utils.getSparkClassLoader.getResource(defaultLogProps)) match { case Some(url) => PropertyConfigurator.configure(url) log.info(s"Using Spark's default log4j profile: $defaultLogProps") diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index c12bd922d40e4..f89b2bffd1676 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -292,7 +292,7 @@ private[spark] class Executor( * created by the interpreter to the search path */ private def createClassLoader(): MutableURLClassLoader = { - val loader = this.getClass.getClassLoader + val currentLoader = Utils.getContextOrSparkClassLoader // For each of the jars in the jarSet, add them to the class loader. // We assume each of the files has already been fetched. @@ -301,8 +301,8 @@ private[spark] class Executor( }.toArray val userClassPathFirst = conf.getBoolean("spark.files.userClassPathFirst", false) userClassPathFirst match { - case true => new ChildExecutorURLClassLoader(urls, loader) - case false => new ExecutorURLClassLoader(urls, loader) + case true => new ChildExecutorURLClassLoader(urls, currentLoader) + case false => new ExecutorURLClassLoader(urls, currentLoader) } } diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index df36a06485c77..6fc702fdb1512 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -50,21 +50,13 @@ private[spark] class MesosExecutorBackend executorInfo: ExecutorInfo, frameworkInfo: FrameworkInfo, slaveInfo: SlaveInfo) { - val cl = Thread.currentThread.getContextClassLoader - try { - // Work around for SPARK-1480 - Thread.currentThread.setContextClassLoader(getClass.getClassLoader) - logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) - this.driver = driver - val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) - executor = new Executor( - executorInfo.getExecutorId.getValue, - slaveInfo.getHostname, - properties) - } finally { - // Work around for SPARK-1480 - Thread.currentThread.setContextClassLoader(cl) - } + logInfo("Registered with Mesos as executor ID " + executorInfo.getExecutorId.getValue) + this.driver = driver + val properties = Utils.deserialize[Array[(String, String)]](executorInfo.getData.toByteArray) + executor = new Executor( + executorInfo.getExecutorId.getValue, + slaveInfo.getHostname, + properties) } override def launchTask(d: ExecutorDriver, taskInfo: TaskInfo) { diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala index 3e3e18c3537d0..1b7a5d1f1980a 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsConfig.scala @@ -24,6 +24,7 @@ import scala.collection.mutable import scala.util.matching.Regex import org.apache.spark.Logging +import org.apache.spark.util.Utils private[spark] class MetricsConfig(val configFile: Option[String]) extends Logging { @@ -50,7 +51,7 @@ private[spark] class MetricsConfig(val configFile: Option[String]) extends Loggi try { is = configFile match { case Some(f) => new FileInputStream(f) - case None => getClass.getClassLoader.getResourceAsStream(METRICS_CONF) + case None => Utils.getSparkClassLoader.getResourceAsStream(METRICS_CONF) } if (is != null) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 083fb895d8696..0b381308b61ff 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -54,7 +54,6 @@ private[spark] object ResultTask { def deserializeInfo(stageId: Int, bytes: Array[Byte]): (RDD[_], (TaskContext, Iterator[_]) => _) = { - val loader = Thread.currentThread.getContextClassLoader val in = new GZIPInputStream(new ByteArrayInputStream(bytes)) val ser = SparkEnv.get.closureSerializer.newInstance() val objIn = ser.deserializeStream(in) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala index e4eced383c3a5..6c5827f75e636 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulableBuilder.scala @@ -23,6 +23,7 @@ import java.util.{NoSuchElementException, Properties} import scala.xml.XML import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.util.Utils /** * An interface to build Schedulable tree @@ -72,7 +73,7 @@ private[spark] class FairSchedulableBuilder(val rootPool: Pool, conf: SparkConf) schedulerAllocFile.map { f => new FileInputStream(f) }.getOrElse { - getClass.getClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) + Utils.getSparkClassLoader.getResourceAsStream(DEFAULT_SCHEDULER_FILE) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala index cb4ad4ae9350c..c9ad2b151daf0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskResultGetter.scala @@ -85,13 +85,13 @@ private[spark] class TaskResultGetter(sparkEnv: SparkEnv, scheduler: TaskSchedul try { if (serializedData != null && serializedData.limit() > 0) { reason = serializer.get().deserialize[TaskEndReason]( - serializedData, getClass.getClassLoader) + serializedData, Utils.getSparkClassLoader) } } catch { case cnd: ClassNotFoundException => // Log an error but keep going here -- the task failed, so not catastropic if we can't // deserialize the reason. - val loader = Thread.currentThread.getContextClassLoader + val loader = Utils.getContextOrSparkClassLoader logError( "Could not deserialize TaskEndReason: ClassNotFound with classloader " + loader) case ex: Throwable => {} diff --git a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala index 5e5883554fcc1..e9163deaf2036 100644 --- a/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/JavaSerializer.scala @@ -23,6 +23,7 @@ import java.nio.ByteBuffer import org.apache.spark.SparkConf import org.apache.spark.annotation.DeveloperApi import org.apache.spark.util.ByteBufferInputStream +import org.apache.spark.util.Utils private[spark] class JavaSerializationStream(out: OutputStream, counterReset: Int) extends SerializationStream { @@ -86,7 +87,7 @@ private[spark] class JavaSerializerInstance(counterReset: Int) extends Serialize } def deserializeStream(s: InputStream): DeserializationStream = { - new JavaDeserializationStream(s, Thread.currentThread.getContextClassLoader) + new JavaDeserializationStream(s, Utils.getContextOrSparkClassLoader) } def deserializeStream(s: InputStream, loader: ClassLoader): DeserializationStream = { diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 62a4e3d0f6a42..3ae147a36c8a4 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -33,6 +33,7 @@ import org.json4s.JValue import org.json4s.jackson.JsonMethods.{pretty, render} import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.util.Utils /** * Utilities for launching a web server using Jetty's HTTP Server class @@ -124,7 +125,7 @@ private[spark] object JettyUtils extends Logging { contextHandler.setInitParameter("org.eclipse.jetty.servlet.Default.gzip", "false") val staticHandler = new DefaultServlet val holder = new ServletHolder(staticHandler) - Option(getClass.getClassLoader.getResource(resourceBase)) match { + Option(Utils.getSparkClassLoader.getResource(resourceBase)) match { case Some(res) => holder.setInitParameter("resourceBase", res.toString) case None => diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 166f48ce7342e..a3af4e7b91692 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -116,6 +116,21 @@ private[spark] object Utils extends Logging { } } + /** + * Get the ClassLoader which loaded Spark. + */ + def getSparkClassLoader = getClass.getClassLoader + + /** + * Get the Context ClassLoader on this thread or, if not present, the ClassLoader that + * loaded Spark. + * + * This should be used whenever passing a ClassLoader to Class.ForName or finding the currently + * active loader when setting up ClassLoader delegation chains. + */ + def getContextOrSparkClassLoader = + Option(Thread.currentThread().getContextClassLoader).getOrElse(getSparkClassLoader) + /** * Primitive often used when writing {@link java.nio.ByteBuffer} to {@link java.io.DataOutput}. */ diff --git a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala index c40cfc0696fce..e2050e95a1b88 100644 --- a/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/executor/ExecutorURLClassLoaderSuite.scala @@ -17,12 +17,12 @@ package org.apache.spark.executor -import java.io.File import java.net.URLClassLoader import org.scalatest.FunSuite -import org.apache.spark.TestUtils +import org.apache.spark.{LocalSparkContext, SparkContext, SparkException, TestUtils} +import org.apache.spark.util.Utils class ExecutorURLClassLoaderSuite extends FunSuite { @@ -63,5 +63,33 @@ class ExecutorURLClassLoaderSuite extends FunSuite { } } + test("driver sets context class loader in local mode") { + // Test the case where the driver program sets a context classloader and then runs a job + // in local mode. This is what happens when ./spark-submit is called with "local" as the + // master. + val original = Thread.currentThread().getContextClassLoader + val className = "ClassForDriverTest" + val jar = TestUtils.createJarWithClasses(Seq(className)) + val contextLoader = new URLClassLoader(Array(jar), Utils.getContextOrSparkClassLoader) + Thread.currentThread().setContextClassLoader(contextLoader) + + val sc = new SparkContext("local", "driverLoaderTest") + + try { + sc.makeRDD(1 to 5, 2).mapPartitions { x => + val loader = Thread.currentThread().getContextClassLoader + Class.forName(className, true, loader).newInstance() + Seq().iterator + }.count() + } + catch { + case e: SparkException if e.getMessage.contains("ClassNotFoundException") => + fail("Local executor could not find class", e) + case t: Throwable => fail("Unexpected exception ", t) + } + + sc.stop() + Thread.currentThread().setContextClassLoader(original) + } } diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 5a367b6bb79de..beb40e87024bd 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -39,6 +39,7 @@ import scala.reflect.api.{Mirror, TypeCreator, Universe => ApiUniverse} import org.apache.spark.Logging import org.apache.spark.SparkConf import org.apache.spark.SparkContext +import org.apache.spark.util.Utils /** The Scala interactive shell. It provides a read-eval-print loop * around the Interpreter class. @@ -130,7 +131,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, def history = in.history /** The context class loader at the time this object was created */ - protected val originalClassLoader = Thread.currentThread.getContextClassLoader + protected val originalClassLoader = Utils.getContextOrSparkClassLoader // classpath entries added via :cp var addedClasspath: String = "" @@ -177,7 +178,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, override lazy val formatting = new Formatting { def prompt = SparkILoop.this.prompt } - override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) + override protected def parentClassLoader = SparkHelper.explicitParentLoader(settings).getOrElse(classOf[SparkILoop].getClassLoader) } /** Create a new interpreter. */ @@ -871,7 +872,7 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, } val u: scala.reflect.runtime.universe.type = scala.reflect.runtime.universe - val m = u.runtimeMirror(getClass.getClassLoader) + val m = u.runtimeMirror(Utils.getSparkClassLoader) private def tagOfStaticClass[T: ClassTag]: u.TypeTag[T] = u.TypeTag[T]( m, 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 a001d953592db..49fc4f70fdfae 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 @@ -19,6 +19,8 @@ package org.apache.spark.sql.catalyst import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File} +import org.apache.spark.util.{Utils => SparkUtils} + package object util { /** * Returns a path to a temporary file that probably does not exist. @@ -54,7 +56,7 @@ package object util { def resourceToString( resource:String, encoding: String = "UTF-8", - classLoader: ClassLoader = this.getClass.getClassLoader) = { + classLoader: ClassLoader = SparkUtils.getSparkClassLoader) = { val inStream = classLoader.getResourceAsStream(resource) val outStream = new ByteArrayOutputStream try { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index df8220b556edd..e92cf5ac4f9df 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -26,6 +26,7 @@ import scala.reflect.runtime.universe.runtimeMirror import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.columnar._ +import org.apache.spark.util.Utils private[sql] case object PassThrough extends CompressionScheme { override val typeId = 0 @@ -254,7 +255,7 @@ private[sql] case object DictionaryEncoding extends CompressionScheme { private val dictionary = { // TODO Can we clean up this mess? Maybe move this to `DataType`? implicit val classTag = { - val mirror = runtimeMirror(getClass.getClassLoader) + val mirror = runtimeMirror(Utils.getSparkClassLoader) ClassTag[T#JvmType](mirror.runtimeClass(columnType.scalaTag.tpe)) } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index d8e1b970c1d88..c30ae5bcc02d0 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -25,6 +25,7 @@ import com.esotericsoftware.kryo.{Serializer, Kryo} import org.apache.spark.{SparkEnv, SparkConf} import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair +import org.apache.spark.util.Utils class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { @@ -44,7 +45,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { kryo.register(classOf[scala.collection.mutable.ArrayBuffer[_]]) kryo.register(classOf[scala.math.BigDecimal], new BigDecimalSerializer) kryo.setReferences(false) - kryo.setClassLoader(this.getClass.getClassLoader) + kryo.setClassLoader(Utils.getSparkClassLoader) kryo } } From 037fe4d2ba01be5610baa3dd9c5c9d3a5e5e1064 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Sun, 13 Apr 2014 13:18:52 -0700 Subject: [PATCH 272/397] [SPARK-1415] Hadoop min split for wholeTextFiles() JIRA issue [here](https://issues.apache.org/jira/browse/SPARK-1415). New Hadoop API of `InputFormat` does not provide the `minSplits` parameter, which makes the API incompatible between `HadoopRDD` and `NewHadoopRDD`. The PR is for constructing compatible APIs. Though `minSplits` is deprecated by New Hadoop API, we think it is better to make APIs compatible here. **Note** that `minSplits` in `wholeTextFiles` could only be treated as a *suggestion*, the real number of splits may not be greater than `minSplits` due to `isSplitable()=false`. Author: Xusen Yin Closes #376 from yinxusen/hadoop-min-split and squashes the following commits: 76417f6 [Xusen Yin] refine comments c10af60 [Xusen Yin] refine comments and rewrite new class for wholeTextFile 766d05b [Xusen Yin] refine Java API and comments 4875755 [Xusen Yin] add minSplits for WholeTextFiles --- .../scala/org/apache/spark/SparkContext.scala | 17 ++++-- .../spark/api/java/JavaSparkContext.scala | 14 ++++- .../input/WholeTextFileInputFormat.scala | 14 +++++ .../org/apache/spark/rdd/NewHadoopRDD.scala | 60 +++++++++++++++---- .../java/org/apache/spark/JavaAPISuite.java | 2 +- .../WholeTextFileRecordReaderSuite.scala | 2 +- 6 files changed, 90 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 5a36e6f5c19a9..456070fa7c5ef 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -454,14 +454,21 @@ class SparkContext(config: SparkConf) extends Logging { * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are preferred, as each file will be loaded fully in memory. + * @note Small files are preferred, large file is also allowable, but may cause bad performance. + * + * @param minSplits A suggestion value of the minimal splitting number for input data. */ - def wholeTextFiles(path: String): RDD[(String, String)] = { - newAPIHadoopFile( - path, + def wholeTextFiles(path: String, minSplits: Int = defaultMinSplits): RDD[(String, String)] = { + val job = new NewHadoopJob(hadoopConfiguration) + NewFileInputFormat.addInputPath(job, new Path(path)) + val updateConf = job.getConfiguration + new WholeTextFileRDD( + this, classOf[WholeTextFileInputFormat], classOf[String], - classOf[String]) + classOf[String], + updateConf, + minSplits) } /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 1e8242a2cbbce..7fbefe1cb0fb1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -177,7 +177,19 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * (a-hdfs-path/part-nnnnn, its content) * }}} * - * @note Small files are preferred, as each file will be loaded fully in memory. + * @note Small files are preferred, large file is also allowable, but may cause bad performance. + * + * @param minSplits A suggestion value of the minimal splitting number for input data. + */ + def wholeTextFiles(path: String, minSplits: Int): JavaPairRDD[String, String] = + new JavaPairRDD(sc.wholeTextFiles(path, minSplits)) + + /** + * Read a directory of text files from HDFS, a local file system (available on all nodes), or any + * Hadoop-supported file system URI. Each file is read as a single record and returned in a + * key-value pair, where the key is the path of each file, the value is the content of each file. + * + * @see `wholeTextFiles(path: String, minSplits: Int)`. */ def wholeTextFiles(path: String): JavaPairRDD[String, String] = new JavaPairRDD(sc.wholeTextFiles(path)) diff --git a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala index 4887fb6b84eb2..80d055a89573b 100644 --- a/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala +++ b/core/src/main/scala/org/apache/spark/input/WholeTextFileInputFormat.scala @@ -17,6 +17,8 @@ package org.apache.spark.input +import scala.collection.JavaConversions._ + import org.apache.hadoop.fs.Path import org.apache.hadoop.mapreduce.InputSplit import org.apache.hadoop.mapreduce.JobContext @@ -44,4 +46,16 @@ private[spark] class WholeTextFileInputFormat extends CombineFileInputFormat[Str context, classOf[WholeTextFileRecordReader]) } + + /** + * Allow minSplits set by end-user in order to keep compatibility with old Hadoop API. + */ + def setMaxSplitSize(context: JobContext, minSplits: Int) { + val files = listStatus(context) + val totalLen = files.map { file => + if (file.isDir) 0L else file.getLen + }.sum + val maxSplitSize = Math.ceil(totalLen * 1.0 / (if (minSplits == 0) 1 else minSplits)).toLong + super.setMaxSplitSize(maxSplitSize) + } } diff --git a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala index 2d8dfa5a1645a..8684b645bc361 100644 --- a/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/NewHadoopRDD.scala @@ -24,11 +24,18 @@ import org.apache.hadoop.conf.{Configurable, Configuration} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ -import org.apache.spark.{InterruptibleIterator, Logging, Partition, SerializableWritable, SparkContext, TaskContext} import org.apache.spark.annotation.DeveloperApi - -private[spark] -class NewHadoopPartition(rddId: Int, val index: Int, @transient rawSplit: InputSplit with Writable) +import org.apache.spark.input.WholeTextFileInputFormat +import org.apache.spark.InterruptibleIterator +import org.apache.spark.Logging +import org.apache.spark.Partition +import org.apache.spark.SerializableWritable +import org.apache.spark.{SparkContext, TaskContext} + +private[spark] class NewHadoopPartition( + rddId: Int, + val index: Int, + @transient rawSplit: InputSplit with Writable) extends Partition { val serializableHadoopSplit = new SerializableWritable(rawSplit) @@ -65,17 +72,19 @@ class NewHadoopRDD[K, V]( private val confBroadcast = sc.broadcast(new SerializableWritable(conf)) // private val serializableConf = new SerializableWritable(conf) - private val jobtrackerId: String = { + private val jobTrackerId: String = { val formatter = new SimpleDateFormat("yyyyMMddHHmm") formatter.format(new Date()) } - @transient private val jobId = new JobID(jobtrackerId, id) + @transient protected val jobId = new JobID(jobTrackerId, id) override def getPartitions: Array[Partition] = { val inputFormat = inputFormatClass.newInstance - if (inputFormat.isInstanceOf[Configurable]) { - inputFormat.asInstanceOf[Configurable].setConf(conf) + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => } val jobContext = newJobContext(conf, jobId) val rawSplits = inputFormat.getSplits(jobContext).toArray @@ -91,11 +100,13 @@ class NewHadoopRDD[K, V]( val split = theSplit.asInstanceOf[NewHadoopPartition] logInfo("Input split: " + split.serializableHadoopSplit) val conf = confBroadcast.value.value - val attemptId = newTaskAttemptID(jobtrackerId, id, isMap = true, split.index, 0) + val attemptId = newTaskAttemptID(jobTrackerId, id, isMap = true, split.index, 0) val hadoopAttemptContext = newTaskAttemptContext(conf, attemptId) val format = inputFormatClass.newInstance - if (format.isInstanceOf[Configurable]) { - format.asInstanceOf[Configurable].setConf(conf) + format match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => } val reader = format.createRecordReader( split.serializableHadoopSplit.value, hadoopAttemptContext) @@ -141,3 +152,30 @@ class NewHadoopRDD[K, V]( def getConf: Configuration = confBroadcast.value.value } +private[spark] class WholeTextFileRDD( + sc : SparkContext, + inputFormatClass: Class[_ <: WholeTextFileInputFormat], + keyClass: Class[String], + valueClass: Class[String], + @transient conf: Configuration, + minSplits: Int) + extends NewHadoopRDD[String, String](sc, inputFormatClass, keyClass, valueClass, conf) { + + override def getPartitions: Array[Partition] = { + val inputFormat = inputFormatClass.newInstance + inputFormat match { + case configurable: Configurable => + configurable.setConf(conf) + case _ => + } + val jobContext = newJobContext(conf, jobId) + inputFormat.setMaxSplitSize(jobContext, minSplits) + val rawSplits = inputFormat.getSplits(jobContext).toArray + val result = new Array[Partition](rawSplits.size) + for (i <- 0 until rawSplits.size) { + result(i) = new NewHadoopPartition(id, i, rawSplits(i).asInstanceOf[InputSplit with Writable]) + } + result + } +} + diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index ab2fdac553349..8d2e9f1846343 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -626,7 +626,7 @@ public void wholeTextFiles() throws IOException { container.put(tempDirName+"/part-00000", new Text(content1).toString()); container.put(tempDirName+"/part-00001", new Text(content2).toString()); - JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName); + JavaPairRDD readRDD = sc.wholeTextFiles(tempDirName, 3); List> result = readRDD.collect(); for (Tuple2 res : result) { diff --git a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala index e89b296d41026..33d6de9a76405 100644 --- a/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala +++ b/core/src/test/scala/org/apache/spark/input/WholeTextFileRecordReaderSuite.scala @@ -73,7 +73,7 @@ class WholeTextFileRecordReaderSuite extends FunSuite with BeforeAndAfterAll { createNativeFile(dir, filename, contents) } - val res = sc.wholeTextFiles(dir.toString).collect() + val res = sc.wholeTextFiles(dir.toString, 3).collect() assert(res.size === WholeTextFileRecordReaderSuite.fileNames.size, "Number of files read out does not fit with the actual value.") From 7dbca68e92416ec5f023c8807bb06470c01a6d3a Mon Sep 17 00:00:00 2001 From: Cheng Lian Date: Mon, 14 Apr 2014 15:22:43 -0700 Subject: [PATCH 273/397] [BUGFIX] In-memory columnar storage bug fixes Fixed several bugs of in-memory columnar storage to make `HiveInMemoryCompatibilitySuite` pass. @rxin @marmbrus It is reasonable to include `HiveInMemoryCompatibilitySuite` in this PR, but I didn't, since it significantly increases test execution time. What do you think? **UPDATE** `HiveCompatibilitySuite` has been made to cache tables in memory. `HiveInMemoryCompatibilitySuite` was removed. Author: Cheng Lian Author: Michael Armbrust Closes #374 from liancheng/inMemBugFix and squashes the following commits: 6ad6d9b [Cheng Lian] Merged HiveCompatibilitySuite and HiveInMemoryCompatibilitySuite 5bdbfe7 [Cheng Lian] Revert 882c538 & 8426ddc, which introduced regression 882c538 [Cheng Lian] Remove attributes field from InMemoryColumnarTableScan 32cc9ce [Cheng Lian] Code style cleanup 99382bf [Cheng Lian] Enable compression by default 4390bcc [Cheng Lian] Report error for any Throwable in HiveComparisonTest d1df4fd [Michael Armbrust] Remove test tables that might always get created anyway? ab9e807 [Michael Armbrust] Fix the logged console version of failed test cases to use the new syntax. 1965123 [Michael Armbrust] Don't use coalesce for gathering all data to a single partition, as it does not work correctly with mutable rows. e36cdd0 [Michael Armbrust] Spelling. 2d0e168 [Michael Armbrust] Run Hive tests in-memory too. 6360723 [Cheng Lian] Made PreInsertionCasts support SparkLogicalPlan and InMemoryColumnarTableScan c9b0f6f [Cheng Lian] Let InsertIntoTable support InMemoryColumnarTableScan 9c8fc40 [Cheng Lian] Disable compression by default e619995 [Cheng Lian] Bug fix: incorrect byte order in CompressionScheme.columnHeaderSize 8426ddc [Cheng Lian] Bug fix: InMemoryColumnarTableScan should cache columns specified by the attributes argument 036cd09 [Cheng Lian] Clean up unused imports 44591a5 [Cheng Lian] Bug fix: NullableColumnAccessor.hasNext must take nulls into account 052bf41 [Cheng Lian] Bug fix: should only gather compressibility info for non-null values 95b3301 [Cheng Lian] Fixed bugs in IntegralDelta --- .../org/apache/spark/sql/SchemaRDD.scala | 2 +- .../sql/columnar/NullableColumnAccessor.scala | 2 + .../CompressibleColumnBuilder.scala | 4 +- .../compression/CompressionScheme.scala | 4 +- .../compression/compressionSchemes.scala | 20 ++++----- .../apache/spark/sql/execution/Exchange.scala | 9 +++- .../spark/sql/execution/SparkPlan.scala | 4 +- .../apache/spark/sql/CachedTableSuite.scala | 5 +-- ...scala => InMemoryColumnarQuerySuite.scala} | 12 +++++- .../NullableColumnAccessorSuite.scala | 4 ++ .../compression/IntegralDeltaSuite.scala | 15 +++++-- .../spark/sql/hive/HiveMetastoreCatalog.scala | 42 ++++++++++++------- .../spark/sql/hive/HiveStrategies.scala | 4 ++ .../org/apache/spark/sql/hive/TestHive.scala | 10 ++--- .../org/apache/spark/sql/hive/hiveUdfs.scala | 16 +++---- .../hive/execution/HiveComparisonTest.scala | 10 +---- .../execution/HiveCompatibilitySuite.scala | 12 +++++- 17 files changed, 109 insertions(+), 66 deletions(-) rename sql/core/src/test/scala/org/apache/spark/sql/columnar/{ColumnarQuerySuite.scala => InMemoryColumnarQuerySuite.scala} (79%) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 16da7fd92bffe..91500416eefaa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -99,7 +99,7 @@ class SchemaRDD( def baseSchemaRDD = this // ========================================================================================= - // RDD functions: Copy the interal row representation so we present immutable data to users. + // RDD functions: Copy the internal row representation so we present immutable data to users. // ========================================================================================= override def compute(split: Partition, context: TaskContext): Iterator[Row] = diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala index 7d49ab07f7a53..b7f8826861a2c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnAccessor.scala @@ -54,4 +54,6 @@ private[sql] trait NullableColumnAccessor extends ColumnAccessor { pos += 1 } + + abstract override def hasNext = seenNulls < nullCount || super.hasNext } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala index fd3b1adf9687a..0f808f68f2eec 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressibleColumnBuilder.scala @@ -65,7 +65,9 @@ private[sql] trait CompressibleColumnBuilder[T <: NativeType] abstract override def appendFrom(row: Row, ordinal: Int) { super.appendFrom(row, ordinal) - gatherCompressibilityStats(row, ordinal) + if (!row.isNullAt(ordinal)) { + gatherCompressibilityStats(row, ordinal) + } } abstract override def build() = { diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala index c605a8e4434e3..ba1810dd2ae66 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/CompressionScheme.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.columnar.compression -import java.nio.ByteBuffer +import java.nio.{ByteOrder, ByteBuffer} import org.apache.spark.sql.catalyst.types.NativeType import org.apache.spark.sql.columnar.{ColumnType, NativeColumnType} @@ -84,7 +84,7 @@ private[sql] object CompressionScheme { } def columnHeaderSize(columnBuffer: ByteBuffer): Int = { - val header = columnBuffer.duplicate() + val header = columnBuffer.duplicate().order(ByteOrder.nativeOrder) val nullCount = header.getInt(4) // Column type ID + null count + null positions 4 + 4 + 4 * nullCount diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala index e92cf5ac4f9df..800009d3195e1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/compression/compressionSchemes.scala @@ -397,26 +397,27 @@ private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends Comp if (initial) { initial = false - prev = value _compressedSize += 1 + columnType.defaultSize } else { val (smallEnough, _) = byteSizedDelta(value, prev) _compressedSize += (if (smallEnough) 1 else 1 + columnType.defaultSize) } + + prev = value } override def compress(from: ByteBuffer, to: ByteBuffer, columnType: NativeColumnType[I]) = { to.putInt(typeId) if (from.hasRemaining) { - val prev = columnType.extract(from) - + var prev = columnType.extract(from) to.put(Byte.MinValue) columnType.append(prev, to) while (from.hasRemaining) { val current = columnType.extract(from) val (smallEnough, delta) = byteSizedDelta(current, prev) + prev = current if (smallEnough) { to.put(delta) @@ -443,13 +444,8 @@ private[sql] sealed abstract class IntegralDelta[I <: IntegralType] extends Comp override def next() = { val delta = buffer.get() - - if (delta > Byte.MinValue) { - addDelta(prev, delta) - } else { - prev = columnType.extract(buffer) - prev - } + prev = if (delta > Byte.MinValue) addDelta(prev, delta) else columnType.extract(buffer) + prev } override def hasNext = buffer.hasRemaining @@ -465,7 +461,7 @@ private[sql] case object IntDelta extends IntegralDelta[IntegerType.type] { override protected def byteSizedDelta(x: Int, y: Int): (Boolean, Byte) = { val delta = x - y - if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) } } @@ -478,6 +474,6 @@ private[sql] case object LongDelta extends IntegralDelta[LongType.type] { override protected def byteSizedDelta(x: Long, y: Long): (Boolean, Byte) = { val delta = x - y - if (delta < Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) + if (math.abs(delta) <= Byte.MaxValue) (true, delta.toByte) else (false, 0: Byte) } } 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 450c142c0baa4..070557e47c4c7 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 @@ -61,7 +61,14 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una shuffled.map(_._1) case SinglePartition => - child.execute().coalesce(1, shuffle = true) + val rdd = child.execute().mapPartitions { iter => + val mutablePair = new MutablePair[Null, Row]() + iter.map(r => mutablePair.update(null, r)) + } + val partitioner = new HashPartitioner(1) + val shuffled = new ShuffledRDD[Null, Row, MutablePair[Null, Row]](rdd, partitioner) + shuffled.setSerializer(new SparkSqlSerializer(new SparkConf(false))) + shuffled.map(_._2) case _ => sys.error(s"Exchange not implemented for $newPartitioning") // TODO: Handle BroadcastPartitioning. diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index daa423cb8ea1a..5d89697db5f99 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -70,8 +70,8 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) SparkLogicalPlan( alreadyPlanned match { case ExistingRdd(output, rdd) => ExistingRdd(output.map(_.newInstance), rdd) - case InMemoryColumnarTableScan(output, child) => - InMemoryColumnarTableScan(output.map(_.newInstance), child) + case scan @ InMemoryColumnarTableScan(output, child) => + scan.copy(attributes = output.map(_.newInstance)) case _ => sys.error("Multiple instance of the same relation detected.") }).asInstanceOf[this.type] } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala index 7c6a642278226..0331f90272a99 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/CachedTableSuite.scala @@ -17,11 +17,10 @@ package org.apache.spark.sql -import org.scalatest.FunSuite import org.apache.spark.sql.TestData._ -import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.columnar.InMemoryColumnarTableScan +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.test.TestSQLContext class CachedTableSuite extends QueryTest { TestData // Load test tables. diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala similarity index 79% rename from sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala rename to sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala index 2ed4cf2170f9d..16a13b8a74960 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnarQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/InMemoryColumnarQuerySuite.scala @@ -18,10 +18,11 @@ package org.apache.spark.sql.columnar import org.apache.spark.sql.{QueryTest, TestData} +import org.apache.spark.sql.catalyst.expressions.Row import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.test.TestSQLContext -class ColumnarQuerySuite extends QueryTest { +class InMemoryColumnarQuerySuite extends QueryTest { import TestData._ import TestSQLContext._ @@ -32,6 +33,15 @@ class ColumnarQuerySuite extends QueryTest { checkAnswer(scan, testData.collect().toSeq) } + test("projection") { + val plan = TestSQLContext.executePlan(testData.select('value, 'key).logicalPlan).executedPlan + val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) + + checkAnswer(scan, testData.collect().map { + case Row(key: Int, value: String) => value -> key + }.toSeq) + } + test("SPARK-1436 regression: in-memory columns must be able to be accessed multiple times") { val plan = TestSQLContext.executePlan(testData.logicalPlan).executedPlan val scan = SparkLogicalPlan(InMemoryColumnarTableScan(plan.output, plan)) diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala index 4a21eb6201a69..35ab14cbc353d 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala @@ -68,12 +68,16 @@ class NullableColumnAccessorSuite extends FunSuite { val row = new GenericMutableRow(1) (0 until 4).foreach { _ => + assert(accessor.hasNext) accessor.extractTo(row, 0) assert(row(0) === randomRow(0)) + assert(accessor.hasNext) accessor.extractTo(row, 0) assert(row.isNullAt(0)) } + + assert(!accessor.hasNext) } } } diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala index 1390e5eef6106..ce419ca7269ba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/compression/IntegralDeltaSuite.scala @@ -22,6 +22,7 @@ import org.scalatest.FunSuite import org.apache.spark.sql.catalyst.expressions.GenericMutableRow import org.apache.spark.sql.catalyst.types.IntegralType import org.apache.spark.sql.columnar._ +import org.apache.spark.sql.columnar.ColumnarTestUtils._ class IntegralDeltaSuite extends FunSuite { testIntegralDelta(new IntColumnStats, INT, IntDelta) @@ -63,7 +64,7 @@ class IntegralDeltaSuite extends FunSuite { } else { val oneBoolean = columnType.defaultSize 1 + oneBoolean + deltas.map { - d => if (math.abs(d) < Byte.MaxValue) 1 else 1 + oneBoolean + d => if (math.abs(d) <= Byte.MaxValue) 1 else 1 + oneBoolean }.sum }) @@ -78,7 +79,7 @@ class IntegralDeltaSuite extends FunSuite { expectResult(input.head, "The first value is wrong")(columnType.extract(buffer)) (input.tail, deltas).zipped.foreach { (value, delta) => - if (delta < Byte.MaxValue) { + if (math.abs(delta) <= Byte.MaxValue) { expectResult(delta, "Wrong delta")(buffer.get()) } else { expectResult(Byte.MinValue, "Expecting escaping mark here")(buffer.get()) @@ -105,11 +106,17 @@ class IntegralDeltaSuite extends FunSuite { test(s"$scheme: simple case") { val input = columnType match { - case INT => Seq(1: Int, 2: Int, 130: Int) - case LONG => Seq(1: Long, 2: Long, 130: Long) + case INT => Seq(2: Int, 1: Int, 2: Int, 130: Int) + case LONG => Seq(2: Long, 1: Long, 2: Long, 130: Long) } skeleton(input.map(_.asInstanceOf[I#JvmType])) } + + test(s"$scheme: long random series") { + // Have to workaround with `Any` since no `ClassTag[I#JvmType]` available here. + val input = Array.fill[Any](10000)(makeRandomValue(columnType)) + skeleton(input.map(_.asInstanceOf[I#JvmType])) + } } } 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 fc053c56c052d..c36b5878cb007 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala @@ -33,6 +33,8 @@ import org.apache.spark.sql.catalyst.plans.logical import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.catalyst.types._ +import org.apache.spark.sql.execution.SparkLogicalPlan +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan /* Implicit conversions */ import scala.collection.JavaConversions._ @@ -115,23 +117,31 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { 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, output) if input.dataType != output.dataType => - Alias(Cast(input, output.dataType), input.name)() - case (input, _) => input - } - - p.copy(child = logical.Project(castedChildOutput, child)) + castChildOutput(p, table, child) + + case p @ logical.InsertIntoTable(SparkLogicalPlan(InMemoryColumnarTableScan( + _, HiveTableScan(_, table, _))), _, child, _) => + castChildOutput(p, table, child) + } + + def castChildOutput(p: InsertIntoTable, table: MetastoreRelation, child: LogicalPlan) = { + 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, output) if input.dataType != output.dataType => + Alias(Cast(input, output.dataType), input.name)() + case (input, _) => input } + + p.copy(child = logical.Project(castedChildOutput, child)) + } } } 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 3ca1d93c11fa9..ac817b21a152e 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 @@ -23,6 +23,7 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ +import org.apache.spark.sql.columnar.InMemoryColumnarTableScan trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. @@ -42,6 +43,9 @@ 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(SparkLogicalPlan(InMemoryColumnarTableScan( + _, HiveTableScan(_, table, _))), partition, child, overwrite) => + InsertIntoHiveTable(table, partition, planLater(child), overwrite)(hiveContext) :: Nil case _ => Nil } } diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 2fea9702954d7..465e5f146fe71 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -160,12 +160,6 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(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("dest1", - "CREATE TABLE IF NOT EXISTS dest1 (key INT, value STRING)".cmd), - TestTable("dest2", - "CREATE TABLE IF NOT EXISTS dest2 (key INT, value STRING)".cmd), - 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)") @@ -257,6 +251,7 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { private val loadedTables = new collection.mutable.HashSet[String] + var cacheTables: Boolean = false def loadTestTable(name: String) { if (!(loadedTables contains name)) { // Marks the table as loaded first to prevent infite mutually recursive table loading. @@ -265,6 +260,9 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { val createCmds = testTables.get(name).map(_.commands).getOrElse(sys.error(s"Unknown test table $name")) createCmds.foreach(_()) + + if (cacheTables) + cacheTable(name) } } 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 f9b437d435eba..55a4363af6c76 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 @@ -130,8 +130,7 @@ trait HiveFunctionFactory { } } -abstract class HiveUdf - extends Expression with Logging with HiveFunctionFactory { +abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { self: Product => type UDFType @@ -146,7 +145,7 @@ abstract class HiveUdf lazy val functionInfo = getFunctionInfo(name) lazy val function = createFunction[UDFType](name) - override def toString = s"${nodeName}#${functionInfo.getDisplayName}(${children.mkString(",")})" + override def toString = s"$nodeName#${functionInfo.getDisplayName}(${children.mkString(",")})" } case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { @@ -202,10 +201,11 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } } -case class HiveGenericUdf( - name: String, - children: Seq[Expression]) extends HiveUdf with HiveInspectors { +case class HiveGenericUdf(name: String, children: Seq[Expression]) + extends HiveUdf with HiveInspectors { + import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ + type UDFType = GenericUDF @transient @@ -357,7 +357,7 @@ case class HiveGenericUdaf( override def toString = s"$nodeName#$name(${children.mkString(",")})" - def newInstance = new HiveUdafFunction(name, children, this) + def newInstance() = new HiveUdafFunction(name, children, this) } /** @@ -435,7 +435,7 @@ case class HiveGenericUdtf( } } - override def toString() = s"$nodeName#$name(${children.mkString(",")})" + override def toString = s"$nodeName#$name(${children.mkString(",")})" } case class HiveUdafFunction( 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 3cc4562a88d66..6c91f40d0f925 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 @@ -218,10 +218,7 @@ abstract class HiveComparisonTest val quotes = "\"\"\"" queryList.zipWithIndex.map { case (query, i) => - s""" - |val q$i = $quotes$query$quotes.q - |q$i.stringResult() - """.stripMargin + s"""val q$i = hql($quotes$query$quotes); q$i.collect()""" }.mkString("\n== Console version of this test ==\n", "\n", "\n") } @@ -287,7 +284,6 @@ abstract class HiveComparisonTest |Error: ${e.getMessage} |${stackTraceToString(e)} |$queryString - |$consoleTestCase """.stripMargin stringToFile( new File(hiveFailedDirectory, testCaseName), @@ -304,7 +300,7 @@ abstract class HiveComparisonTest val catalystResults = queryList.zip(hiveResults).map { case (queryString, hive) => val query = new TestHive.HiveQLQueryExecution(queryString) try { (query, prepareAnswer(query, query.stringResult())) } catch { - case e: Exception => + case e: Throwable => val errorMessage = s""" |Failed to execute query using catalyst: @@ -313,8 +309,6 @@ abstract class HiveComparisonTest |$query |== HIVE - ${hive.size} row(s) == |${hive.mkString("\n")} - | - |$consoleTestCase """.stripMargin stringToFile(new File(failedDirectory, testCaseName), errorMessage + consoleTestCase) fail(errorMessage) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index f76e16bc1afc5..c3cfa3d25a5c2 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -17,16 +17,26 @@ package org.apache.spark.sql.hive.execution +import org.scalatest.BeforeAndAfter + import org.apache.spark.sql.hive.TestHive /** * Runs the test cases that are included in the hive distribution. */ -class HiveCompatibilitySuite extends HiveQueryFileTest { +class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // TODO: bundle in jar files... get from classpath lazy val hiveQueryDir = TestHive.getHiveFile("ql/src/test/queries/clientpositive") def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) + override def beforeAll() { + TestHive.cacheTables = true + } + + override def afterAll() { + TestHive.cacheTables = false + } + /** 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. From 268b53567c93538c03cb66276ed9e05c9f1d3ac6 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 14 Apr 2014 15:51:54 -0700 Subject: [PATCH 274/397] HOTFIX: Use file name and not paths for excludes --- .rat-excludes | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/.rat-excludes b/.rat-excludes index 8954330bd10a7..9e9abb3f10bbf 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -40,5 +40,5 @@ work golden test.out/* .*iml -python/metastore/service.properties -python/metastore/db.lck +service.properties +db.lck From 0247b5c5467ca1b0d03ba929a78fa4d805582d84 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Mon, 14 Apr 2014 19:50:00 -0700 Subject: [PATCH 275/397] SPARK-1488. Resolve scalac feature warnings during build For your consideration: scalac currently notes a number of feature warnings during compilation: ``` [warn] there were 65 feature warning(s); re-run with -feature for details ``` Warnings are like: ``` [warn] /Users/srowen/Documents/spark/core/src/main/scala/org/apache/spark/SparkContext.scala:1261: implicit conversion method rddToPairRDDFunctions should be enabled [warn] by making the implicit value scala.language.implicitConversions visible. [warn] This can be achieved by adding the import clause 'import scala.language.implicitConversions' [warn] or by setting the compiler option -language:implicitConversions. [warn] See the Scala docs for value scala.language.implicitConversions for a discussion [warn] why the feature should be explicitly enabled. [warn] implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = [warn] ^ ``` scalac is suggesting that it's just best practice to explicitly enable certain language features by importing them where used. This PR simply adds the imports it suggests (and squashes one other Java warning along the way). This leaves just deprecation warnings in the build. Author: Sean Owen Closes #404 from srowen/SPARK-1488 and squashes the following commits: 8598980 [Sean Owen] Quiet scalac warnings about language features by explicitly importing language features. 39bc831 [Sean Owen] Enable -feature in scalac to emit language feature warnings --- bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala | 2 ++ core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + .../main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala | 1 + .../src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala | 1 + core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala | 1 + .../scala/org/apache/spark/api/java/JavaSparkContext.scala | 1 + .../scala/org/apache/spark/deploy/FaultToleranceTest.scala | 1 + .../src/main/scala/org/apache/spark/deploy/master/Master.scala | 1 + .../src/main/scala/org/apache/spark/deploy/worker/Worker.scala | 1 + .../scala/org/apache/spark/network/ConnectionManager.scala | 1 + core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala | 1 + core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala | 1 + .../scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala | 1 + .../src/main/scala/org/apache/spark/scheduler/ResultTask.scala | 1 + .../main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala | 1 + .../scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala | 1 + core/src/main/scala/org/apache/spark/ui/JettyUtils.scala | 1 + .../org/apache/spark/util/TimeStampedWeakValueHashMap.scala | 1 + core/src/main/scala/org/apache/spark/util/Vector.scala | 1 + .../test/scala/org/apache/spark/ConnectionManagerSuite.scala | 2 +- core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala | 2 ++ core/src/test/scala/org/apache/spark/DriverSuite.scala | 2 ++ core/src/test/scala/org/apache/spark/PipedRDDSuite.scala | 3 +-- .../test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala | 1 + .../scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala | 1 + .../scala/org/apache/spark/storage/BlockManagerSuite.scala | 3 +++ .../scala/org/apache/spark/storage/DiskBlockManagerSuite.scala | 1 + core/src/test/scala/org/apache/spark/ui/UISuite.scala | 1 + .../org/apache/spark/util/random/XORShiftRandomSuite.scala | 2 ++ .../org/apache/spark/streaming/examples/ZeroMQWordCount.scala | 2 ++ graphx/src/main/scala/org/apache/spark/graphx/Graph.scala | 1 + .../org/apache/spark/graphx/impl/MessageToPartition.scala | 1 + .../main/scala/org/apache/spark/graphx/impl/Serializers.scala | 3 ++- .../scala/org/apache/spark/graphx/util/BytecodeUtils.scala | 1 + .../main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala | 1 + .../scala/org/apache/spark/mllib/util/MFDataGenerator.scala | 1 + .../java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java | 1 + pom.xml | 1 + project/SparkBuild.scala | 2 +- .../main/scala/org/apache/spark/sql/catalyst/SqlParser.scala | 1 + .../scala/org/apache/spark/streaming/StreamingContext.scala | 1 + .../org/apache/spark/streaming/api/java/JavaDStream.scala | 1 + .../org/apache/spark/streaming/api/java/JavaDStreamLike.scala | 1 + .../org/apache/spark/streaming/api/java/JavaPairDStream.scala | 1 + .../org/apache/spark/streaming/receivers/ActorReceiver.scala | 1 + .../apache/spark/streaming/scheduler/NetworkInputTracker.scala | 1 + .../org/apache/spark/streaming/StreamingContextSuite.scala | 2 ++ .../src/test/scala/org/apache/spark/streaming/UISuite.scala | 1 + .../org/apache/spark/tools/JavaAPICompletenessChecker.scala | 1 + 49 files changed, 58 insertions(+), 5 deletions(-) diff --git a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala index 69144e3e657bf..8e0f82ddb8897 100644 --- a/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala +++ b/bagel/src/test/scala/org/apache/spark/bagel/BagelSuite.scala @@ -24,6 +24,8 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark._ import org.apache.spark.storage.StorageLevel +import scala.language.postfixOps + class TestVertex(val active: Boolean, val age: Int) extends Vertex with Serializable class TestMessage(val targetId: String) extends Message[String] with Serializable diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 456070fa7c5ef..3ddc0d5eeefb8 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -25,6 +25,7 @@ import java.util.UUID.randomUUID import scala.collection.{Map, Set} import scala.collection.generic.Growable import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.language.implicitConversions import scala.reflect.{ClassTag, classTag} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 537f410b0ca26..4330cef3965ee 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -19,6 +19,7 @@ package org.apache.spark.api.java import java.lang.{Double => JDouble} +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.Partitioner diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index a41c7dbda2afc..e5b2c8a5e7cb1 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -21,6 +21,7 @@ import java.util.{Comparator, List => JList} import java.lang.{Iterable => JIterable} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 01d9357a2556d..327c1552dc941 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -17,6 +17,7 @@ package org.apache.spark.api.java +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index 7fbefe1cb0fb1..e6a3f06b0ea42 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -22,6 +22,7 @@ import java.util.{Map => JMap} import scala.collection.JavaConversions import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional diff --git a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala index f4eb1601be3e4..47dbcd87c35b5 100644 --- a/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala +++ b/core/src/main/scala/org/apache/spark/deploy/FaultToleranceTest.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ListBuffer import scala.concurrent.{Await, future, promise} import scala.concurrent.ExecutionContext.Implicits.global import scala.concurrent.duration._ +import scala.language.postfixOps import scala.sys.process._ import org.json4s._ diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 6c58e741df001..81f990bfa6513 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -23,6 +23,7 @@ import java.util.Date import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet} import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.postfixOps import scala.util.Random import akka.actor._ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala index 52c164ca3c574..dd0a1360abe14 100755 --- a/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/Worker.scala @@ -23,6 +23,7 @@ import java.util.Date import scala.collection.mutable.HashMap import scala.concurrent.duration._ +import scala.language.postfixOps import akka.actor._ import akka.remote.{DisassociatedEvent, RemotingLifecycleEvent} diff --git a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala index cfee41c61362e..dcbbc1853186b 100644 --- a/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala +++ b/core/src/main/scala/org/apache/spark/network/ConnectionManager.scala @@ -33,6 +33,7 @@ import scala.collection.mutable.SynchronizedQueue import scala.concurrent.{Await, ExecutionContext, Future, Promise} import scala.concurrent.duration._ +import scala.language.postfixOps import org.apache.spark._ import org.apache.spark.util.{SystemClock, Utils} diff --git a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala index c6e79557f08a1..9ff76892aed32 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoGroupedRDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import org.apache.spark.{InterruptibleIterator, Partition, Partitioner, SparkEnv, TaskContext} import org.apache.spark.{Dependency, OneToOneDependency, ShuffleDependency} diff --git a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala index 44401a663440c..c45b759f007cc 100644 --- a/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/CoalescedRDD.scala @@ -21,6 +21,7 @@ import java.io.{IOException, ObjectOutputStream} import scala.collection.mutable import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import scala.reflect.ClassTag import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 7367c08b5d324..0800c5684c60f 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -20,6 +20,7 @@ package org.apache.spark.scheduler import java.util.Properties import scala.collection.mutable.Map +import scala.language.existentials import org.apache.spark._ import org.apache.spark.executor.TaskMetrics diff --git a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala index 0b381308b61ff..0e8d551e4b2ab 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ResultTask.scala @@ -21,6 +21,7 @@ import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap +import scala.language.existentials import org.apache.spark._ import org.apache.spark.rdd.{RDD, RDDCheckpointData} diff --git a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala index 23f3b3e824762..02b62de7e36b6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ShuffleMapTask.scala @@ -21,6 +21,7 @@ import java.io._ import java.util.zip.{GZIPInputStream, GZIPOutputStream} import scala.collection.mutable.HashMap +import scala.language.existentials import org.apache.spark._ import org.apache.spark.executor.ShuffleWriteMetrics diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index acd152dda89d4..a3439b525fde1 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -25,6 +25,7 @@ import scala.concurrent.duration._ import scala.collection.mutable.ArrayBuffer import scala.collection.mutable.HashMap import scala.collection.mutable.HashSet +import scala.language.postfixOps import scala.util.Random import org.apache.spark._ diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 3ae147a36c8a4..750f5a501c213 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -22,6 +22,7 @@ import javax.servlet.DispatcherType import javax.servlet.http.{HttpServlet, HttpServletRequest, HttpServletResponse} import scala.annotation.tailrec +import scala.language.implicitConversions import scala.util.{Failure, Success, Try} import scala.xml.Node diff --git a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala index b65017d6806c6..f5be5856c2109 100644 --- a/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala +++ b/core/src/main/scala/org/apache/spark/util/TimeStampedWeakValueHashMap.scala @@ -21,6 +21,7 @@ import java.lang.ref.WeakReference import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable +import scala.language.implicitConversions import org.apache.spark.Logging diff --git a/core/src/main/scala/org/apache/spark/util/Vector.scala b/core/src/main/scala/org/apache/spark/util/Vector.scala index 1a647fa1c9d84..c6cab82c3e546 100644 --- a/core/src/main/scala/org/apache/spark/util/Vector.scala +++ b/core/src/main/scala/org/apache/spark/util/Vector.scala @@ -17,6 +17,7 @@ package org.apache.spark.util +import scala.language.implicitConversions import scala.util.Random import org.apache.spark.util.random.XORShiftRandom diff --git a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala index 80f7ec00c74b2..df6b2604c8d8a 100644 --- a/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ConnectionManagerSuite.scala @@ -25,7 +25,7 @@ import org.apache.spark.network.{ConnectionManager, Message, ConnectionManagerId import scala.concurrent.Await import scala.concurrent.TimeoutException import scala.concurrent.duration._ - +import scala.language.postfixOps /** * Test the ConnectionManager with various security settings. diff --git a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala index e50981cf6fb20..5a8310090890d 100644 --- a/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala +++ b/core/src/test/scala/org/apache/spark/ContextCleanerSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark import java.lang.ref.WeakReference import scala.collection.mutable.{HashSet, SynchronizedSet} +import scala.language.existentials +import scala.language.postfixOps import scala.util.Random import org.scalatest.{BeforeAndAfter, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/DriverSuite.scala b/core/src/test/scala/org/apache/spark/DriverSuite.scala index 7f59bdcce4cc7..de4bd90c8f7e5 100644 --- a/core/src/test/scala/org/apache/spark/DriverSuite.scala +++ b/core/src/test/scala/org/apache/spark/DriverSuite.scala @@ -29,6 +29,8 @@ import org.scalatest.time.SpanSugar._ import org.apache.spark.util.Utils +import scala.language.postfixOps + class DriverSuite extends FunSuite with Timeouts { test("driver should exit after finishing") { diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index 867b28cc0d971..dfe057515efa0 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -19,8 +19,6 @@ package org.apache.spark import java.io.File -import com.google.common.io.Files - import org.scalatest.FunSuite import org.apache.spark.rdd.{HadoopRDD, PipedRDD, HadoopPartition} @@ -28,6 +26,7 @@ import org.apache.hadoop.mapred.{JobConf, TextInputFormat, FileSplit} import org.apache.hadoop.fs.Path import scala.collection.Map +import scala.language.postfixOps import scala.sys.process._ import scala.util.Try diff --git a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala index 3b833f2e41867..28197657e9bad 100644 --- a/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/AsyncRDDActionsSuite.scala @@ -22,6 +22,7 @@ import java.util.concurrent.Semaphore import scala.concurrent.{Await, TimeoutException} import scala.concurrent.duration.Duration import scala.concurrent.ExecutionContext.Implicits.global +import scala.language.postfixOps import org.scalatest.{BeforeAndAfterAll, FunSuite} import org.scalatest.concurrent.Timeouts diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index db4df1d1212ff..35a7ac9d049c2 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import scala.Tuple2 import scala.collection.mutable.{HashSet, HashMap, Map} +import scala.language.reflectiveCalls import org.scalatest.{BeforeAndAfter, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala index e10ec7d2624a0..907428db80af3 100644 --- a/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/BlockManagerSuite.scala @@ -33,6 +33,9 @@ import org.apache.spark.scheduler.LiveListenerBus import org.apache.spark.serializer.{JavaSerializer, KryoSerializer} import org.apache.spark.util.{AkkaUtils, ByteBufferInputStream, SizeEstimator, Utils} +import scala.language.implicitConversions +import scala.language.postfixOps + class BlockManagerSuite extends FunSuite with BeforeAndAfter with PrivateMethodTester { private val conf = new SparkConf(false) var store: BlockManager = null diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 808ddfdcf45d8..9b29e2a8a55de 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.storage import java.io.{File, FileWriter} import scala.collection.mutable +import scala.language.reflectiveCalls import com.google.common.io.Files import org.scalatest.{BeforeAndAfterEach, FunSuite} diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index b85c483ca2a08..ed02b0ba00d43 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -21,6 +21,7 @@ import java.net.ServerSocket import javax.servlet.http.HttpServletRequest import scala.io.Source +import scala.language.postfixOps import scala.util.{Failure, Success, Try} import org.eclipse.jetty.server.Server diff --git a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala index 39199a1a17ccd..0865c6386f7cd 100644 --- a/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/random/XORShiftRandomSuite.scala @@ -22,6 +22,8 @@ import org.scalatest.matchers.ShouldMatchers import org.apache.spark.util.Utils.times +import scala.language.reflectiveCalls + class XORShiftRandomSuite extends FunSuite with ShouldMatchers { def fixture = new { diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 445d2028582af..6f88db1abf19d 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -27,6 +27,8 @@ import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.zeromq._ +import scala.language.implicitConversions + /** * A simple publisher for demonstration purposes, repeatedly publishes random Messages * every one second. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index ef05623d7a0a1..45349692cbf6c 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -17,6 +17,7 @@ package org.apache.spark.graphx +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.graphx.impl._ diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala index 9d4f3750cb8e4..c45ba3d2f8c24 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/MessageToPartition.scala @@ -17,6 +17,7 @@ package org.apache.spark.graphx.impl +import scala.language.implicitConversions import scala.reflect.{classTag, ClassTag} import org.apache.spark.Partitioner diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala index 2f2c524df6394..2f0531ee5f379 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/Serializers.scala @@ -20,10 +20,11 @@ package org.apache.spark.graphx.impl import java.io.{EOFException, InputStream, OutputStream} import java.nio.ByteBuffer -import org.apache.spark.SparkConf import org.apache.spark.graphx._ import org.apache.spark.serializer._ +import scala.language.existentials + private[graphx] class VertexIdMsgSerializer extends Serializer with Serializable { override def newInstance(): SerializerInstance = new ShuffleSerializerInstance { diff --git a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala index 087b1156f690b..be6b9047d932d 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/util/BytecodeUtils.scala @@ -20,6 +20,7 @@ package org.apache.spark.graphx.util import java.io.{ByteArrayInputStream, ByteArrayOutputStream} import scala.collection.mutable.HashSet +import scala.language.existentials import org.apache.spark.util.Utils diff --git a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala index 873de871fd884..365b5e75d7f75 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/rdd/RDDFunctions.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.rdd +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.rdd.RDD diff --git a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala index 3f413faca6bb4..b76fbe89c3681 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/util/MFDataGenerator.scala @@ -17,6 +17,7 @@ package org.apache.spark.mllib.util +import scala.language.postfixOps import scala.util.Random import org.jblas.DoubleMatrix diff --git a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java index c6d8425ffc38d..1421067dc61ed 100644 --- a/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java +++ b/mllib/src/test/java/org/apache/spark/mllib/linalg/JavaVectorsSuite.java @@ -36,6 +36,7 @@ public void denseArrayConstruction() { @Test public void sparseArrayConstruction() { + @SuppressWarnings("unchecked") Vector v = Vectors.sparse(3, Lists.>newArrayList( new Tuple2(0, 2.0), new Tuple2(2, 3.0))); diff --git a/pom.xml b/pom.xml index 5f66cbe768592..0eacedf7a6533 100644 --- a/pom.xml +++ b/pom.xml @@ -648,6 +648,7 @@ -unchecked -deprecation + -feature -Xms1024m diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index a6058bba3d211..aac07b9f6129d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -162,7 +162,7 @@ object SparkBuild extends Build { organization := "org.apache.spark", version := SPARK_VERSION, scalaVersion := "2.10.4", - scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", + scalacOptions := Seq("-Xmax-classfile-name", "120", "-unchecked", "-deprecation", "-feature", "-target:" + SCALAC_JVM_VERSION), javacOptions := Seq("-target", JAVAC_JVM_VERSION, "-source", JAVAC_JVM_VERSION), unmanagedJars in Compile <<= baseDirectory map { base => (base / "lib" ** "*.jar").classpath }, 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 index 5b6aea81cb7d1..13a19d0adf5e6 100644 --- 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.catalyst +import scala.language.implicitConversions import scala.util.parsing.combinator.lexical.StdLexical import scala.util.parsing.combinator.syntactical.StandardTokenParsers import scala.util.parsing.input.CharArrayReader.EofCh diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index ff5d0aaa3d0bd..e9a4f7ba22576 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -22,6 +22,7 @@ import java.util.concurrent.atomic.AtomicInteger import scala.collection.Map import scala.collection.mutable.Queue +import scala.language.implicitConversions import scala.reflect.ClassTag import akka.actor.{Props, SupervisorStrategy} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 721d50273259e..13e2bacc92edc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -23,6 +23,7 @@ import org.apache.spark.api.java.JavaRDD import org.apache.spark.storage.StorageLevel import org.apache.spark.rdd.RDD +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.DStream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala index bb2f492d06a00..a6184de4e83c1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStreamLike.scala @@ -22,6 +22,7 @@ import java.lang.{Long => JLong} import java.util.{List => JList} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.api.java.{JavaPairRDD, JavaRDD, JavaRDDLike} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala index 2ac943d7bf781..cb8e8f00a7b82 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairDStream.scala @@ -21,6 +21,7 @@ import java.lang.{Long => JLong, Iterable => JIterable} import java.util.{List => JList} import scala.collection.JavaConversions._ +import scala.language.implicitConversions import scala.reflect.ClassTag import com.google.common.base.Optional diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala index f5984d03c5342..da0d364ae7bdb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala @@ -23,6 +23,7 @@ import akka.actor.{ PossiblyHarmful, OneForOneStrategy } import akka.actor.SupervisorStrategy._ import scala.concurrent.duration._ +import scala.language.postfixOps import scala.reflect.ClassTag import org.apache.spark.storage.{StorageLevel, StreamBlockId} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala index a1e6f5176825a..438e72a7ced89 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} +import scala.language.existentials import akka.actor._ diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index efd0d22ecb57a..ad5367ab941a2 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -28,6 +28,8 @@ import org.scalatest.concurrent.Timeouts import org.scalatest.exceptions.TestFailedDueToTimeoutException import org.scalatest.time.SpanSugar._ +import scala.language.postfixOps + class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts with Logging { val master = "local[2]" diff --git a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala index 35538ec188f67..031e93ab24a70 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/UISuite.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming import scala.io.Source +import scala.language.postfixOps import org.scalatest.FunSuite import org.scalatest.concurrent.Eventually._ diff --git a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala index 420522433e1e8..8d0f09933c8d3 100644 --- a/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala +++ b/tools/src/main/scala/org/apache/spark/tools/JavaAPICompletenessChecker.scala @@ -20,6 +20,7 @@ package org.apache.spark.tools import java.lang.reflect.Method import scala.collection.mutable.ArrayBuffer +import scala.language.existentials import org.apache.spark._ import org.apache.spark.api.java._ From c99bcb7feaa761c5826f2e1d844d0502a3b79538 Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Tue, 15 Apr 2014 00:07:55 -0700 Subject: [PATCH 276/397] SPARK-1374: PySpark API for SparkSQL An initial API that exposes SparkSQL functionality in PySpark. A PythonRDD composed of dictionaries, with string keys and primitive values (boolean, float, int, long, string) can be converted into a SchemaRDD that supports sql queries. ``` from pyspark.context import SQLContext sqlCtx = SQLContext(sc) rdd = sc.parallelize([{"field1" : 1, "field2" : "row1"}, {"field1" : 2, "field2": "row2"}, {"field1" : 3, "field2": "row3"}]) srdd = sqlCtx.applySchema(rdd) sqlCtx.registerRDDAsTable(srdd, "table1") srdd2 = sqlCtx.sql("SELECT field1 AS f1, field2 as f2 from table1") srdd2.collect() ``` The last line yields ```[{"f1" : 1, "f2" : "row1"}, {"f1" : 2, "f2": "row2"}, {"f1" : 3, "f2": "row3"}]``` Author: Ahir Reddy Author: Michael Armbrust Closes #363 from ahirreddy/pysql and squashes the following commits: 0294497 [Ahir Reddy] Updated log4j properties to supress Hive Warns 307d6e0 [Ahir Reddy] Style fix 6f7b8f6 [Ahir Reddy] Temporary fix MIMA checker. Since we now assemble Spark jar with Hive, we don't want to check the interfaces of all of our hive dependencies 3ef074a [Ahir Reddy] Updated documentation because classes moved to sql.py 29245bf [Ahir Reddy] Cache underlying SchemaRDD instead of generating and caching PythonRDD f2312c7 [Ahir Reddy] Moved everything into sql.py a19afe4 [Ahir Reddy] Doc fixes 6d658ba [Ahir Reddy] Remove the metastore directory created by the HiveContext tests in SparkSQL 521ff6d [Ahir Reddy] Trying to get spark to build with hive ab95eba [Ahir Reddy] Set SPARK_HIVE=true on jenkins ded03e7 [Ahir Reddy] Added doc test for HiveContext 22de1d4 [Ahir Reddy] Fixed maven pyrolite dependency e4da06c [Ahir Reddy] Display message if hive is not built into spark 227a0be [Michael Armbrust] Update API links. Fix Hive example. 58e2aa9 [Michael Armbrust] Build Docs for pyspark SQL Api. Minor fixes. 4285340 [Michael Armbrust] Fix building of Hive API Docs. 38a92b0 [Michael Armbrust] Add note to future non-python developers about python docs. 337b201 [Ahir Reddy] Changed com.clearspring.analytics stream version from 2.4.0 to 2.5.1 to match SBT build, and added pyrolite to maven build 40491c9 [Ahir Reddy] PR Changes + Method Visibility 1836944 [Michael Armbrust] Fix comments. e00980f [Michael Armbrust] First draft of python sql programming guide. b0192d3 [Ahir Reddy] Added Long, Double and Boolean as usable types + unit test f98a422 [Ahir Reddy] HiveContexts 79621cf [Ahir Reddy] cleaning up cruft b406ba0 [Ahir Reddy] doctest formatting 20936a5 [Ahir Reddy] Added tests and documentation e4d21b4 [Ahir Reddy] Added pyrolite dependency 79f739d [Ahir Reddy] added more tests 7515ba0 [Ahir Reddy] added more tests :) d26ec5e [Ahir Reddy] added test e9f5b8d [Ahir Reddy] adding tests 906d180 [Ahir Reddy] added todo explaining cost of creating Row object in python 251f99d [Ahir Reddy] for now only allow dictionaries as input 09b9980 [Ahir Reddy] made jrdd explicitly lazy c608947 [Ahir Reddy] SchemaRDD now has all RDD operations 725c91e [Ahir Reddy] awesome row objects 55d1c76 [Ahir Reddy] return row objects 4fe1319 [Ahir Reddy] output dictionaries correctly be079de [Ahir Reddy] returning dictionaries works cd5f79f [Ahir Reddy] Switched to using Scala SQLContext e948bd9 [Ahir Reddy] yippie 4886052 [Ahir Reddy] even better c0fb1c6 [Ahir Reddy] more working 043ca85 [Ahir Reddy] working 5496f9f [Ahir Reddy] doesn't crash b8b904b [Ahir Reddy] Added schema rdd class 67ba875 [Ahir Reddy] java to python, and python to java bcc0f23 [Ahir Reddy] Java to python ab6025d [Ahir Reddy] compiling --- core/pom.xml | 5 + .../apache/spark/api/python/PythonRDD.scala | 32 ++ dev/run-tests | 1 + docs/README.md | 2 +- docs/_plugins/copy_api_dirs.rb | 4 +- docs/sql-programming-guide.md | 103 ++++- pom.xml | 2 +- project/SparkBuild.scala | 3 +- python/pyspark/__init__.py | 18 +- python/pyspark/java_gateway.py | 4 + python/pyspark/sql.py | 363 ++++++++++++++++++ python/run-tests | 4 + .../org/apache/spark/sql/SQLContext.scala | 27 ++ .../org/apache/spark/sql/SchemaRDD.scala | 23 ++ .../org/apache/spark/sql/hive/TestHive.scala | 3 +- sql/hive/src/test/resources/log4j.properties | 3 + .../spark/tools/GenerateMIMAIgnore.scala | 4 +- 17 files changed, 589 insertions(+), 12 deletions(-) create mode 100644 python/pyspark/sql.py diff --git a/core/pom.xml b/core/pom.xml index a1bdd8ec68aeb..d87e2bca030e3 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -266,6 +266,11 @@ junit-interface test + + org.spark-project + pyrolite + 2.0 + target/scala-${scala.binary.version}/classes diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 32f1100406d74..f9d86fed34d0f 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -25,6 +25,8 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import net.razorvine.pickle.{Pickler, Unpickler} + import org.apache.spark._ import org.apache.spark.api.java.{JavaSparkContext, JavaPairRDD, JavaRDD} import org.apache.spark.broadcast.Broadcast @@ -284,6 +286,36 @@ private[spark] object PythonRDD { file.close() } + /** + * Convert an RDD of serialized Python dictionaries to Scala Maps + * TODO: Support more Python types. + */ + def pythonToJavaMap(pyRDD: JavaRDD[Array[Byte]]): JavaRDD[Map[String, _]] = { + pyRDD.rdd.mapPartitions { iter => + val unpickle = new Unpickler + // TODO: Figure out why flatMap is necessay for pyspark + iter.flatMap { row => + unpickle.loads(row) match { + case objs: java.util.ArrayList[JMap[String, _] @unchecked] => objs.map(_.toMap) + // Incase the partition doesn't have a collection + case obj: JMap[String @unchecked, _] => Seq(obj.toMap) + } + } + } + } + + /** + * Convert and RDD of Java objects to and RDD of serialized Python objects, that is usable by + * PySpark. + */ + def javaToPython(jRDD: JavaRDD[Any]): JavaRDD[Array[Byte]] = { + jRDD.rdd.mapPartitions { iter => + val pickle = new Pickler + iter.map { row => + pickle.dumps(row) + } + } + } } private diff --git a/dev/run-tests b/dev/run-tests index 6ad674a2ba127..0725b681f1a1b 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -34,6 +34,7 @@ else fi JAVA_VERSION=$($java_cmd -version 2>&1 | sed 's/java version "\(.*\)\.\(.*\)\..*"/\1\2/; 1q') [ "$JAVA_VERSION" -ge 18 ] && echo "" || echo "[Warn] Java 8 tests will not run because JDK version is < 1.8." +export SPARK_HIVE=true echo "=========================================================================" echo "Running Apache RAT checks" diff --git a/docs/README.md b/docs/README.md index 0678fc5c86706..75b1811ba99af 100644 --- a/docs/README.md +++ b/docs/README.md @@ -42,7 +42,7 @@ To mark a block of code in your markdown to be syntax highlighted by jekyll duri You can build just the Spark scaladoc by running `sbt/sbt doc` from the SPARK_PROJECT_ROOT directory. -Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. +Similarly, you can build just the PySpark epydoc by running `epydoc --config epydoc.conf` from the SPARK_PROJECT_ROOT/pyspark directory. Documentation is only generated for classes that are listed as public in `__init__.py`. When you run `jekyll` in the docs directory, it will also copy over the scaladoc for the various Spark subprojects into the docs directory (and then also into the _site directory). We use a jekyll plugin to run `sbt/sbt doc` before building the site so if you haven't run it (recently) it may take some time as it generates all of the scaladoc. The jekyll plugin also generates the PySpark docs using [epydoc](http://epydoc.sourceforge.net/). diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index bbd56d2fd13bb..05f0bd47a88a5 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -32,8 +32,8 @@ curr_dir = pwd cd("..") - puts "Running sbt/sbt doc from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt doc` + puts "Running 'sbt/sbt doc hive/doc' from " + pwd + "; this may take a few minutes..." + puts `sbt/sbt doc hive/doc` puts "Moving back into docs dir." cd("docs") diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index a59393e1424de..6f616fb7c2448 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -20,7 +20,7 @@ a schema that describes the data types of each column in the row. A SchemaRDD i in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). -**All of the examples on this page use sample data included in the Spark distribution and can be run in the spark-shell.** +**All of the examples on this page use sample data included in the Spark distribution and can be run in the `spark-shell`.** @@ -33,6 +33,19 @@ a schema that describes the data types of each column in the row. A JavaSchemaR in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). + +
      + +Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using +Spark. At the core of this component is a new type of RDD, +[SchemaRDD](api/pyspark/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed +[Row](api/pyspark/pyspark.sql.Row-class.html) objects along with +a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table +in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet +file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). + +**All of the examples on this page use sample data included in the Spark distribution and can be run in the `pyspark` shell.** +
      *************************************************************************************************** @@ -44,7 +57,7 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac The entry point into all relational functionality in Spark is the [SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its -decendents. To create a basic SQLContext, all you need is a SparkContext. +descendants. To create a basic SQLContext, all you need is a SparkContext. {% highlight scala %} val sc: SparkContext // An existing SparkContext. @@ -60,7 +73,7 @@ import sqlContext._ The entry point into all relational functionality in Spark is the [JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one -of its decendents. To create a basic JavaSQLContext, all you need is a JavaSparkContext. +of its descendants. To create a basic JavaSQLContext, all you need is a JavaSparkContext. {% highlight java %} JavaSparkContext ctx = ...; // An existing JavaSparkContext. @@ -69,6 +82,19 @@ JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx); +
      + +The entry point into all relational functionality in Spark is the +[SQLContext](api/pyspark/pyspark.sql.SQLContext-class.html) class, or one +of its decedents. To create a basic SQLContext, all you need is a SparkContext. + +{% highlight python %} +from pyspark.sql import SQLContext +sqlCtx = SQLContext(sc) +{% endhighlight %} + +
      + ## Running SQL on RDDs @@ -81,7 +107,7 @@ One type of table that is supported by Spark SQL is an RDD of Scala case classes defines the schema of the table. The names of the arguments to the case class are read using reflection and become the names of the columns. Case classes can also be nested or contain complex types such as Sequences or Arrays. This RDD can be implicitly converted to a SchemaRDD and then be -registered as a table. Tables can used in subsequent SQL statements. +registered as a table. Tables can be used in subsequent SQL statements. {% highlight scala %} val sqlContext = new org.apache.spark.sql.SQLContext(sc) @@ -176,6 +202,34 @@ List teenagerNames = teenagers.map(new Function() { +
      + +One type of table that is supported by Spark SQL is an RDD of dictionaries. The keys of the +dictionary define the columns names of the table, and the types are inferred by looking at the first +row. Any RDD of dictionaries can converted to a SchemaRDD and then registered as a table. Tables +can be used in subsequent SQL statements. + +{% highlight python %} +# Load a text file and convert each line to a dictionary. +lines = sc.textFile("examples/src/main/resources/people.txt") +parts = lines.map(lambda l: l.split(",")) +people = parts.map(lambda p: {"name": p[0], "age": int(p[1])}) + +# Infer the schema, and register the SchemaRDD as a table. +# In future versions of PySpark we would like to add support for registering RDDs with other +# datatypes as tables +peopleTable = sqlCtx.inferSchema(people) +peopleTable.registerAsTable("people") + +# SQL can be run over SchemaRDDs that have been registered as a table. +teenagers = sqlCtx.sql("SELECT name FROM people WHERE age >= 13 AND age <= 19") + +# The results of SQL queries are RDDs and support all the normal RDD operations. +teenNames = teenagers.map(lambda p: "Name: " + p.name) +{% endhighlight %} + +
      + **Note that Spark SQL currently uses a very basic SQL parser.** @@ -231,6 +285,27 @@ parquetFile.registerAsTable("parquetFile"); JavaSchemaRDD teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19"); +{% endhighlight %} + + + +
      + +{% highlight python %} + +peopleTable # The SchemaRDD from the previous example. + +# SchemaRDDs can be saved as parquet files, maintaining the schema information. +peopleTable.saveAsParquetFile("people.parquet") + +# Read in the parquet file created above. Parquet files are self-describing so the schema is preserved. +# The result of loading a parquet file is also a SchemaRDD. +parquetFile = sqlCtx.parquetFile("people.parquet") + +# Parquet files can also be registered as tables and then used in SQL statements. +parquetFile.registerAsTable("parquetFile"); +teenagers = sqlCtx.sql("SELECT name FROM parquetFile WHERE age >= 13 AND age <= 19") + {% endhighlight %}
      @@ -318,4 +393,24 @@ Row[] results = hiveCtx.hql("FROM src SELECT key, value").collect(); +
      + +When working with Hive one must construct a `HiveContext`, which inherits from `SQLContext`, and +adds support for finding tables in in the MetaStore and writing queries using HiveQL. In addition to +the `sql` method a `HiveContext` also provides an `hql` methods, which allows queries to be +expressed in HiveQL. + +{% highlight python %} + +from pyspark.sql import HiveContext +hiveCtx = HiveContext(sc) + +hiveCtx.hql("CREATE TABLE IF NOT EXISTS src (key INT, value STRING)") +hiveCtx.hql("LOAD DATA LOCAL INPATH 'examples/src/main/resources/kv1.txt' INTO TABLE src") + +# Queries can be expressed in HiveQL. +results = hiveCtx.hql("FROM src SELECT key, value").collect() + +{% endhighlight %} +
      diff --git a/pom.xml b/pom.xml index 0eacedf7a6533..cd204376de5db 100644 --- a/pom.xml +++ b/pom.xml @@ -262,7 +262,7 @@ com.clearspring.analytics stream - 2.4.0 + 2.5.1 + + it.unimi.dsi + fastutil + + SPARK_MASTER_OPTS for master settings 4982331 [Patrick Wendell] Remove SPARK_LIBRARY_PATH 6eaf7d0 [Patrick Wendell] executorJavaOpts 0faa3b6 [Patrick Wendell] Stash of adding config options in submit script and YARN ac2d65e [Patrick Wendell] Change spark.local.dir -> SPARK_LOCAL_DIRS --- .rat-excludes | 1 + bin/run-example | 1 - bin/spark-class | 2 +- bin/spark-submit | 7 +- conf/spark-defaults.conf.template | 7 + conf/spark-env.sh.template | 43 ++-- .../scala/org/apache/spark/SparkConf.scala | 76 +++++++ .../scala/org/apache/spark/SparkContext.scala | 37 ++-- .../org/apache/spark/deploy/Client.scala | 15 +- .../org/apache/spark/deploy/Command.scala | 5 +- .../org/apache/spark/deploy/SparkSubmit.scala | 53 +++-- .../spark/deploy/SparkSubmitArguments.scala | 190 ++++++++++++++---- .../spark/deploy/client/TestClient.scala | 4 +- .../spark/deploy/worker/CommandUtils.scala | 19 +- .../spark/deploy/worker/DriverRunner.scala | 16 +- .../spark/deploy/worker/ExecutorRunner.scala | 4 +- .../org/apache/spark/executor/Executor.scala | 5 +- .../cluster/SparkDeploySchedulerBackend.scala | 13 +- .../mesos/CoarseMesosSchedulerBackend.scala | 23 ++- .../cluster/mesos/MesosSchedulerBackend.scala | 2 +- .../spark/deploy/JsonProtocolSuite.scala | 15 +- .../spark/deploy/SparkSubmitSuite.scala | 111 ++++++++-- .../deploy/worker/DriverRunnerTest.scala | 2 +- .../deploy/worker/ExecutorRunnerTest.scala | 3 +- dev/audit-release/audit_release.py | 1 + .../src/main/scala/SparkApp.scala | 7 +- .../src/main/scala/GraphxApp.scala | 10 +- .../src/main/scala/StreamingApp.scala | 8 +- docs/cluster-overview.md | 135 ++++++++----- docs/configuration.md | 50 +++-- docs/quick-start.md | 178 ++++++++-------- docs/scala-programming-guide.md | 46 +++-- docs/sql-programming-guide.md | 2 +- .../spark/deploy/yarn/ApplicationMaster.scala | 3 + .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../spark/deploy/yarn/ExecutorLauncher.scala | 6 + .../deploy/yarn/YarnAllocationHandler.scala | 4 + .../apache/spark/deploy/yarn/ClientBase.scala | 137 +++++++------ .../deploy/yarn/ExecutorRunnableUtil.scala | 21 +- .../cluster/YarnClientSchedulerBackend.scala | 4 +- .../spark/deploy/yarn/ApplicationMaster.scala | 3 + .../org/apache/spark/deploy/yarn/Client.scala | 4 +- .../spark/deploy/yarn/ExecutorLauncher.scala | 6 + .../deploy/yarn/YarnAllocationHandler.scala | 4 + 44 files changed, 886 insertions(+), 401 deletions(-) create mode 100644 conf/spark-defaults.conf.template diff --git a/.rat-excludes b/.rat-excludes index 9e9abb3f10bbf..50766954ef070 100644 --- a/.rat-excludes +++ b/.rat-excludes @@ -11,6 +11,7 @@ RELEASE control docs fairscheduler.xml.template +spark-defaults.conf.template log4j.properties log4j.properties.template metrics.properties.template diff --git a/bin/run-example b/bin/run-example index 5af95a08c6c41..b2999198a8d41 100755 --- a/bin/run-example +++ b/bin/run-example @@ -75,7 +75,6 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$SPARK_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then JAVA_OPTS="$JAVA_OPTS `cat $FWDIR/conf/java-opts`" diff --git a/bin/spark-class b/bin/spark-class index 1b0d309cc5b1c..6871e180c9fa8 100755 --- a/bin/spark-class +++ b/bin/spark-class @@ -98,7 +98,7 @@ fi # Set JAVA_OPTS to be able to load native libraries and to set heap size JAVA_OPTS="$OUR_JAVA_OPTS" -JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$SPARK_LIBRARY_PATH" +JAVA_OPTS="$JAVA_OPTS -Djava.library.path=$_SPARK_LIBRARY_PATH" JAVA_OPTS="$JAVA_OPTS -Xms$OUR_JAVA_MEM -Xmx$OUR_JAVA_MEM" # Load extra JAVA_OPTS from conf/java-opts, if it exists if [ -e "$FWDIR/conf/java-opts" ] ; then diff --git a/bin/spark-submit b/bin/spark-submit index d92d55a032bd5..498d0b27bacdf 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -25,8 +25,13 @@ while (($#)); do DEPLOY_MODE=$2 elif [ $1 = "--driver-memory" ]; then DRIVER_MEMORY=$2 + elif [ $1 = "--driver-library-path" ]; then + export _SPARK_LIBRARY_PATH=$2 + elif [ $1 = "--driver-class-path" ]; then + export SPARK_CLASSPATH="$SPARK_CLASSPATH:$2" + elif [ $1 = "--driver-java-options" ]; then + export SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $2" fi - shift done diff --git a/conf/spark-defaults.conf.template b/conf/spark-defaults.conf.template new file mode 100644 index 0000000000000..f840ff681d019 --- /dev/null +++ b/conf/spark-defaults.conf.template @@ -0,0 +1,7 @@ +# Default system properties included when running spark-submit. +# This is useful for setting default environmental settings. + +# Example: +# spark.master spark://master:7077 +# spark.eventLog.enabled true +# spark.eventLog.dir hdfs://namenode:8021/directory diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 6432a566089be..177a21cc0377f 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -1,22 +1,41 @@ #!/usr/bin/env bash -# This file contains environment variables required to run Spark. Copy it as -# spark-env.sh and edit that to configure Spark for your site. -# -# The following variables can be set in this file: +# This file is sourced when running various Spark programs. +# Copy it as spark-env.sh and edit that to configure Spark for your site. + +# Options read when launching programs locally with +# ./bin/run-example or ./bin/spark-submit +# - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# - SPARK_PUBLIC_DNS, to set the public dns name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append + +# Options read by executors and drivers running inside the cluster # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node +# - SPARK_PUBLIC_DNS, to set the public DNS name of the driver program +# - SPARK_CLASSPATH, default classpath entries to append +# - SPARK_LOCAL_DIRS, storage directories to use on this node for shuffle and RDD data # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos -# - SPARK_JAVA_OPTS, to set node-specific JVM options for Spark. Note that -# we recommend setting app-wide options in the application's driver program. -# Examples of node-specific options : -Dspark.local.dir, GC options -# Examples of app-wide options : -Dspark.serializer -# -# If using the standalone deploy mode, you can also set variables for it here: + +# Options read in YARN client mode +# - SPARK_EXECUTOR_INSTANCES, Number of workers to start (Default: 2) +# - SPARK_EXECUTOR_CORES, Number of cores for the workers (Default: 1). +# - SPARK_EXECUTOR_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) +# - SPARK_DRIVER_MEMORY, Memory for Master (e.g. 1000M, 2G) (Default: 512 Mb) +# - SPARK_YARN_APP_NAME, The name of your application (Default: Spark) +# - SPARK_YARN_QUEUE, The hadoop queue to use for allocation requests (Default: ‘default’) +# - SPARK_YARN_DIST_FILES, Comma separated list of files to be distributed with the job. +# - SPARK_YARN_DIST_ARCHIVES, Comma separated list of archives to be distributed with the job. + +# Options for the daemons used in the standalone deploy mode: # - SPARK_MASTER_IP, to bind the master to a different IP address or hostname # - SPARK_MASTER_PORT / SPARK_MASTER_WEBUI_PORT, to use non-default ports +# - SPARK_MASTER_OPTS, to set config properties only for the master (e.g. "-Dx=y") # - SPARK_WORKER_CORES, to set the number of cores to use on this machine -# - SPARK_WORKER_MEMORY, to set how much memory to use (e.g. 1000m, 2g) +# - SPARK_WORKER_MEMORY, to set how much total memory workers have to give executors (e.g. 1000m, 2g) # - SPARK_WORKER_PORT / SPARK_WORKER_WEBUI_PORT # - SPARK_WORKER_INSTANCES, to set the number of worker processes per node # - SPARK_WORKER_DIR, to set the working directory of worker processes -# - SPARK_PUBLIC_DNS, to set the public dns name of the master +# - SPARK_WORKER_OPTS, to set config properties only for the worker (e.g. "-Dx=y") +# - SPARK_HISTORY_OPTS, to set config properties only for the history server (e.g. "-Dx=y") +# - SPARK_DAEMON_OPTS, to set config properties for all daemons (e.g. "-Dx=y") +# - SPARK_PUBLIC_DNS, to set the public dns name of the master or workers diff --git a/core/src/main/scala/org/apache/spark/SparkConf.scala b/core/src/main/scala/org/apache/spark/SparkConf.scala index b947feb891ee6..bd21fdc5a18e4 100644 --- a/core/src/main/scala/org/apache/spark/SparkConf.scala +++ b/core/src/main/scala/org/apache/spark/SparkConf.scala @@ -208,6 +208,82 @@ class SparkConf(loadDefaults: Boolean) extends Cloneable with Logging { new SparkConf(false).setAll(settings) } + /** Checks for illegal or deprecated config settings. Throws an exception for the former. Not + * idempotent - may mutate this conf object to convert deprecated settings to supported ones. */ + private[spark] def validateSettings() { + if (settings.contains("spark.local.dir")) { + val msg = "In Spark 1.0 and later spark.local.dir will be overridden by the value set by " + + "the cluster manager (via SPARK_LOCAL_DIRS in mesos/standalone and LOCAL_DIRS in YARN)." + logWarning(msg) + } + + val executorOptsKey = "spark.executor.extraJavaOptions" + val executorClasspathKey = "spark.executor.extraClassPath" + val driverOptsKey = "spark.driver.extraJavaOptions" + val driverClassPathKey = "spark.driver.extraClassPath" + + // Validate spark.executor.extraJavaOptions + settings.get(executorOptsKey).map { javaOpts => + if (javaOpts.contains("-Dspark")) { + val msg = s"$executorOptsKey is not allowed to set Spark options (was '$javaOpts)'. " + + "Set them directly on a SparkConf or in a properties file when using ./bin/spark-submit." + throw new Exception(msg) + } + if (javaOpts.contains("-Xmx") || javaOpts.contains("-Xms")) { + val msg = s"$executorOptsKey is not allowed to alter memory settings (was '$javaOpts'). " + + "Use spark.executor.memory instead." + throw new Exception(msg) + } + } + + // Check for legacy configs + sys.env.get("SPARK_JAVA_OPTS").foreach { value => + val error = + s""" + |SPARK_JAVA_OPTS was detected (set to '$value'). + |This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with conf/spark-defaults.conf to set defaults for an application + | - ./spark-submit with --driver-java-options to set -X options for a driver + | - spark.executor.extraJavaOptions to set -X options for executors + | - SPARK_DAEMON_OPTS to set java options for standalone daemons (i.e. master, worker) + """.stripMargin + logError(error) + + for (key <- Seq(executorOptsKey, driverOptsKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_JAVA_OPTS. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } + + sys.env.get("SPARK_CLASSPATH").foreach { value => + val error = + s""" + |SPARK_CLASSPATH was detected (set to '$value'). + | This has undefined behavior when running on a cluster and is deprecated in Spark 1.0+. + | + |Please instead use: + | - ./spark-submit with --driver-class-path to augment the driver classpath + | - spark.executor.extraClassPath to augment the executor classpath + """.stripMargin + logError(error) + + for (key <- Seq(executorClasspathKey, driverClassPathKey)) { + if (getOption(key).isDefined) { + throw new SparkException(s"Found both $key and SPARK_CLASSPATH. Use only the former.") + } else { + logWarning(s"Setting '$key' to '$value' as a work-around.") + set(key, value) + } + } + } + } + /** * Return a string listing all keys and values, one per line. This is useful to print the * configuration out for debugging. diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index d3ef75bc7335a..7933d68d67d96 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -148,6 +148,7 @@ class SparkContext(config: SparkConf) extends Logging { this(master, appName, sparkHome, jars, Map(), Map()) private[spark] val conf = config.clone() + conf.validateSettings() /** * Return a copy of this SparkContext's configuration. The configuration ''cannot'' be @@ -159,7 +160,7 @@ class SparkContext(config: SparkConf) extends Logging { throw new SparkException("A master URL must be set in your configuration") } if (!conf.contains("spark.app.name")) { - throw new SparkException("An application must be set in your configuration") + throw new SparkException("An application name must be set in your configuration") } if (conf.getBoolean("spark.logConf", false)) { @@ -170,11 +171,11 @@ class SparkContext(config: SparkConf) extends Logging { conf.setIfMissing("spark.driver.host", Utils.localHostName()) conf.setIfMissing("spark.driver.port", "0") - val jars: Seq[String] = if (conf.contains("spark.jars")) { - conf.get("spark.jars").split(",").filter(_.size != 0) - } else { - null - } + val jars: Seq[String] = + conf.getOption("spark.jars").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten + + val files: Seq[String] = + conf.getOption("spark.files").map(_.split(",")).map(_.filter(_.size != 0)).toSeq.flatten val master = conf.get("spark.master") val appName = conf.get("spark.app.name") @@ -235,6 +236,10 @@ class SparkContext(config: SparkConf) extends Logging { jars.foreach(addJar) } + if (files != null) { + files.foreach(addFile) + } + private def warnSparkMem(value: String): String = { logWarning("Using SPARK_MEM to set amount of memory to use per executor process is " + "deprecated, please use spark.executor.memory instead.") @@ -247,22 +252,20 @@ class SparkContext(config: SparkConf) extends Logging { .map(Utils.memoryStringToMb) .getOrElse(512) - // Environment variables to pass to our executors - private[spark] val executorEnvs = HashMap[String, String]() - for (key <- Seq("SPARK_CLASSPATH", "SPARK_LIBRARY_PATH", "SPARK_JAVA_OPTS"); - value <- Option(System.getenv(key))) { - executorEnvs(key) = value - } + // Environment variables to pass to our executors. + // NOTE: This should only be used for test related settings. + private[spark] val testExecutorEnvs = HashMap[String, String]() + // Convert java options to env vars as a work around // since we can't set env vars directly in sbt. - for { (envKey, propKey) <- Seq(("SPARK_HOME", "spark.home"), ("SPARK_TESTING", "spark.testing")) + for { (envKey, propKey) <- Seq(("SPARK_TESTING", "spark.testing")) value <- Option(System.getenv(envKey)).orElse(Option(System.getProperty(propKey)))} { - executorEnvs(envKey) = value + testExecutorEnvs(envKey) = value } // The Mesos scheduler backend relies on this environment variable to set executor memory. // TODO: Set this only in the Mesos scheduler. - executorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" - executorEnvs ++= conf.getExecutorEnv + testExecutorEnvs("SPARK_EXECUTOR_MEMORY") = executorMemory + "m" + testExecutorEnvs ++= conf.getExecutorEnv // Set SPARK_USER for user who is running SparkContext. val sparkUser = Option { @@ -270,7 +273,7 @@ class SparkContext(config: SparkConf) extends Logging { }.getOrElse { SparkContext.SPARK_UNKNOWN_USER } - executorEnvs("SPARK_USER") = sparkUser + testExecutorEnvs("SPARK_USER") = sparkUser // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) diff --git a/core/src/main/scala/org/apache/spark/deploy/Client.scala b/core/src/main/scala/org/apache/spark/deploy/Client.scala index 8fd2c7e95b966..7ead1171525d2 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Client.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Client.scala @@ -54,8 +54,21 @@ private class ClientActor(driverArgs: ClientArguments, conf: SparkConf) extends System.getenv().foreach{case (k, v) => env(k) = v} val mainClass = "org.apache.spark.deploy.worker.DriverWrapper" + + val classPathConf = "spark.driver.extraClassPath" + val classPathEntries = sys.props.get(classPathConf).toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + + val libraryPathConf = "spark.driver.extraLibraryPath" + val libraryPathEntries = sys.props.get(libraryPathConf).toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + + val javaOptionsConf = "spark.driver.extraJavaOptions" + val javaOpts = sys.props.get(javaOptionsConf) val command = new Command(mainClass, Seq("{{WORKER_URL}}", driverArgs.mainClass) ++ - driverArgs.driverOptions, env) + driverArgs.driverOptions, env, classPathEntries, libraryPathEntries, javaOpts) val driverDescription = new DriverDescription( driverArgs.jarUrl, diff --git a/core/src/main/scala/org/apache/spark/deploy/Command.scala b/core/src/main/scala/org/apache/spark/deploy/Command.scala index fa8af9a646750..32f3ba385084f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/Command.scala +++ b/core/src/main/scala/org/apache/spark/deploy/Command.scala @@ -22,5 +22,8 @@ import scala.collection.Map private[spark] case class Command( mainClass: String, arguments: Seq[String], - environment: Map[String, String]) { + environment: Map[String, String], + classPathEntries: Seq[String], + libraryPathEntries: Seq[String], + extraJavaOptions: Option[String] = None) { } diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index e5d593cade8b3..1b1e0fce0e0df 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -17,14 +17,12 @@ package org.apache.spark.deploy -import java.io.{PrintStream, File} +import java.io.{File, PrintStream} import java.net.{URI, URL} -import org.apache.spark.executor.ExecutorURLClassLoader +import scala.collection.mutable.{ArrayBuffer, HashMap, Map} -import scala.collection.mutable.ArrayBuffer -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map +import org.apache.spark.executor.ExecutorURLClassLoader /** * Scala code behind the spark-submit script. The script handles setting up the classpath with @@ -63,7 +61,8 @@ object SparkSubmit { /** * @return * a tuple containing the arguments for the child, a list of classpath - * entries for the child, and the main class for the child + * entries for the child, a list of system propertes, a list of env vars + * and the main class for the child */ private[spark] def createLaunchEnv(appArgs: SparkSubmitArguments): (ArrayBuffer[String], ArrayBuffer[String], Map[String, String], String) = { @@ -123,6 +122,12 @@ object SparkSubmit { val options = List[OptionAssigner]( new OptionAssigner(appArgs.master, ALL_CLUSTER_MGRS, false, sysProp = "spark.master"), + new OptionAssigner(appArgs.driverExtraClassPath, STANDALONE | YARN, true, + sysProp = "spark.driver.extraClassPath"), + new OptionAssigner(appArgs.driverExtraJavaOptions, STANDALONE | YARN, true, + sysProp = "spark.driver.extraJavaOptions"), + new OptionAssigner(appArgs.driverExtraLibraryPath, STANDALONE | YARN, true, + sysProp = "spark.driver.extraLibraryPath"), new OptionAssigner(appArgs.driverMemory, YARN, true, clOption = "--driver-memory"), new OptionAssigner(appArgs.name, YARN, true, clOption = "--name"), new OptionAssigner(appArgs.queue, YARN, true, clOption = "--queue"), @@ -142,10 +147,14 @@ object SparkSubmit { new OptionAssigner(appArgs.files, YARN, true, clOption = "--files"), new OptionAssigner(appArgs.archives, YARN, false, sysProp = "spark.yarn.dist.archives"), new OptionAssigner(appArgs.archives, YARN, true, clOption = "--archives"), - new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars") + new OptionAssigner(appArgs.jars, YARN, true, clOption = "--addJars"), + new OptionAssigner(appArgs.files, LOCAL | STANDALONE | MESOS, true, sysProp = "spark.files"), + new OptionAssigner(appArgs.jars, LOCAL | STANDALONE | MESOS, false, sysProp = "spark.jars"), + new OptionAssigner(appArgs.name, LOCAL | STANDALONE | MESOS, false, + sysProp = "spark.app.name") ) - // more jars + // For client mode make any added jars immediately visible on the classpath if (appArgs.jars != null && !deployOnCluster) { for (jar <- appArgs.jars.split(",")) { childClasspath += jar @@ -163,6 +172,14 @@ object SparkSubmit { } } + // For standalone mode, add the application jar automatically so the user doesn't have to + // call sc.addJar. TODO: Standalone mode in the cluster + if (clusterManager == STANDALONE) { + val existingJars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) + sysProps.put("spark.jars", (existingJars ++ Seq(appArgs.primaryResource)).mkString(",")) + println("SPARK JARS" + sysProps.get("spark.jars")) + } + if (deployOnCluster && clusterManager == STANDALONE) { if (appArgs.supervise) { childArgs += "--supervise" @@ -173,7 +190,7 @@ object SparkSubmit { childArgs += (appArgs.master, appArgs.primaryResource, appArgs.mainClass) } - // args + // Arguments to be passed to user program if (appArgs.childArgs != null) { if (!deployOnCluster || clusterManager == STANDALONE) { childArgs ++= appArgs.childArgs @@ -184,6 +201,10 @@ object SparkSubmit { } } + for ((k, v) <- appArgs.getDefaultSparkProperties) { + if (!sysProps.contains(k)) sysProps(k) = v + } + (childArgs, childClasspath, sysProps, childMainClass) } @@ -191,11 +212,11 @@ object SparkSubmit { sysProps: Map[String, String], childMainClass: String, verbose: Boolean = false) { if (verbose) { - System.err.println(s"Main class:\n$childMainClass") - System.err.println(s"Arguments:\n${childArgs.mkString("\n")}") - System.err.println(s"System properties:\n${sysProps.mkString("\n")}") - System.err.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") - System.err.println("\n") + printStream.println(s"Main class:\n$childMainClass") + printStream.println(s"Arguments:\n${childArgs.mkString("\n")}") + printStream.println(s"System properties:\n${sysProps.mkString("\n")}") + printStream.println(s"Classpath elements:\n${childClasspath.mkString("\n")}") + printStream.println("\n") } val loader = new ExecutorURLClassLoader(new Array[URL](0), @@ -226,6 +247,10 @@ object SparkSubmit { } } +/** + * Provides an indirection layer for passing arguments as system properties or flags to + * the user's driver program or to downstream launcher tools. + */ private[spark] class OptionAssigner(val value: String, val clusterManager: Int, val deployOnCluster: Boolean, diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 834b3df2f164b..02502adfbd0c4 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -17,18 +17,28 @@ package org.apache.spark.deploy -import scala.collection.mutable.ArrayBuffer +import java.io.{File, FileInputStream, IOException} +import java.util.Properties + +import scala.collection.JavaConversions._ +import scala.collection.mutable.{HashMap, ArrayBuffer} + +import org.apache.spark.SparkException /** * Parses and encapsulates arguments from the spark-submit script. */ private[spark] class SparkSubmitArguments(args: Array[String]) { - var master: String = "local" + var master: String = null var deployMode: String = null var executorMemory: String = null var executorCores: String = null var totalExecutorCores: String = null + var propertiesFile: String = null var driverMemory: String = null + var driverExtraClassPath: String = null + var driverExtraLibraryPath: String = null + var driverExtraJavaOptions: String = null var driverCores: String = null var supervise: Boolean = false var queue: String = null @@ -42,42 +52,102 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { var jars: String = null var verbose: Boolean = false - loadEnvVars() parseOpts(args.toList) + loadDefaults() + checkRequiredArguments() + + /** Return default present in the currently defined defaults file. */ + def getDefaultSparkProperties = { + val defaultProperties = new HashMap[String, String]() + if (verbose) SparkSubmit.printStream.println(s"Using properties file: $propertiesFile") + Option(propertiesFile).foreach { filename => + val file = new File(filename) + SparkSubmitArguments.getPropertiesFromFile(file).foreach { case (k, v) => + if (k.startsWith("spark")) { + defaultProperties(k) = v + if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") + } + else { + SparkSubmit.printWarning(s"Ignoring non-spark config property: $k=$v") + } + } + } + defaultProperties + } + + /** Fill in any undefined values based on the current properties file or built-in defaults. */ + private def loadDefaults() = { + + // Use common defaults file, if not specified by user + if (propertiesFile == null) { + sys.env.get("SPARK_HOME").foreach { sparkHome => + val sep = File.separator + val defaultPath = s"${sparkHome}${sep}conf${sep}spark-defaults.conf" + val file = new File(defaultPath) + if (file.exists()) { + propertiesFile = file.getAbsolutePath + } + } + } + + val defaultProperties = getDefaultSparkProperties + // Use properties file as fallback for values which have a direct analog to + // arguments in this script. + master = Option(master).getOrElse(defaultProperties.get("spark.master").orNull) + executorMemory = Option(executorMemory) + .getOrElse(defaultProperties.get("spark.executor.memory").orNull) + executorCores = Option(executorCores) + .getOrElse(defaultProperties.get("spark.executor.cores").orNull) + totalExecutorCores = Option(totalExecutorCores) + .getOrElse(defaultProperties.get("spark.cores.max").orNull) + name = Option(name).getOrElse(defaultProperties.get("spark.app.name").orNull) + jars = Option(jars).getOrElse(defaultProperties.get("spark.jars").orNull) - // Sanity checks - if (args.length == 0) printUsageAndExit(-1) - if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") - if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + // This supports env vars in older versions of Spark + master = Option(master).getOrElse(System.getenv("MASTER")) + deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) + + // Global defaults. These should be keep to minimum to avoid confusing behavior. + master = Option(master).getOrElse("local") + } + + /** Ensure that required fields exists. Call this only once all defaults are loaded. */ + private def checkRequiredArguments() = { + if (args.length == 0) printUsageAndExit(-1) + if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") + if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + } override def toString = { s"""Parsed arguments: - | master $master - | deployMode $deployMode - | executorMemory $executorMemory - | executorCores $executorCores - | totalExecutorCores $totalExecutorCores - | driverMemory $driverMemory - | drivercores $driverCores - | supervise $supervise - | queue $queue - | numExecutors $numExecutors - | files $files - | archives $archives - | mainClass $mainClass - | primaryResource $primaryResource - | name $name - | childArgs [${childArgs.mkString(" ")}] - | jars $jars - | verbose $verbose + | master $master + | deployMode $deployMode + | executorMemory $executorMemory + | executorCores $executorCores + | totalExecutorCores $totalExecutorCores + | propertiesFile $propertiesFile + | driverMemory $driverMemory + | driverCores $driverCores + | driverExtraClassPath $driverExtraClassPath + | driverExtraLibraryPath $driverExtraLibraryPath + | driverExtraJavaOptions $driverExtraJavaOptions + | supervise $supervise + | queue $queue + | numExecutors $numExecutors + | files $files + | archives $archives + | mainClass $mainClass + | primaryResource $primaryResource + | name $name + | childArgs [${childArgs.mkString(" ")}] + | jars $jars + | verbose $verbose + | + |Default properties from $propertiesFile: + |${getDefaultSparkProperties.mkString(" ", "\n ", "\n")} """.stripMargin } - private def loadEnvVars() { - Option(System.getenv("MASTER")).map(master = _) - Option(System.getenv("DEPLOY_MODE")).map(deployMode = _) - } - private def parseOpts(opts: List[String]): Unit = opts match { case ("--name") :: value :: tail => name = value @@ -122,6 +192,22 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { driverCores = value parseOpts(tail) + case ("--driver-class-path") :: value :: tail => + driverExtraClassPath = value + parseOpts(tail) + + case ("--driver-java-options") :: value :: tail => + driverExtraJavaOptions = value + parseOpts(tail) + + case ("--driver-library-path") :: value :: tail => + driverExtraLibraryPath = value + parseOpts(tail) + + case ("--properties-file") :: value :: tail => + propertiesFile = value + parseOpts(tail) + case ("--supervise") :: tail => supervise = true parseOpts(tail) @@ -154,6 +240,18 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { parseOpts(tail) case value :: tail => + if (value.startsWith("-")) { + val errMessage = s"Unrecognized option '$value'." + val suggestion: Option[String] = value match { + case v if v.startsWith("--") && v.contains("=") => + val parts = v.split("=") + Some(s"Perhaps you want '${parts(0)} ${parts(1)}'?") + case _ => + None + } + SparkSubmit.printErrorAndExit(errMessage + suggestion.map(" " + _).getOrElse("")) + } + if (primaryResource != null) { val error = s"Found two conflicting resources, $value and $primaryResource." + " Expecting only one resource." @@ -178,11 +276,21 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | --class CLASS_NAME Name of your app's main class (required for Java apps). | --arg ARG Argument to be passed to your application's main class. This | option can be specified multiple times for multiple args. - | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). | --name NAME The name of your application (Default: 'Spark'). | --jars JARS A comma-separated list of local jars to include on the | driver classpath and that SparkContext.addJar will work | with. Doesn't work on standalone with 'cluster' deploy mode. + | --files FILES Comma separated list of files to be placed in the working dir + | of each executor. + | --properties-file FILE Path to a file from which to load extra properties. If not + | specified, this will look for conf/spark-defaults.conf. + | + | --driver-memory MEM Memory for driver (e.g. 1000M, 2G) (Default: 512M). + | --driver-java-options Extra Java options to pass to the driver + | --driver-library-path Extra library path entries to pass to the driver + | --driver-class-path Extra class path entries to pass to the driver + | + | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | | Spark standalone with cluster deploy mode only: | --driver-cores NUM Cores for driver (Default: 1). @@ -193,14 +301,28 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { | | YARN-only: | --executor-cores NUM Number of cores per executor (Default: 1). - | --executor-memory MEM Memory per executor (e.g. 1000M, 2G) (Default: 1G). | --queue QUEUE_NAME The YARN queue to submit to (Default: 'default'). | --num-executors NUM Number of executors to (Default: 2). - | --files FILES Comma separated list of files to be placed in the working dir - | of each executor. | --archives ARCHIVES Comma separated list of archives to be extracted into the | working dir of each executor.""".stripMargin ) SparkSubmit.exitFn() } } + +object SparkSubmitArguments { + /** Load properties present in the given file. */ + def getPropertiesFromFile(file: File): Seq[(String, String)] = { + require(file.exists(), s"Properties file ${file.getName} does not exist") + val inputStream = new FileInputStream(file) + val properties = new Properties() + try { + properties.load(inputStream) + } catch { + case e: IOException => + val message = s"Failed when loading Spark properties file ${file.getName}" + throw new SparkException(message, e) + } + properties.stringPropertyNames().toSeq.map(k => (k, properties(k))) + } +} diff --git a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala index 63f166d401059..888dd45e93c6a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala +++ b/core/src/main/scala/org/apache/spark/deploy/client/TestClient.scala @@ -49,8 +49,8 @@ private[spark] object TestClient { val (actorSystem, port) = AkkaUtils.createActorSystem("spark", Utils.localIpAddress, 0, conf = conf, securityManager = new SecurityManager(conf)) val desc = new ApplicationDescription( - "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map()), - Some("dummy-spark-home"), "ignored") + "TestClient", Some(1), 512, Command("spark.deploy.client.TestExecutor", Seq(), Map(), Seq(), + Seq()), Some("dummy-spark-home"), "ignored") val listener = new TestListener val client = new AppClient(actorSystem, Array(url), desc, listener, new SparkConf) client.start() diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala index 0c761dfc93a1f..9103c885fa96c 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/CommandUtils.scala @@ -46,21 +46,26 @@ object CommandUtils extends Logging { * the way the JAVA_OPTS are assembled there. */ def buildJavaOpts(command: Command, memory: Int, sparkHome: String): Seq[String] = { - val libraryOpts = getEnv("SPARK_LIBRARY_PATH", command) - .map(p => List("-Djava.library.path=" + p)) - .getOrElse(Nil) - val workerLocalOpts = Option(getenv("SPARK_JAVA_OPTS")) - .map(Utils.splitCommandString).getOrElse(Nil) - val userOpts = getEnv("SPARK_JAVA_OPTS", command).map(Utils.splitCommandString).getOrElse(Nil) val memoryOpts = Seq(s"-Xms${memory}M", s"-Xmx${memory}M") + // Note, this will coalesce multiple options into a single command component + val extraOpts = command.extraJavaOptions.toSeq + val libraryOpts = + if (command.libraryPathEntries.size > 0) { + val joined = command.libraryPathEntries.mkString(File.pathSeparator) + Seq(s"-Djava.library.path=$joined") + } else { + Seq() + } // Figure out our classpath with the external compute-classpath script val ext = if (System.getProperty("os.name").startsWith("Windows")) ".cmd" else ".sh" val classPath = Utils.executeAndGetOutput( Seq(sparkHome + "/bin/compute-classpath" + ext), extraEnvironment=command.environment) + val userClassPath = command.classPathEntries.mkString(File.pathSeparator) + val classPathWithUser = classPath + File.pathSeparator + userClassPath - Seq("-cp", classPath) ++ libraryOpts ++ workerLocalOpts ++ userOpts ++ memoryOpts + Seq("-cp", classPathWithUser) ++ libraryOpts ++ extraOpts ++ memoryOpts } /** Spawn a thread that will redirect a given stream to a file */ diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index b4df1a0dd4718..f918b42c83bc6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -20,7 +20,7 @@ package org.apache.spark.deploy.worker import java.io._ import scala.collection.JavaConversions._ -import scala.collection.mutable.Map +import scala.collection.Map import akka.actor.ActorRef import com.google.common.base.Charsets @@ -74,13 +74,17 @@ private[spark] class DriverRunner( // Make sure user application jar is on the classpath // TODO: If we add ability to submit multiple jars they should also be added here - val env = Map(driverDesc.command.environment.toSeq: _*) - env("SPARK_CLASSPATH") = env.getOrElse("SPARK_CLASSPATH", "") + s":$localJarFilename" - val newCommand = Command(driverDesc.command.mainClass, - driverDesc.command.arguments.map(substituteVariables), env) + val classPath = driverDesc.command.classPathEntries ++ Seq(s"$localJarFilename") + val newCommand = Command( + driverDesc.command.mainClass, + driverDesc.command.arguments.map(substituteVariables), + driverDesc.command.environment, + classPath, + driverDesc.command.libraryPathEntries, + driverDesc.command.extraJavaOptions) val command = CommandUtils.buildCommandSeq(newCommand, driverDesc.mem, sparkHome.getAbsolutePath) - launchDriver(command, env, driverDir, driverDesc.supervise) + launchDriver(command, driverDesc.command.environment, driverDir, driverDesc.supervise) } catch { case e: Exception => finalException = Some(e) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index 2edd921066876..f94cd685e8eb0 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -99,7 +99,9 @@ private[spark] class ExecutorRunner( def getCommandSeq = { val command = Command(appDesc.command.mainClass, - appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment) + appDesc.command.arguments.map(substituteVariables) ++ Seq(appId), appDesc.command.environment, + appDesc.command.classPathEntries, appDesc.command.libraryPathEntries, + appDesc.command.extraJavaOptions) CommandUtils.buildCommandSeq(command, memory, sparkHome.getAbsolutePath) } diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index f89b2bffd1676..2bfb9c387e1c9 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -64,9 +64,10 @@ private[spark] class Executor( // to what Yarn on this system said was available. This will be used later when SparkEnv // created. if (java.lang.Boolean.valueOf( - System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) - { + System.getProperty("SPARK_YARN_MODE", System.getenv("SPARK_YARN_MODE")))) { conf.set("spark.local.dir", getYarnLocalDirs()) + } else if (sys.env.contains("SPARK_LOCAL_DIRS")) { + conf.set("spark.local.dir", sys.env("SPARK_LOCAL_DIRS")) } if (!isLocal) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala index 936e9db80573d..9544ca05dca70 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/SparkDeploySchedulerBackend.scala @@ -42,11 +42,20 @@ private[spark] class SparkDeploySchedulerBackend( // The endpoint for executors to talk to us val driverUrl = "akka.tcp://spark@%s:%s/user/%s".format( - conf.get("spark.driver.host"), conf.get("spark.driver.port"), + conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) val args = Seq(driverUrl, "{{EXECUTOR_ID}}", "{{HOSTNAME}}", "{{CORES}}", "{{WORKER_URL}}") + val extraJavaOpts = sc.conf.getOption("spark.executor.extraJavaOptions") + val classPathEntries = sys.props.get("spark.executor.extraClassPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val libraryPathEntries = sys.props.get("spark.executor.extraLibraryPath").toSeq.flatMap { cp => + cp.split(java.io.File.pathSeparator) + } + val command = Command( - "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.executorEnvs) + "org.apache.spark.executor.CoarseGrainedExecutorBackend", args, sc.testExecutorEnvs, + classPathEntries, libraryPathEntries, extraJavaOpts) val sparkHome = sc.getSparkHome() val appDesc = new ApplicationDescription(sc.appName, maxCores, sc.executorMemory, command, sparkHome, sc.ui.appUIAddress, sc.eventLogger.map(_.logDir)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala index 06b041e1fd9a9..2cd9d6c12eaf7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/CoarseMesosSchedulerBackend.scala @@ -111,7 +111,18 @@ private[spark] class CoarseMesosSchedulerBackend( def createCommand(offer: Offer, numCores: Int): CommandInfo = { val environment = Environment.newBuilder() - sc.executorEnvs.foreach { case (key, value) => + val extraClassPath = conf.getOption("spark.executor.extraClassPath") + extraClassPath.foreach { cp => + environment.addVariables( + Environment.Variable.newBuilder().setName("SPARK_CLASSPATH").setValue(cp).build()) + } + val extraJavaOpts = conf.getOption("spark.executor.extraJavaOptions") + + val libraryPathOption = "spark.executor.extraLibraryPath" + val extraLibraryPath = conf.getOption(libraryPathOption).map(p => s"-Djava.library.path=$p") + val extraOpts = Seq(extraJavaOpts, extraLibraryPath).flatten.mkString(" ") + + sc.testExecutorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) @@ -123,20 +134,22 @@ private[spark] class CoarseMesosSchedulerBackend( conf.get("spark.driver.host"), conf.get("spark.driver.port"), CoarseGrainedSchedulerBackend.ACTOR_NAME) + val uri = conf.get("spark.executor.uri", null) if (uri == null) { val runScript = new File(sparkHome, "./bin/spark-class").getCanonicalPath command.setValue( - "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d".format( - runScript, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "\"%s\" org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d".format( + runScript, extraOpts, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) } else { // Grab everything to the first '.'. We'll use that and '*' to // glob the directory "correctly". val basename = uri.split('/').last.split('.').head command.setValue( ("cd %s*; " + - "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %d") - .format(basename, driverUrl, offer.getSlaveId.getValue, offer.getHostname, numCores)) + "./bin/spark-class org.apache.spark.executor.CoarseGrainedExecutorBackend %s %s %s %s %d") + .format(basename, extraOpts, driverUrl, offer.getSlaveId.getValue, + offer.getHostname, numCores)) command.addUris(CommandInfo.URI.newBuilder().setValue(uri)) } command.build() diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala index dfdcafe19fb93..c975f312324ed 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/mesos/MesosSchedulerBackend.scala @@ -90,7 +90,7 @@ private[spark] class MesosSchedulerBackend( "Spark home is not set; set it through the spark.home system " + "property, the SPARK_HOME environment variable or the SparkContext constructor")) val environment = Environment.newBuilder() - sc.executorEnvs.foreach { case (key, value) => + sc.testExecutorEnvs.foreach { case (key, value) => environment.addVariables(Environment.Variable.newBuilder() .setName(key) .setValue(value) diff --git a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala index 9f2924c23b73c..bfae32dae0dc5 100644 --- a/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/JsonProtocolSuite.scala @@ -87,7 +87,7 @@ class JsonProtocolSuite extends FunSuite { } def createAppDesc(): ApplicationDescription = { - val cmd = new Command("mainClass", List("arg1", "arg2"), Map()) + val cmd = new Command("mainClass", List("arg1", "arg2"), Map(), Seq(), Seq()) new ApplicationDescription("name", Some(4), 1234, cmd, Some("sparkHome"), "appUiUrl") } @@ -100,7 +100,7 @@ class JsonProtocolSuite extends FunSuite { def createDriverCommand() = new Command( "org.apache.spark.FakeClass", Seq("some arg --and-some options -g foo"), - Map(("K1", "V1"), ("K2", "V2")) + Map(("K1", "V1"), ("K2", "V2")), Seq("cp1", "cp2"), Seq("lp1", "lp2"), Some("-Dfoo") ) def createDriverDesc() = new DriverDescription("hdfs://some-dir/some.jar", 100, 3, @@ -133,9 +133,12 @@ class JsonProtocolSuite extends FunSuite { def assertValidDataInJson(validateJson: JValue, expectedJson: JValue) { val Diff(c, a, d) = validateJson diff expectedJson - assert(c === JNothing, "Json changed") - assert(a === JNothing, "Json added") - assert(d === JNothing, "Json deleted") + val validatePretty = JsonMethods.pretty(validateJson) + val expectedPretty = JsonMethods.pretty(expectedJson) + val errorMessage = s"Expected:\n$expectedPretty\nFound:\n$validatePretty" + assert(c === JNothing, s"$errorMessage\nChanged:\n${JsonMethods.pretty(c)}") + assert(a === JNothing, s"$errorMessage\nAdded:\n${JsonMethods.pretty(a)}") + assert(d === JNothing, s"$errorMessage\nDelected:\n${JsonMethods.pretty(d)}") } } @@ -165,7 +168,7 @@ object JsonConstants { """ |{"name":"name","cores":4,"memoryperslave":1234, |"user":"%s","sparkhome":"sparkHome", - |"command":"Command(mainClass,List(arg1, arg2),Map())"} + |"command":"Command(mainClass,List(arg1, arg2),Map(),List(),List(),None)"} """.format(System.getProperty("user.name", "")).stripMargin val executorRunnerJsonStr = diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 4e489cd9b66a6..f82d717719b69 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -17,16 +17,16 @@ package org.apache.spark.deploy -import java.io.{OutputStream, PrintStream} +import java.io.{File, OutputStream, PrintStream} import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{SparkConf, SparkContext, SparkEnv, SparkException, TestUtils} +import org.apache.spark.deploy.SparkSubmit._ +import org.apache.spark.util.Utils import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers -import org.apache.spark.deploy.SparkSubmit._ - - class SparkSubmitSuite extends FunSuite with ShouldMatchers { val noOpOutputStream = new OutputStream { @@ -42,7 +42,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } /** Returns true if the script exits and the given search string is printed. */ - def testPrematureExit(input: Array[String], searchString: String): Boolean = { + def testPrematureExit(input: Array[String], searchString: String) = { val printStream = new BufferPrintStream() SparkSubmit.printStream = printStream @@ -60,28 +60,38 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } thread.start() thread.join() - printStream.lineBuffer.find(s => s.contains(searchString)).size > 0 + val joined = printStream.lineBuffer.mkString("\n") + if (!joined.contains(searchString)) { + fail(s"Search string '$searchString' not found in $joined") + } } test("prints usage on empty input") { - testPrematureExit(Array[String](), "Usage: spark-submit") should be (true) + testPrematureExit(Array[String](), "Usage: spark-submit") } test("prints usage with only --help") { - testPrematureExit(Array("--help"), "Usage: spark-submit") should be (true) + testPrematureExit(Array("--help"), "Usage: spark-submit") + } + + test("prints error with unrecognized option") { + testPrematureExit(Array("--blarg"), "Unrecognized option '--blarg'") + testPrematureExit(Array("-bleg"), "Unrecognized option '-bleg'") + testPrematureExit(Array("--master=abc"), + "Unrecognized option '--master=abc'. Perhaps you want '--master abc'?") } test("handles multiple binary definitions") { val adjacentJars = Array("foo.jar", "bar.jar") - testPrematureExit(adjacentJars, "error: Found two conflicting resources") should be (true) + testPrematureExit(adjacentJars, "error: Found two conflicting resources") val nonAdjacentJars = Array("foo.jar", "--master", "123", "--class", "abc", "bar.jar") - testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") should be (true) + testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") } test("handle binary specified but not class") { - testPrematureExit(Array("foo.jar"), "must specify a main class") + testPrematureExit(Array("foo.jar"), "Must specify a main class") } test("handles YARN cluster mode") { @@ -140,12 +150,11 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") - print("child args: " + childArgsStr) childArgsStr.startsWith("--memory 4g --cores 5 --supervise") should be (true) childArgsStr should include ("launch spark://h:p thejar.jar org.SomeClass arg1 arg2") mainClass should be ("org.apache.spark.deploy.Client") classpath should have length (0) - sysProps should have size (0) + sysProps should have size (1) // contains --jar entry } test("handles standalone client mode") { @@ -175,4 +184,80 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { sysProps("spark.executor.memory") should be ("5g") sysProps("spark.cores.max") should be ("5") } + + test("launch simple application with spark-submit") { + runSparkSubmit( + Seq("unUsed.jar", + "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local")) + } + + test("spark submit includes jars passed in through --jar") { + val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) + val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) + val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") + runSparkSubmit( + Seq("unUsed.jar", + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,512]", + "--jars", jarsString)) + } + + // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. + def runSparkSubmit(args: Seq[String]): String = { + val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")).get + Utils.executeAndGetOutput( + Seq("./bin/spark-submit") ++ args, + new File(sparkHome), + Map("SPARK_TESTING" -> "1", "SPARK_HOME" -> sparkHome)) + } +} + +object JarCreationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + val result = sc.makeRDD(1 to 100, 10).mapPartitions{ x => + var foundClasses = false + try { + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + Class.forName("SparkSubmitClassA", true, Thread.currentThread().getContextClassLoader) + foundClasses = true + } catch { + case _: Throwable => // catch all + } + Seq(foundClasses).iterator + }.collect() + if (result.contains(false)) { + throw new Exception("Could not load user defined classes inside of executors") + } + } +} + +object SimpleApplicationTest { + def main(args: Array[String]) { + val conf = new SparkConf() + val sc = new SparkContext(conf) + + val configs = Seq("spark.master", "spark.app.name") + for (config <- configs) { + val masterValue = conf.get(config) + val executorValues = sc + .makeRDD(1 to 100, 10) + .map(x => SparkEnv.get.conf.get(config)) + .collect() + .distinct + if (executorValues.size != 1) { + throw new SparkException(s"Inconsistent values for $config: $executorValues") + } + val executorValue = executorValues(0) + if (executorValue != masterValue) { + throw new SparkException( + s"Master had $config=$masterValue but executor had $config=$executorValue") + } + } + + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala index a2c131b0c9787..4633bc3f7f25e 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/DriverRunnerTest.scala @@ -29,7 +29,7 @@ import org.apache.spark.deploy.{Command, DriverDescription} class DriverRunnerTest extends FunSuite { private def createDriverRunner() = { - val command = new Command("mainClass", Seq(), Map()) + val command = new Command("mainClass", Seq(), Map(), Seq(), Seq()) val driverDescription = new DriverDescription("jarUrl", 512, 1, true, command) new DriverRunner("driverId", new File("workDir"), new File("sparkHome"), driverDescription, null, "akka://1.2.3.4/worker/") diff --git a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala index 3cab8e7b37934..8ae387fa0be6f 100644 --- a/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala +++ b/core/src/test/scala/org/apache/spark/deploy/worker/ExecutorRunnerTest.scala @@ -27,7 +27,8 @@ class ExecutorRunnerTest extends FunSuite { test("command includes appId") { def f(s:String) = new File(s) val sparkHome = sys.env.get("SPARK_HOME").orElse(sys.props.get("spark.home")) - val appDesc = new ApplicationDescription("app name", Some(8), 500, Command("foo", Seq(),Map()), + val appDesc = new ApplicationDescription("app name", Some(8), 500, + Command("foo", Seq(), Map(), Seq(), Seq()), sparkHome, "appUiUrl") val appId = "12345-worker321-9876" val er = new ExecutorRunner(appId, 1, appDesc, 8, 500, null, "blah", "worker321", f(sparkHome.getOrElse(".")), diff --git a/dev/audit-release/audit_release.py b/dev/audit-release/audit_release.py index fa2f02dfecc75..4a816d4101e57 100755 --- a/dev/audit-release/audit_release.py +++ b/dev/audit-release/audit_release.py @@ -114,6 +114,7 @@ def ensure_path_not_present(x): os.environ["SPARK_VERSION"] = RELEASE_VERSION os.environ["SCALA_VERSION"] = SCALA_VERSION os.environ["SPARK_RELEASE_REPOSITORY"] = RELEASE_REPOSITORY +os.environ["SPARK_AUDIT_MASTER"] = "local" for module in modules: os.environ["SPARK_MODULE"] = module ret = run_cmd("sbt clean update", exit_on_failure=False) diff --git a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala index 53fe43215e40e..a89b0d7d38bf1 100644 --- a/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala +++ b/dev/audit-release/sbt_app_core/src/main/scala/SparkApp.scala @@ -24,8 +24,13 @@ import org.apache.spark.SparkContext._ object SimpleApp { def main(args: Array[String]) { + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Spark App").setMaster(master) + case None => new SparkConf().setAppName("Simple Spark App") + } val logFile = "input.txt" - val sc = new SparkContext("local", "Simple App") + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() diff --git a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala index da08e014ebd94..24c7f8d667296 100644 --- a/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala +++ b/dev/audit-release/sbt_app_graphx/src/main/scala/GraphxApp.scala @@ -17,14 +17,20 @@ package main.scala -import org.apache.spark.SparkContext +import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.SparkContext._ import org.apache.spark.graphx._ import org.apache.spark.rdd.RDD object GraphXApp { def main(args: Array[String]) { - val sc = new SparkContext("local", "Simple GraphX App") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple GraphX App").setMaster(master) + case None => new SparkConf().setAppName("Simple Graphx App") + } + val sc = new SparkContext(conf) + SparkContext.jarOfClass(this.getClass).foreach(sc.addJar) + val users: RDD[(VertexId, (String, String))] = sc.parallelize(Array((3L, ("rxin", "student")), (7L, ("jgonzal", "postdoc")), (5L, ("franklin", "prof")), (2L, ("istoica", "prof")), diff --git a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala index 3d0722d2ac45e..a1d8971abe9a4 100644 --- a/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala +++ b/dev/audit-release/sbt_app_streaming/src/main/scala/StreamingApp.scala @@ -27,10 +27,12 @@ import org.apache.spark.streaming._ object SparkStreamingExample { def main(args: Array[String]) { - val conf = new SparkConf(true) - .setMaster("local[2]") - .setAppName("Streaming test") + val conf = sys.env.get("SPARK_AUDIT_MASTER") match { + case Some(master) => new SparkConf().setAppName("Simple Streaming App").setMaster(master) + case None => new SparkConf().setAppName("Simple Streaming App") + } val ssc = new StreamingContext(conf, Seconds(1)) + SparkContext.jarOfClass(this.getClass).foreach(ssc.sparkContext.addJar) val seen = ListBuffer[RDD[Int]]() val rdd1 = ssc.sparkContext.makeRDD(1 to 100, 10) diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index 7f75ea44e4cea..dcc063042628c 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -50,61 +50,78 @@ The system currently supports three cluster managers: In addition, Spark's [EC2 launch scripts](ec2-scripts.html) make it easy to launch a standalone cluster on Amazon EC2. -# Launching Applications - -The recommended way to launch a compiled Spark application is through the spark-submit script (located in the -bin directory), which takes care of setting up the classpath with Spark and its dependencies, as well as -provides a layer over the different cluster managers and deploy modes that Spark supports. It's usage is - - spark-submit `` `` - -Where options are any of: - -- **\--class** - The main class to run. -- **\--master** - The URL of the cluster manager master, e.g. spark://host:port, mesos://host:port, yarn, - or local. -- **\--deploy-mode** - "client" to run the driver in the client process or "cluster" to run the driver in - a process on the cluster. For Mesos, only "client" is supported. -- **\--executor-memory** - Memory per executor (e.g. 1000M, 2G). -- **\--executor-cores** - Number of cores per executor. (Default: 2) -- **\--driver-memory** - Memory for driver (e.g. 1000M, 2G) -- **\--name** - Name of the application. -- **\--arg** - Argument to be passed to the application's main class. This option can be specified - multiple times to pass multiple arguments. -- **\--jars** - A comma-separated list of local jars to include on the driver classpath and that - SparkContext.addJar will work with. Doesn't work on standalone with 'cluster' deploy mode. - -The following currently only work for Spark standalone with cluster deploy mode: - -- **\--driver-cores** - Cores for driver (Default: 1). -- **\--supervise** - If given, restarts the driver on failure. - -The following only works for Spark standalone and Mesos only: - -- **\--total-executor-cores** - Total cores for all executors. - -The following currently only work for YARN: - -- **\--queue** - The YARN queue to place the application in. -- **\--files** - Comma separated list of files to be placed in the working dir of each executor. -- **\--archives** - Comma separated list of archives to be extracted into the working dir of each - executor. -- **\--num-executors** - Number of executors (Default: 2). - -The master and deploy mode can also be set with the MASTER and DEPLOY_MODE environment variables. -Values for these options passed via command line will override the environment variables. - -# Shipping Code to the Cluster - -The recommended way to ship your code to the cluster is to pass it through SparkContext's constructor, -which takes a list of JAR files (Java/Scala) or .egg and .zip libraries (Python) to disseminate to -worker nodes. You can also dynamically add new files to be sent to executors with `SparkContext.addJar` -and `addFile`. - -## URIs for addJar / addFile - -- **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and every executor - pulls the file from the driver HTTP server +# Bundling and Launching Applications + +### Bundling Your Application's Dependencies +If your code depends on other projects, you will need to package them alongside +your application in order to distribute the code to a Spark cluster. To do this, +to create an assembly jar (or "uber" jar) containing your code and its dependencies. Both +[sbt](https://github.com/sbt/sbt-assembly) and +[Maven](http://maven.apache.org/plugins/maven-shade-plugin/) +have assembly plugins. When creating assembly jars, list Spark and Hadoop +as `provided` dependencies; these need not be bundled since they are provided by +the cluster manager at runtime. Once you have an assembled jar you can call the `bin/spark-submit` +script as shown here while passing your jar. + +For Python, you can use the `pyFiles` argument of SparkContext +or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. + +### Launching Applications with ./bin/spark-submit + +Once a user application is bundled, it can be launched using the `spark-submit` script located in +the bin directory. This script takes care of setting up the classpath with Spark and its +dependencies, and can support different cluster managers and deploy modes that Spark supports. +It's usage is + + ./bin/spark-submit --class path.to.your.Class [other options..] + +To enumerate all options available to `spark-submit` run it with the `--help` flag. +Here are a few examples of common options: + +{% highlight bash %} +# Run application locally +./bin/spark-submit my-app.jar \ + --class my.main.ClassName + --master local[8] + +# Run on a Spark cluster +./bin/spark-submit my-app.jar \ + --class my.main.ClassName + --master spark://mycluster:7077 \ + --executor-memory 20G \ + --total-executor-cores 100 + +# Run on a YARN cluster +HADOOP_CONF_DIR=XX /bin/spark-submit my-app.jar \ + --class my.main.ClassName + --master yarn-cluster \ # can also be `yarn-client` for client mode + --executor-memory 20G \ + --num-executors 50 +{% endhighlight %} + +### Loading Configurations from a File + +The `spark-submit` script can load default `SparkConf` values from a properties file and pass them +onto your application. By default it will read configuration options from +`conf/spark-defaults.conf`. Any values specified in the file will be passed on to the +application when run. They can obviate the need for certain flags to `spark-submit`: for +instance, if `spark.master` property is set, you can safely omit the +`--master` flag from `spark-submit`. In general, configuration values explicitly set on a +`SparkConf` take the highest precedence, then flags passed to `spark-submit`, then values +in the defaults file. + +If you are ever unclear where configuration options are coming from. fine-grained debugging +information can be printed by adding the `--verbose` option to `./spark-submit`. + +### Advanced Dependency Management +When using `./bin/spark-submit` jars will be automatically transferred to the cluster. For many +users this is sufficient. However, advanced users can add jars by calling `addFile` or `addJar` +on an existing SparkContext. This can be used to distribute JAR files (Java/Scala) or .egg and +.zip libraries (Python) to executors. Spark uses the following URL scheme to allow different +strategies for disseminating jars: + +- **file:** - Absolute paths and `file:/` URIs are served by the driver's HTTP file server, and + every executor pulls the file from the driver HTTP server - **hdfs:**, **http:**, **https:**, **ftp:** - these pull down files and JARs from the URI as expected - **local:** - a URI starting with local:/ is expected to exist as a local file on each worker node. This means that no network IO will be incurred, and works well for large files/JARs that are pushed to each worker, @@ -138,6 +155,14 @@ The following table summarizes terms you'll see used to refer to cluster concept Application User program built on Spark. Consists of a driver program and executors on the cluster. + + Application jar + + A jar containing the user's Spark application. In some cases users will want to create + an "uber jar" containing their application along with its dependencies. The user's jar + should never include Hadoop or Spark libraries, however, these will be added at runtime. + + Driver program The process running the main() function of the application and creating the SparkContext diff --git a/docs/configuration.md b/docs/configuration.md index a3029837ff0cd..5a4abca2646b6 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -73,6 +73,9 @@ there are at least five properties that you will commonly want to control: Directory to use for "scratch" space in Spark, including map output files and RDDs that get stored on disk. This should be on a fast, local disk in your system. It can also be a comma-separated list of multiple directories on different disks. + + NOTE: In Spark 1.0 and later this will be overriden by SPARK_LOCAL_DIRS (Standalone, Mesos) or + LOCAL_DIRS (YARN) envrionment variables set by the cluster manager. @@ -578,7 +581,7 @@ Apart from these, the following properties are also available, and may be useful to consolidate them onto as few nodes as possible. Spreading out is usually better for data locality in HDFS, but consolidating is more efficient for compute-intensive workloads.
      Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. @@ -591,7 +594,7 @@ Apart from these, the following properties are also available, and may be useful Set this lower on a shared cluster to prevent users from grabbing the whole cluster by default.
      Note: this setting needs to be configured in the standalone cluster master, not in individual - applications; you can set it through SPARK_JAVA_OPTS in spark-env.sh. + applications; you can set it through SPARK_MASTER_OPTS in spark-env.sh. @@ -649,6 +652,34 @@ Apart from these, the following properties are also available, and may be useful Number of cores to allocate for each task. + + spark.executor.extraJavaOptions + (none) + + A string of extra JVM options to pass to executors. For instance, GC settings or other + logging. Note that it is illegal to set Spark properties or heap size settings with this + option. Spark properties should be set using a SparkConf object or the + spark-defaults.conf file used with the spark-submit script. Heap size settings can be set + with spark.executor.memory. + + + + spark.executor.extraClassPath + (none) + + Extra classpath entries to append to the classpath of executors. This exists primarily + for backwards-compatibility with older versions of Spark. Users typically should not need + to set this option. + + + + spark.executor.extraLibraryPath + (none) + + Set a special library path to use when launching executor JVM's. + + + ## Viewing Spark Properties @@ -659,10 +690,9 @@ This is a useful place to check to make sure that your properties have been set # Environment Variables Certain Spark settings can be configured through environment variables, which are read from the `conf/spark-env.sh` -script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). These variables are meant to be for machine-specific settings, such -as library search paths. While Spark properties can also be set there through `SPARK_JAVA_OPTS`, for per-application settings, we recommend setting -these properties within the application instead of in `spark-env.sh` so that different applications can use different -settings. +script in the directory where Spark is installed (or `conf/spark-env.cmd` on Windows). In Standalone and Mesos modes, +this file can give machine specific information such as hostnames. It is also sourced when running local +Spark applications or submission scripts. Note that `conf/spark-env.sh` does not exist by default when Spark is installed. However, you can copy `conf/spark-env.sh.template` to create it. Make sure you make the copy executable. @@ -672,13 +702,7 @@ The following variables can be set in `spark-env.sh`: * `JAVA_HOME`, the location where Java is installed (if it's not on your default `PATH`) * `PYSPARK_PYTHON`, the Python binary to use for PySpark * `SPARK_LOCAL_IP`, to configure which IP address of the machine to bind to. -* `SPARK_LIBRARY_PATH`, to add search directories for native libraries. -* `SPARK_CLASSPATH`, to add elements to Spark's classpath that you want to be present for _all_ applications. - Note that applications can also add dependencies for themselves through `SparkContext.addJar` -- we recommend - doing that when possible. -* `SPARK_JAVA_OPTS`, to add JVM options. This includes Java options like garbage collector settings and any system - properties that you'd like to pass with `-D`. One use case is to set some Spark properties differently on this - machine, e.g., `-Dspark.local.dir=/disk1,/disk2`. +* `SPARK_PUBLIC_DNS`, the hostname your Spark program will advertise to other machines. * Options for the Spark [standalone cluster scripts](spark-standalone.html#cluster-launch-scripts), such as number of cores to use on each machine and maximum memory. diff --git a/docs/quick-start.md b/docs/quick-start.md index 60e8b1ba0eb46..6b4f4ba4254a2 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -99,23 +99,32 @@ scala> linesWithSpark.count() res9: Long = 15 {% endhighlight %} -It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is that these same functions can be used on very large data sets, even when they are striped across tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). +It may seem silly to use Spark to explore and cache a 30-line text file. The interesting part is +that these same functions can be used on very large data sets, even when they are striped across +tens or hundreds of nodes. You can also do this interactively by connecting `bin/spark-shell` to +a cluster, as described in the [programming guide](scala-programming-guide.html#initializing-spark). -# A Standalone App in Scala -Now say we wanted to write a standalone application using the Spark API. We will walk through a simple application in both Scala (with SBT), Java (with Maven), and Python. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. +# A Standalone Application +Now say we wanted to write a standalone application using the Spark API. We will walk through a +simple application in both Scala (with SBT), Java (with Maven), and Python. -We'll create a very simple Spark application in Scala. So simple, in fact, that it's named `SimpleApp.scala`: +
      +
      + +We'll create a very simple Spark application in Scala. So simple, in fact, that it's +named `SimpleApp.scala`: {% highlight scala %} /*** SimpleApp.scala ***/ import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.SparkConf object SimpleApp { def main(args: Array[String]) { - val logFile = "$YOUR_SPARK_HOME/README.md" // Should be some file on your system - val sc = new SparkContext("local", "Simple App", "YOUR_SPARK_HOME", - List("target/scala-{{site.SCALA_BINARY_VERSION}}/simple-project_{{site.SCALA_BINARY_VERSION}}-1.0.jar")) + val logFile = "YOUR_SPARK_HOME/README.md" // Should be some file on your system + val conf = new SparkConf().setAppName("Simple Application") + val sc = new SparkContext(conf) val logData = sc.textFile(logFile, 2).cache() val numAs = logData.filter(line => line.contains("a")).count() val numBs = logData.filter(line => line.contains("b")).count() @@ -124,9 +133,17 @@ object SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in the Spark README. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. We pass the SparkContext constructor four arguments, the type of scheduler we want to use (in this case, a local scheduler), a name for the application, the directory where Spark is installed, and a name for the jar file containing the application's code. The final two arguments are needed in a distributed setting, where Spark is running across several nodes, so we include them for completeness. Spark will automatically ship the jar files you list to slave nodes. +This program just counts the number of lines containing 'a' and the number containing 'b' in the +Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is +installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, +we initialize a SparkContext as part of the program. + +We pass the SparkContext constructor a SparkConf object which contains information about our +application. We also call sc.addJar to make sure that when our application is launched in cluster +mode, the jar file containing it will be shipped automatically to worker nodes. -This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` which explains that Spark is a dependency. This file also adds a repository that Spark depends on: +This file depends on the Spark API, so we'll also include an sbt configuration file, `simple.sbt` +which explains that Spark is a dependency. This file also adds a repository that Spark depends on: {% highlight scala %} name := "Simple Project" @@ -140,15 +157,12 @@ libraryDependencies += "org.apache.spark" %% "spark-core" % "{{site.SPARK_VERSIO resolvers += "Akka Repository" at "http://repo.akka.io/releases/" {% endhighlight %} -If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS: - -{% highlight scala %} -libraryDependencies += "org.apache.hadoop" % "hadoop-client" % "" -{% endhighlight %} - -Finally, for sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` according to the typical directory structure. Once that is in place, we can create a JAR package containing the application's code, then use `sbt/sbt run` to execute our program. +For sbt to work correctly, we'll need to layout `SimpleApp.scala` and `simple.sbt` +according to the typical directory structure. Once that is in place, we can create a JAR package +containing the application's code, then use the `spark-submit` script to run our program. {% highlight bash %} +# Your directory layout should look like this $ find . . ./simple.sbt @@ -157,27 +171,36 @@ $ find . ./src/main/scala ./src/main/scala/SimpleApp.scala -$ sbt/sbt package -$ sbt/sbt run +# Package a jar containing your application +$ sbt package +... +[info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar + +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit target/scala-2.10/simple-project_2.10-1.0.jar \ + --class "SimpleApp" \ + --master local[4] ... Lines with a: 46, Lines with b: 23 {% endhighlight %} -# A Standalone App in Java -Now say we wanted to write a standalone application using the Java API. We will walk through doing this with Maven. If you are using other build systems, consider using the Spark assembly JAR described in the developer guide. +
      +
      +This example will use Maven to compile an application jar, but any similar build system will work. We'll create a very simple Spark application, `SimpleApp.java`: {% highlight java %} /*** SimpleApp.java ***/ import org.apache.spark.api.java.*; +import org.apache.spark.SparkConf; import org.apache.spark.api.java.function.Function; public class SimpleApp { public static void main(String[] args) { - String logFile = "$YOUR_SPARK_HOME/README.md"; // Should be some file on your system - JavaSparkContext sc = new JavaSparkContext("local", "Simple App", - "$YOUR_SPARK_HOME", new String[]{"target/simple-project-1.0.jar"}); + String logFile = "YOUR_SPARK_HOME/README.md"; // Should be some file on your system + SparkConf conf = new SparkConf().setAppName("Simple Application"); + JavaSparkContext sc = new JavaSparkContext(conf); JavaRDD logData = sc.textFile(logFile).cache(); long numAs = logData.filter(new Function() { @@ -193,9 +216,16 @@ public class SimpleApp { } {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala example, we initialize a SparkContext, though we use the special `JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by `JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes that extend `spark.api.java.function.Function`. The [Java programming guide](java-programming-guide.html) describes these differences in more detail. +This program just counts the number of lines containing 'a' and the number containing 'b' in a text +file. Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. +As with the Scala example, we initialize a SparkContext, though we use the special +`JavaSparkContext` class to get a Java-friendly one. We also create RDDs (represented by +`JavaRDD`) and run transformations on them. Finally, we pass functions to Spark by creating classes +that extend `spark.api.java.function.Function`. The +[Java programming guide](java-programming-guide.html) describes these differences in more detail. -To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. Note that Spark artifacts are tagged with a Scala version. +To build the program, we also write a Maven `pom.xml` file that lists Spark as a dependency. +Note that Spark artifacts are tagged with a Scala version. {% highlight xml %} @@ -221,16 +251,6 @@ To build the program, we also write a Maven `pom.xml` file that lists Spark as a {% endhighlight %} -If you also wish to read data from Hadoop's HDFS, you will also need to add a dependency on `hadoop-client` for your version of HDFS: - -{% highlight xml %} - - org.apache.hadoop - hadoop-client - ... - -{% endhighlight %} - We lay out these files according to the canonical Maven directory structure: {% highlight bash %} $ find . @@ -241,16 +261,25 @@ $ find . ./src/main/java/SimpleApp.java {% endhighlight %} -Now, we can execute the application using Maven: +Now, we can package the application using Maven and execute it with `./bin/spark-submit`. {% highlight bash %} +# Package a jar containing your application $ mvn package -$ mvn exec:java -Dexec.mainClass="SimpleApp" +... +[INFO] Building jar: {..}/{..}/target/simple-project-1.0.jar + +# Use spark-submit to run your application +$ YOUR_SPARK_HOME/bin/spark-submit target/simple-project-1.0.jar \ + --class "SimpleApp" \ + --master local[4] ... Lines with a: 46, Lines with b: 23 {% endhighlight %} -# A Standalone App in Python +
      +
      + Now we will show how to write a standalone application using the Python API (PySpark). As an example, we'll create a simple Spark application, `SimpleApp.py`: @@ -259,7 +288,7 @@ As an example, we'll create a simple Spark application, `SimpleApp.py`: """SimpleApp.py""" from pyspark import SparkContext -logFile = "$YOUR_SPARK_HOME/README.md" # Should be some file on your system +logFile = "YOUR_SPARK_HOME/README.md" # Should be some file on your system sc = SparkContext("local", "Simple App") logData = sc.textFile(logFile).cache() @@ -270,11 +299,15 @@ print "Lines with a: %i, lines with b: %i" % (numAs, numBs) {% endhighlight %} -This program just counts the number of lines containing 'a' and the number containing 'b' in a text file. -Note that you'll need to replace $YOUR_SPARK_HOME with the location where Spark is installed. +This program just counts the number of lines containing 'a' and the number containing 'b' in a +text file. +Note that you'll need to replace YOUR_SPARK_HOME with the location where Spark is installed. As with the Scala and Java examples, we use a SparkContext to create RDDs. -We can pass Python functions to Spark, which are automatically serialized along with any variables that they reference. -For applications that use custom classes or third-party libraries, we can add those code dependencies to SparkContext to ensure that they will be available on remote machines; this is described in more detail in the [Python programming guide](python-programming-guide.html). +We can pass Python functions to Spark, which are automatically serialized along with any variables +that they reference. +For applications that use custom classes or third-party libraries, we can add those code +dependencies to SparkContext to ensure that they will be available on remote machines; this is +described in more detail in the [Python programming guide](python-programming-guide.html). `SimpleApp` is simple enough that we do not need to specify any code dependencies. We can run this application using the `bin/pyspark` script: @@ -286,57 +319,12 @@ $ ./bin/pyspark SimpleApp.py Lines with a: 46, Lines with b: 23 {% endhighlight python %} -# Running on a Cluster - -There are a few additional considerations when running applicaitons on a -[Spark](spark-standalone.html), [YARN](running-on-yarn.html), or -[Mesos](running-on-mesos.html) cluster. - -### Including Your Dependencies -If your code depends on other projects, you will need to ensure they are also -present on the slave nodes. A popular approach is to create an -assembly jar (or "uber" jar) containing your code and its dependencies. Both -[sbt](https://github.com/sbt/sbt-assembly) and -[Maven](http://maven.apache.org/plugins/maven-assembly-plugin/) -have assembly plugins. When creating assembly jars, list Spark -itself as a `provided` dependency; it need not be bundled since it is -already present on the slaves. Once you have an assembled jar, -add it to the SparkContext as shown here. It is also possible to add -your dependent jars one-by-one using the `addJar` method of `SparkContext`. - -For Python, you can use the `pyFiles` argument of SparkContext -or its `addPyFile` method to add `.py`, `.zip` or `.egg` files to be distributed. - -### Setting Configuration Options -Spark includes several [configuration options](configuration.html#spark-properties) -that influence the behavior of your application. -These should be set by building a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) -object and passing it to the SparkContext constructor. -For example, in Java and Scala, you can do: - -{% highlight scala %} -import org.apache.spark.{SparkConf, SparkContext} -val conf = new SparkConf() - .setMaster("local") - .setAppName("My application") - .set("spark.executor.memory", "1g") -val sc = new SparkContext(conf) -{% endhighlight %} - -Or in Python: - -{% highlight scala %} -from pyspark import SparkConf, SparkContext -conf = SparkConf() -conf.setMaster("local") -conf.setAppName("My application") -conf.set("spark.executor.memory", "1g")) -sc = SparkContext(conf = conf) -{% endhighlight %} +
      +
      -### Accessing Hadoop Filesystems +# Where to go from here +Congratulations on running your first Spark application! -The examples here access a local file. To read data from a distributed -filesystem, such as HDFS, include -[Hadoop version information](index.html#a-note-about-hadoop-versions) -in your build file. By default, Spark builds against HDFS 1.0.4. +* For an in-depth overview of the API see "Programming Guides" menu section. +* For running applications on a cluster head to the [deployment overview](cluster-overview.html). +* For configuration options available to Spark applications see the [configuration page](configuration.html). \ No newline at end of file diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 2b0a51e9dfc54..4431da0721ac7 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -25,36 +25,43 @@ To write a Spark application, you need to add a dependency on Spark. If you use artifactId = spark-core_{{site.SCALA_BINARY_VERSION}} version = {{site.SPARK_VERSION}} -In addition, if you wish to access an HDFS cluster, you need to add a dependency on `hadoop-client` for your version of HDFS: +In addition, if you wish to access an HDFS cluster, you need to add a dependency on +`hadoop-client` for your version of HDFS. Some common HDFS version tags are listed on the +[third party distributions](hadoop-third-party-distributions.html) page. groupId = org.apache.hadoop artifactId = hadoop-client version = -For other build systems, you can run `sbt/sbt assembly` to pack Spark and its dependencies into one JAR (`assembly/target/scala-{{site.SCALA_BINARY_VERSION}}/spark-assembly-{{site.SPARK_VERSION}}-hadoop*.jar`), then add this to your CLASSPATH. Set the HDFS version as described [here](index.html#a-note-about-hadoop-versions). - Finally, you need to import some Spark classes and implicit conversions into your program. Add the following lines: {% highlight scala %} import org.apache.spark.SparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.SparkConf {% endhighlight %} # Initializing Spark -The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark how to access a cluster. -This is done through the following constructor: +The first thing a Spark program must do is to create a `SparkContext` object, which tells Spark +how to access a cluster. To create a `SparkContext` you first need to build a `SparkConf` object +that contains information about your application. {% highlight scala %} -new SparkContext(master, appName, [sparkHome], [jars]) +val conf = new SparkConf().setAppName().setMaster() +new SparkContext(conf) {% endhighlight %} -or through `new SparkContext(conf)`, which takes a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) -object for more advanced configuration. - -The `master` parameter is a string specifying a [Spark or Mesos cluster URL](#master-urls) to connect to, or a special "local" string to run in local mode, as described below. `appName` is a name for your application, which will be shown in the cluster web UI. Finally, the last two parameters are needed to deploy your code to a cluster if running in distributed mode, as described later. +The `` parameter is a string specifying a [Spark, Mesos or YARN cluster URL](#master-urls) +to connect to, or a special "local" string to run in local mode, as described below. `` is +a name for your application, which will be shown in the cluster web UI. It's also possible to set +these variables [using a configuration file](cluster-overview.html#loading-configurations-from-a-file) +which avoids hard-coding the master name in your application. -In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the context connects to using the `MASTER` environment variable, and you can add JARs to the classpath with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use +In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the +variable called `sc`. Making your own SparkContext will not work. You can set which master the +context connects to using the `MASTER` environment variable, and you can add JARs to the classpath +with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use {% highlight bash %} $ MASTER=local[4] ./bin/spark-shell @@ -83,21 +90,16 @@ The master URL passed to Spark can be in one of the following formats: The host parameter is the hostname of the Mesos master. The port must be whichever one the master is configured to use, which is 5050 by default. + yarn-client Connect to a YARN cluster in +client mode. The cluster location will be inferred based on the local Hadoop configuration. + + yarn-cluster Connect to a YARN cluster in +cluster mode. The cluster location will be inferred based on the local Hadoop configuration. + If no master URL is specified, the spark shell defaults to "local[*]". -For running on YARN, Spark launches an instance of the standalone deploy cluster within YARN; see [running on YARN](running-on-yarn.html) for details. - -### Deploying Code on a Cluster - -If you want to run your application on a cluster, you will need to specify the two optional parameters to `SparkContext` to let it find your code: - -* `sparkHome`: The path at which Spark is installed on your worker machines (it should be the same on all of them). -* `jars`: A list of JAR files on the local machine containing your application's code and any dependencies, which Spark will deploy to all the worker nodes. You'll need to package your application into a set of JARs using your build system. For example, if you're using SBT, the [sbt-assembly](https://github.com/sbt/sbt-assembly) plugin is a good way to make a single JAR with your code and dependencies. - -If you run `bin/spark-shell` on a cluster, you can add JARs to it by specifying the `ADD_JARS` environment variable before you launch it. This variable should contain a comma-separated list of JARs. For example, `ADD_JARS=a.jar,b.jar ./bin/spark-shell` will launch a shell with `a.jar` and `b.jar` on its classpath. In addition, any new classes you define in the shell will automatically be distributed. - # Resilient Distributed Datasets (RDDs) Spark revolves around the concept of a _resilient distributed dataset_ (RDD), which is a fault-tolerant collection of elements that can be operated on in parallel. There are currently two types of RDDs: *parallelized collections*, which take an existing Scala collection and run functions on it in parallel, and *Hadoop datasets*, which run functions on each record of a file in Hadoop distributed file system or any other storage system supported by Hadoop. Both types of RDDs can be operated on through the same methods. diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 6f616fb7c2448..8e98cc0c80a34 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -69,7 +69,7 @@ import sqlContext._ -
      +
      The entry point into all relational functionality in Spark is the [JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index f078d06aafad0..2f74965900baf 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -39,6 +39,9 @@ import org.apache.spark.{Logging, SecurityManager, SparkConf, SparkContext} import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils +/** + * An application master that runs the users driver program and allocates executors. + */ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 0179b0600c61f..00c7649e68e13 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -33,7 +33,9 @@ import org.apache.hadoop.yarn.util.{Apps, Records} import org.apache.spark.{Logging, SparkConf} - +/** + * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's alpha API. + */ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 21f14576efe8a..ea356f33eb998 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -34,6 +34,12 @@ import org.apache.spark.util.{Utils, AkkaUtils} import org.apache.spark.scheduler.cluster.CoarseGrainedSchedulerBackend import org.apache.spark.scheduler.SplitInfo +/** + * An application master that allocates executors on behalf of a driver that is running outside + * the cluster. + * + * This is used only in yarn-client mode. + */ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index d6d46a5f6ce42..95f0f9d0ff2bc 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -54,6 +54,10 @@ object AllocationType extends Enumeration { // Note that right now, we assume all node asks as uniform in terms of capabilities and priority // Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for // more info on how we are requesting for containers. + +/** + * Acquires resources for executors from a ResourceManager and launches executors in new containers. + */ private[yarn] class YarnAllocationHandler( val conf: Configuration, val resourceManager: AMRMProtocol, diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index 566de712fc280..c00b63669ca8e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -22,16 +22,13 @@ import java.net.{InetAddress, UnknownHostException, URI, URISyntaxException} import java.nio.ByteBuffer import scala.collection.JavaConversions._ -import scala.collection.mutable.HashMap -import scala.collection.mutable.Map +import scala.collection.mutable.{HashMap, ListBuffer, Map} import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs._ -import org.apache.hadoop.fs.permission.FsPermission; -import org.apache.hadoop.io.DataOutputBuffer +import org.apache.hadoop.fs.permission.FsPermission import org.apache.hadoop.mapred.Master import org.apache.hadoop.mapreduce.MRJobConfig -import org.apache.hadoop.net.NetUtils import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.util.StringUtils import org.apache.hadoop.yarn.api._ @@ -39,19 +36,18 @@ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.protocolrecords._ import org.apache.hadoop.yarn.api.records._ import org.apache.hadoop.yarn.conf.YarnConfiguration -import org.apache.hadoop.yarn.ipc.YarnRPC -import org.apache.hadoop.yarn.util.{Records, Apps} - -import org.apache.spark.{Logging, SparkConf} -import org.apache.spark.util.Utils -import org.apache.spark.deploy.SparkHadoopUtil -import org.apache.hadoop.yarn.api.ApplicationConstants.Environment - +import org.apache.hadoop.yarn.util.{Apps, Records} +import org.apache.spark.{Logging, SparkConf, SparkContext} /** * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, - * which will launch a Spark master process and negotiate resources throughout its duration. + * Client submits an application to the YARN ResourceManager. + * + * Depending on the deployment mode this will launch one of two application master classes: + * 1. In standalone mode, it will launch an [[org.apache.spark.deploy.yarn.ApplicationMaster]] + * which launches a driver program inside of the cluster. + * 2. In client mode, it will launch an [[org.apache.spark.deploy.yarn.ExecutorLauncher]] to + * request executors on behalf of a driver running outside of the cluster. */ trait ClientBase extends Logging { val args: ClientArguments @@ -70,7 +66,6 @@ trait ClientBase extends Logging { // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { Map( - (System.getenv("SPARK_JAR") == null) -> "Error: You must set SPARK_JAR environment variable!", ((args.userJar == null && args.amClass == classOf[ApplicationMaster].getName) -> "Error: You must specify a user jar when running in standalone mode!"), (args.userClass == null) -> "Error: You must specify a user class!", @@ -208,7 +203,7 @@ trait ClientBase extends Logging { val statCache: Map[URI, FileStatus] = HashMap[URI, FileStatus]() Map( - ClientBase.SPARK_JAR -> System.getenv("SPARK_JAR"), ClientBase.APP_JAR -> args.userJar, + ClientBase.SPARK_JAR -> ClientBase.getSparkJar, ClientBase.APP_JAR -> args.userJar, ClientBase.LOG4J_PROP -> System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) ).foreach { case(destName, _localPath) => val localPath: String = if (_localPath != null) _localPath.trim() else "" @@ -251,8 +246,10 @@ trait ClientBase extends Logging { logInfo("Setting up the launch environment") val env = new HashMap[String, String]() + + val extraCp = sparkConf.getOption("spark.driver.extraClassPath") val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) - ClientBase.populateClasspath(args, yarnConf, sparkConf, log4jConf, env) + ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp) env("SPARK_YARN_MODE") = "true" env("SPARK_YARN_STAGING_DIR") = stagingDir env("SPARK_USER") = UserGroupInformation.getCurrentUser().getShortUserName() @@ -268,9 +265,6 @@ trait ClientBase extends Logging { YarnSparkHadoopUtil.setEnvFromInputString(env, System.getenv("SPARK_YARN_USER_ENV"), File.pathSeparator) - // Add each SPARK_* key to the environment. - System.getenv().filterKeys(_.startsWith("SPARK")).foreach { case (k,v) => env(k) = v } - env } @@ -299,13 +293,13 @@ trait ClientBase extends Logging { val amMemory = calculateAMMemory(newApp) - var JAVA_OPTS = "" + val JAVA_OPTS = ListBuffer[String]() // Add Xmx for AM memory JAVA_OPTS += "-Xmx" + amMemory + "m" val tmpDir = new Path(Environment.PWD.$(), YarnConfiguration.DEFAULT_CONTAINER_TEMP_DIR) - JAVA_OPTS += " -Djava.io.tmpdir=" + tmpDir + JAVA_OPTS += "-Djava.io.tmpdir=" + tmpDir // TODO: Remove once cpuset version is pushed out. // The context is, default gc for server class machines ends up using all cores to do gc - @@ -319,35 +313,48 @@ trait ClientBase extends Logging { if (useConcurrentAndIncrementalGC) { // In our expts, using (default) throughput collector has severe perf ramifications in // multi-tenant machines - JAVA_OPTS += " -XX:+UseConcMarkSweepGC " - JAVA_OPTS += " -XX:+CMSIncrementalMode " - JAVA_OPTS += " -XX:+CMSIncrementalPacing " - JAVA_OPTS += " -XX:CMSIncrementalDutyCycleMin=0 " - JAVA_OPTS += " -XX:CMSIncrementalDutyCycle=10 " + JAVA_OPTS += "-XX:+UseConcMarkSweepGC" + JAVA_OPTS += "-XX:+CMSIncrementalMode" + JAVA_OPTS += "-XX:+CMSIncrementalPacing" + JAVA_OPTS += "-XX:CMSIncrementalDutyCycleMin=0" + JAVA_OPTS += "-XX:CMSIncrementalDutyCycle=10" } - if (env.isDefinedAt("SPARK_JAVA_OPTS")) { - JAVA_OPTS += " " + env("SPARK_JAVA_OPTS") + // TODO: it might be nicer to pass these as an internal environment variable rather than + // as Java options, due to complications with string parsing of nested quotes. + if (args.amClass == classOf[ExecutorLauncher].getName) { + // If we are being launched in client mode, forward the spark-conf options + // onto the executor launcher + for ((k, v) <- sparkConf.getAll) { + JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" + } + } else { + // If we are being launched in standalone mode, capture and forward any spark + // system properties (e.g. set by spark-class). + for ((k, v) <- sys.props.filterKeys(_.startsWith("spark"))) { + JAVA_OPTS += "-D" + k + "=" + "\\\"" + v + "\\\"" + } + sys.props.get("spark.driver.extraJavaOptions").foreach(opts => JAVA_OPTS += opts) + sys.props.get("spark.driver.libraryPath").foreach(p => JAVA_OPTS += s"-Djava.library.path=$p") } JAVA_OPTS += ClientBase.getLog4jConfiguration(localResources) // Command for the ApplicationMaster - val commands = List[String]( - Environment.JAVA_HOME.$() + "/bin/java" + - " -server " + - JAVA_OPTS + - " " + args.amClass + - " --class " + args.userClass + - " --jar " + args.userJar + - userArgsToString(args) + - " --executor-memory " + args.executorMemory + - " --executor-cores " + args.executorCores + - " --num-executors " + args.numExecutors + - " 1> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout" + - " 2> " + ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") - - logInfo("Command for starting the Spark ApplicationMaster: " + commands(0)) - amContainer.setCommands(commands) + val commands = Seq(Environment.JAVA_HOME.$() + "/bin/java", "-server") ++ + JAVA_OPTS ++ + Seq(args.amClass, "--class", args.userClass, "--jar ", args.userJar, + userArgsToString(args), + "--executor-memory", args.executorMemory.toString, + "--executor-cores", args.executorCores.toString, + "--num-executors ", args.numExecutors.toString, + "1>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stdout", + "2>", ApplicationConstants.LOG_DIR_EXPANSION_VAR + "/stderr") + + logInfo("Command for starting the Spark ApplicationMaster: " + commands) + + // TODO: it would be nicer to just make sure there are no null commands here + val printableCommands = commands.map(s => if (s == null) "null" else s).toList + amContainer.setCommands(printableCommands) setupSecurityToken(amContainer) amContainer @@ -361,6 +368,8 @@ object ClientBase { val LOG4J_CONF_ENV_KEY: String = "SPARK_LOG4J_CONF" val LOCAL_SCHEME = "local" + def getSparkJar = sys.env.get("SPARK_JAR").getOrElse(SparkContext.jarOfClass(this.getClass).head) + // Based on code from org.apache.hadoop.mapreduce.v2.util.MRApps def populateHadoopClasspath(conf: Configuration, env: HashMap[String, String]) { val classpathEntries = Option(conf.getStrings( @@ -433,10 +442,9 @@ object ClientBase { " -Dlog4j.configuration=" + log4jConf } - def populateClasspath(args: ClientArguments, conf: Configuration, sparkConf: SparkConf, - log4jConf: String, env: HashMap[String, String]) { - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, Environment.PWD.$(), - File.pathSeparator) + def populateClasspath(conf: Configuration, sparkConf: SparkConf, log4jConf: String, + env: HashMap[String, String], extraClassPath: Option[String] = None) { + if (log4jConf != null) { // If a custom log4j config file is provided as a local: URI, add its parent directory to the // classpath. Note that this only works if the custom config's file name is @@ -448,19 +456,26 @@ object ClientBase { File.pathSeparator) } } + + /** Add entry to the classpath. */ + def addClasspathEntry(path: String) = Apps.addToEnvironment(env, Environment.CLASSPATH.name, path) + /** Add entry to the classpath. Interpreted as a path relative to the working directory. */ + def addPwdClasspathEntry(entry: String) = addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry) + + extraClassPath.foreach(addClasspathEntry) + + addClasspathEntry(Environment.PWD.$()) // Normally the users app.jar is last in case conflicts with spark jars - val userClasspathFirst = sparkConf.get("spark.yarn.user.classpath.first", "false") - .toBoolean - if (userClasspathFirst) { - addUserClasspath(args, env) - } - addClasspathEntry(System.getenv("SPARK_JAR"), SPARK_JAR, env); - ClientBase.populateHadoopClasspath(conf, env) - if (!userClasspathFirst) { - addUserClasspath(args, env) + if (sparkConf.get("spark.yarn.user.classpath.first", "false").toBoolean) { + addPwdClasspathEntry(APP_JAR) + addPwdClasspathEntry(SPARK_JAR) + ClientBase.populateHadoopClasspath(conf, env) + } else { + addPwdClasspathEntry(SPARK_JAR) + ClientBase.populateHadoopClasspath(conf, env) + addPwdClasspathEntry(APP_JAR) } - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, - Environment.PWD.$() + Path.SEPARATOR + "*", File.pathSeparator) + addPwdClasspathEntry("*") } /** diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala index 40b38661f794d..7d07f6f68046a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ExecutorRunnableUtil.scala @@ -19,26 +19,18 @@ package org.apache.spark.deploy.yarn import java.io.File import java.net.URI -import java.nio.ByteBuffer -import java.security.PrivilegedExceptionAction import scala.collection.JavaConversions._ import scala.collection.mutable.HashMap -import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.Path -import org.apache.hadoop.io.DataOutputBuffer -import org.apache.hadoop.net.NetUtils -import org.apache.hadoop.security.UserGroupInformation import org.apache.hadoop.yarn.api._ import org.apache.hadoop.yarn.api.ApplicationConstants.Environment import org.apache.hadoop.yarn.api.records._ -import org.apache.hadoop.yarn.api.protocolrecords._ -import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} - -import org.apache.spark.{SparkConf, Logging} import org.apache.hadoop.yarn.conf.YarnConfiguration +import org.apache.hadoop.yarn.util.{Apps, ConverterUtils, Records} +import org.apache.spark.{Logging, SparkConf} trait ExecutorRunnableUtil extends Logging { @@ -58,8 +50,10 @@ trait ExecutorRunnableUtil extends Logging { // Set the JVM memory val executorMemoryString = executorMemory + "m" JAVA_OPTS += "-Xms" + executorMemoryString + " -Xmx" + executorMemoryString + " " - if (env.isDefinedAt("SPARK_JAVA_OPTS")) { - JAVA_OPTS += env("SPARK_JAVA_OPTS") + " " + + // Set extra Java options for the executor, if defined + sys.props.get("spark.executor.extraJavaOptions").foreach { opts => + JAVA_OPTS += opts } JAVA_OPTS += " -Djava.io.tmpdir=" + @@ -162,8 +156,9 @@ trait ExecutorRunnableUtil extends Logging { def prepareEnvironment: HashMap[String, String] = { val env = new HashMap[String, String]() + val extraCp = sparkConf.getOption("spark.executor.extraClassPath") val log4jConf = System.getenv(ClientBase.LOG4J_CONF_ENV_KEY) - ClientBase.populateClasspath(null, yarnConf, sparkConf, log4jConf, env) + ClientBase.populateClasspath(yarnConf, sparkConf, log4jConf, env, extraCp) if (log4jConf != null) { env(ClientBase.LOG4J_CONF_ENV_KEY) = log4jConf } diff --git a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala index 161918859e7c4..ce2dde0631ed9 100644 --- a/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala +++ b/yarn/common/src/main/scala/org/apache/spark/scheduler/cluster/YarnClientSchedulerBackend.scala @@ -19,7 +19,7 @@ package org.apache.spark.scheduler.cluster import org.apache.hadoop.yarn.api.records.{ApplicationId, YarnApplicationState} import org.apache.spark.{SparkException, Logging, SparkContext} -import org.apache.spark.deploy.yarn.{Client, ClientArguments} +import org.apache.spark.deploy.yarn.{Client, ClientArguments, ExecutorLauncher} import org.apache.spark.scheduler.TaskSchedulerImpl import scala.collection.mutable.ArrayBuffer @@ -54,7 +54,7 @@ private[spark] class YarnClientSchedulerBackend( "--class", "notused", "--jar", null, "--args", hostport, - "--am-class", "org.apache.spark.deploy.yarn.ExecutorLauncher" + "--am-class", classOf[ExecutorLauncher].getName ) // process any optional arguments, given either as environment variables diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index b225be6a79c0e..90e807160d4b6 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -42,6 +42,9 @@ import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.util.Utils +/** + * An application master that runs the user's driver program and allocates executors. + */ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala index 77eb1276a0c4e..2e2fb5d4fa787 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/Client.scala @@ -34,9 +34,7 @@ import org.apache.spark.{Logging, SparkConf} /** - * The entry point (starting in Client#main() and Client#run()) for launching Spark on YARN. The - * Client submits an application to the global ResourceManager to launch Spark's ApplicationMaster, - * which will launch a Spark master process and negotiate resources throughout its duration. + * Version of [[org.apache.spark.deploy.yarn.ClientBase]] tailored to YARN's stable API. */ class Client(clientArgs: ClientArguments, hadoopConf: Configuration, spConf: SparkConf) extends YarnClientImpl with ClientBase with Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index 67ed591c78bf9..a14bb377aa133 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -35,6 +35,12 @@ import org.apache.spark.scheduler.SplitInfo import org.apache.hadoop.yarn.client.api.AMRMClient import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest +/** + * An application master that allocates executors on behalf of a driver that is running outside + * the cluster. + * + * This is used only in yarn-client mode. + */ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sparkConf: SparkConf) extends Logging { diff --git a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 4fafae1aff26f..a979fe4d62630 100644 --- a/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/stable/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -56,6 +56,10 @@ object AllocationType extends Enumeration { // Note that right now, we assume all node asks as uniform in terms of capabilities and priority // Refer to http://developer.yahoo.com/blogs/hadoop/posts/2011/03/mapreduce-nextgen-scheduler/ for // more info on how we are requesting for containers. + +/** + * Acquires resources for executors from a ResourceManager and launches executors in new containers. + */ private[yarn] class YarnAllocationHandler( val conf: Configuration, val amClient: AMRMClient[ContainerRequest], From af46f1fd02b913beeac530a766f94e0c6c85674f Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 21 Apr 2014 12:37:43 -0700 Subject: [PATCH 323/397] [Hot Fix] Ignore org.apache.spark.ui.UISuite tests #446 faced a connection refused exception from these tests, causing them to timeout and fail after a long time. For now, let's disable these tests. (We recently disabled the corresponding test in streaming in 7863ecca35be9af1eca0dfe5fd8806c5dd710fd6. These tests are very similar). Author: Andrew Or Closes #466 from andrewor14/ignore-ui-tests and squashes the following commits: 6f5a362 [Andrew Or] Ignore org.apache.spark.ui.UISuite tests --- core/src/test/scala/org/apache/spark/ui/UISuite.scala | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/ui/UISuite.scala b/core/src/test/scala/org/apache/spark/ui/UISuite.scala index ed02b0ba00d43..fff8020ade6d1 100644 --- a/core/src/test/scala/org/apache/spark/ui/UISuite.scala +++ b/core/src/test/scala/org/apache/spark/ui/UISuite.scala @@ -36,7 +36,7 @@ import scala.xml.Node class UISuite extends FunSuite { - test("basic ui visibility") { + ignore("basic ui visibility") { withSpark(new SparkContext("local", "test")) { sc => // test if the ui is visible, and all the expected tabs are visible eventually(timeout(10 seconds), interval(50 milliseconds)) { @@ -50,7 +50,7 @@ class UISuite extends FunSuite { } } - test("visibility at localhost:4040") { + ignore("visibility at localhost:4040") { withSpark(new SparkContext("local", "test")) { sc => // test if visible from http://localhost:4040 eventually(timeout(10 seconds), interval(50 milliseconds)) { @@ -60,7 +60,7 @@ class UISuite extends FunSuite { } } - test("attaching a new tab") { + ignore("attaching a new tab") { withSpark(new SparkContext("local", "test")) { sc => val sparkUI = sc.ui From b7df31eb34523c1aaae1301b36b38a928f40e1ad Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Mon, 21 Apr 2014 12:48:02 -0700 Subject: [PATCH 324/397] SPARK-1539: RDDPage.scala contains RddPage class SPARK-1386 changed RDDPage to RddPage but didn't change the filename. I tried sbt/sbt publish-local. Inside the spark-core jar, the unit name is RDDPage.class and hence I got the following error: ~~~ [error] (run-main) java.lang.NoClassDefFoundError: org/apache/spark/ui/storage/RddPage java.lang.NoClassDefFoundError: org/apache/spark/ui/storage/RddPage at org.apache.spark.ui.SparkUI.initialize(SparkUI.scala:59) at org.apache.spark.ui.SparkUI.(SparkUI.scala:52) at org.apache.spark.ui.SparkUI.(SparkUI.scala:42) at org.apache.spark.SparkContext.(SparkContext.scala:215) at MovieLensALS$.main(MovieLensALS.scala:38) at MovieLensALS.main(MovieLensALS.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) Caused by: java.lang.ClassNotFoundException: org.apache.spark.ui.storage.RddPage at java.net.URLClassLoader$1.run(URLClassLoader.java:366) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) at org.apache.spark.ui.SparkUI.initialize(SparkUI.scala:59) at org.apache.spark.ui.SparkUI.(SparkUI.scala:52) at org.apache.spark.ui.SparkUI.(SparkUI.scala:42) at org.apache.spark.SparkContext.(SparkContext.scala:215) at MovieLensALS$.main(MovieLensALS.scala:38) at MovieLensALS.main(MovieLensALS.scala) at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:57) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:606) ~~~ This can be fixed after renaming RddPage to RDDPage, or renaming RDDPage.scala to RddPage.scala. I chose the former since the name `RDD` is common in Spark code. Author: Xiangrui Meng Closes #454 from mengxr/rddpage-fix and squashes the following commits: f75e544 [Xiangrui Meng] rename RddPage to RDDPage --- core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala | 2 +- .../src/main/scala/org/apache/spark/ui/storage/StorageTab.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala index d07f1c9b20fcf..2155633b8096f 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/RDDPage.scala @@ -26,7 +26,7 @@ import org.apache.spark.ui.{WebUIPage, UIUtils} import org.apache.spark.util.Utils /** Page showing storage details for a given RDD */ -private[ui] class RddPage(parent: StorageTab) extends WebUIPage("rdd") { +private[ui] class RDDPage(parent: StorageTab) extends WebUIPage("rdd") { private val appName = parent.appName private val basePath = parent.basePath private val listener = parent.listener diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index 56429f6c07fcd..c04ef0ae37748 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -30,7 +30,7 @@ private[ui] class StorageTab(parent: SparkUI) extends WebUITab(parent, "storage" val listener = new StorageListener(parent.storageStatusListener) attachPage(new StoragePage(this)) - attachPage(new RddPage(this)) + attachPage(new RDDPage(this)) parent.registerListener(listener) } From 43e4a29dacf61bea870ed5010d6df77bc8009aa3 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Mon, 21 Apr 2014 14:10:23 -0700 Subject: [PATCH 325/397] SPARK-1399: show stage failure reason in UI https://issues.apache.org/jira/browse/SPARK-1399 refactor StageTable a bit to support additional column for failed stage Author: CodingCat Author: Nan Zhu Closes #421 from CodingCat/SPARK-1399 and squashes the following commits: 2caba36 [CodingCat] remove dummy tag 77cf305 [CodingCat] create dummy element to wrap columns 3989ce2 [CodingCat] address Aaron's comments 18fc09f [Nan Zhu] fix compile error 00ea30a [Nan Zhu] address Kay's comments 16ac83d [CodingCat] set a default value of failureReason 35df3df [CodingCat] address andrew's comments 06d21a4 [CodingCat] address andrew's comments 25a6db6 [CodingCat] style fix dc8856d [CodingCat] show stage failure reason in UI --- .../spark/ui/jobs/JobProgressPage.scala | 8 +- .../org/apache/spark/ui/jobs/PoolPage.scala | 3 +- .../org/apache/spark/ui/jobs/StageTable.scala | 89 +++++++++++-------- 3 files changed, 61 insertions(+), 39 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala index 34ff2ac34a7ca..0da62892118d4 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/JobProgressPage.scala @@ -41,10 +41,12 @@ private[ui] class JobProgressPage(parent: JobProgressTab) extends WebUIPage("") val now = System.currentTimeMillis val activeStagesTable = - new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent, parent.killEnabled) + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, + parent, parent.killEnabled) val completedStagesTable = - new StageTable(completedStages.sortBy(_.submissionTime).reverse, parent) - val failedStagesTable = new StageTable(failedStages.sortBy(_.submissionTime).reverse, parent) + new StageTableBase(completedStages.sortBy(_.submissionTime).reverse, parent) + val failedStagesTable = + new FailedStageTable(failedStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs val pools = if (live) sc.getAllPools else Seq[Schedulable]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala index fd83d37583967..0a2bf31833d2b 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/PoolPage.scala @@ -40,7 +40,8 @@ private[ui] class PoolPage(parent: JobProgressTab) extends WebUIPage("pool") { case Some(s) => s.values.toSeq case None => Seq[StageInfo]() } - val activeStagesTable = new StageTable(activeStages.sortBy(_.submissionTime).reverse, parent) + val activeStagesTable = + new StageTableBase(activeStages.sortBy(_.submissionTime).reverse, parent) // For now, pool information is only accessible in live UIs val pools = if (live) Seq(sc.getPoolForName(poolName).get) else Seq[Schedulable]() diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 8c5b1f55fd2dc..2eb8c7a3a3073 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -27,34 +27,37 @@ import org.apache.spark.ui.UIUtils import org.apache.spark.util.Utils /** Page showing list of all ongoing and recently finished stages */ -private[ui] class StageTable( +private[ui] class StageTableBase( stages: Seq[StageInfo], parent: JobProgressTab, killEnabled: Boolean = false) { private val basePath = parent.basePath private val listener = parent.listener - private lazy val isFairScheduler = parent.isFairScheduler + protected def isFairScheduler = parent.isFairScheduler + + protected def columns: Seq[Node] = { + // create dummy element to wrap the columns + Stage Id ++ + {if (isFairScheduler) {Pool Name} else Seq.empty} ++ + Description + Submitted + Duration + Tasks: Succeeded/Total + Shuffle Read + Shuffle Write + } def toNodeSeq: Seq[Node] = { listener.synchronized { - stageTable(stageRow, stages) + stageTable(renderStageRow, stages) } } /** Special table that merges two header cells. */ - private def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { + protected def stageTable[T](makeRow: T => Seq[Node], rows: Seq[T]): Seq[Node] = { - - - {if (isFairScheduler) {} else {}} - - - - - - - + {columns} {rows.map(r => makeRow(r))} @@ -94,8 +97,7 @@ private[ui] class StageTable( .getOrElse(
      {killLink}{nameLink}
      ) } - /** Render an HTML row that represents a stage */ - private def stageRow(s: StageInfo): Seq[Node] = { + protected def stageRow(s: StageInfo): Seq[Node] = { val poolName = listener.stageIdToPool.get(s.stageId) val submissionTime = s.submissionTime match { case Some(t) => UIUtils.formatDate(new Date(t)) @@ -124,25 +126,42 @@ private[ui] class StageTable( case 0 => "" case b => Utils.bytesToString(b) } - - - - {if (isFairScheduler) { - - }} - - - - ++ + {if (isFairScheduler) { + - - - + } else { + Seq.empty + }} ++ + + + + + + + } + + /** Render an HTML row that represents a stage */ + private def renderStageRow(s: StageInfo): Seq[Node] = {stageRow(s)} +} + +private[ui] class FailedStageTable( + stages: Seq[StageInfo], + parent: JobProgressTab, + killEnabled: Boolean = false) + extends StageTableBase(stages, parent, killEnabled) { + + override protected def columns: Seq[Node] = super.columns ++ + + override protected def stageRow(s: StageInfo): Seq[Node] = { + val basicColumns = super.stageRow(s) + val failureReason = + basicColumns ++ failureReason } } From 5a5b3346c79abb659260284fed0ace51942f3193 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Mon, 21 Apr 2014 14:21:17 -0700 Subject: [PATCH 326/397] Dev script: include RC name in git tag --- dev/create-release/create-release.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index bf1c5d7953bd2..fb9d9f9e072a6 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -35,7 +35,7 @@ USER_NAME=pwendell set -e -GIT_TAG=v$RELEASE_VERSION +GIT_TAG=v$RELEASE_VERSION-$RC_NAME # Artifact publishing From 04c37b6f749dc2418cc28c89964cdc687dfcbd51 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Mon, 21 Apr 2014 19:04:49 -0700 Subject: [PATCH 327/397] [SPARK-1332] Improve Spark Streaming's Network Receiver and InputDStream API [WIP] The current Network Receiver API makes it slightly complicated to right a new receiver as one needs to create an instance of BlockGenerator as shown in SocketReceiver https://github.com/apache/spark/blob/master/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala#L51 Exposing the BlockGenerator interface has made it harder to improve the receiving process. The API of NetworkReceiver (which was not a very stable API anyways) needs to be change if we are to ensure future stability. Additionally, the functions like streamingContext.socketStream that create input streams, return DStream objects. That makes it hard to expose functionality (say, rate limits) unique to input dstreams. They should return InputDStream or NetworkInputDStream. This is still not yet implemented. This PR is blocked on the graceful shutdown PR #247 Author: Tathagata Das Closes #300 from tdas/network-receiver-api and squashes the following commits: ea27b38 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api 3a4777c [Tathagata Das] Renamed NetworkInputDStream to ReceiverInputDStream, and ActorReceiver related stuff. 838dd39 [Tathagata Das] Added more events to the StreamingListener to report errors and stopped receivers. a75c7a6 [Tathagata Das] Address some PR comments and fixed other issues. 91bfa72 [Tathagata Das] Fixed bugs. 8533094 [Tathagata Das] Scala style fixes. 028bde6 [Tathagata Das] Further refactored receiver to allow restarting of a receiver. 43f5290 [Tathagata Das] Made functions that create input streams return InputDStream and NetworkInputDStream, for both Scala and Java. 2c94579 [Tathagata Das] Fixed graceful shutdown by removing interrupts on receiving thread. 9e37a0b [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into network-receiver-api 3223e95 [Tathagata Das] Refactored the code that runs the NetworkReceiver into further classes and traits to make them more testable. a36cc48 [Tathagata Das] Refactored the NetworkReceiver API for future stability. --- .../scala/org/apache/spark/ui/UIUtils.scala | 6 +- .../streaming/examples/ActorWordCount.scala | 6 +- .../streaming/flume/FlumeInputDStream.scala | 28 +- .../spark/streaming/flume/FlumeUtils.scala | 10 +- .../streaming/flume/JavaFlumeStreamSuite.java | 6 +- .../streaming/flume/FlumeStreamSuite.scala | 6 +- .../streaming/kafka/KafkaInputDStream.scala | 19 +- .../spark/streaming/kafka/KafkaUtils.scala | 14 +- .../streaming/kafka/JavaKafkaStreamSuite.java | 10 +- .../streaming/kafka/KafkaStreamSuite.scala | 10 +- .../streaming/mqtt/MQTTInputDStream.scala | 41 +- .../spark/streaming/mqtt/MQTTUtils.scala | 12 +- .../streaming/mqtt/JavaMQTTStreamSuite.java | 6 +- .../streaming/mqtt/MQTTStreamSuite.scala | 6 +- .../twitter/TwitterInputDStream.scala | 23 +- .../streaming/twitter/TwitterUtils.scala | 20 +- .../twitter/TwitterStreamSuite.scala | 20 +- .../streaming/zeromq/ZeroMQReceiver.scala | 7 +- .../spark/streaming/zeromq/ZeroMQUtils.scala | 16 +- .../zeromq/JavaZeroMQStreamSuite.java | 8 +- .../streaming/zeromq/ZeroMQStreamSuite.scala | 9 +- project/MimaBuild.scala | 29 +- .../apache/spark/streaming/DStreamGraph.scala | 8 +- .../spark/streaming/StreamingContext.scala | 44 ++- .../streaming/api/java/JavaDStream.scala | 4 + .../streaming/api/java/JavaInputDStream.scala | 40 ++ .../api/java/JavaPairInputDStream.scala | 41 ++ .../java/JavaPairReceiverInputDStream.scala | 42 ++ .../api/java/JavaReceiverInputDStream.scala | 41 ++ .../api/java/JavaStreamingContext.scala | 56 ++- .../streaming/dstream/InputDStream.scala | 2 +- .../dstream/NetworkInputDStream.scala | 362 ------------------ .../dstream/PluggableInputDStream.scala | 5 +- .../streaming/dstream/RawInputDStream.scala | 16 +- .../dstream/ReceiverInputDStream.scala | 94 +++++ .../dstream/SocketInputDStream.scala | 62 ++- .../ActorReceiver.scala | 95 ++--- .../streaming/receiver/BlockGenerator.scala | 142 +++++++ .../spark/streaming/receiver/Receiver.scala | 236 ++++++++++++ .../streaming/receiver/ReceiverMessage.scala | 23 ++ .../receiver/ReceiverSupervisor.scala | 180 +++++++++ .../receiver/ReceiverSupervisorImpl.scala | 180 +++++++++ .../streaming/scheduler/JobGenerator.scala | 16 +- .../streaming/scheduler/JobScheduler.scala | 8 +- ...putTracker.scala => ReceiverTracker.scala} | 101 +++-- .../scheduler/StreamingListener.scala | 18 +- .../scheduler/StreamingListenerBus.scala | 4 + .../ui/StreamingJobProgressListener.scala | 10 +- .../spark/streaming/ui/StreamingPage.scala | 18 +- .../spark/streaming/util/RecurringTimer.scala | 4 +- .../apache/spark/streaming/JavaAPISuite.java | 9 +- .../spark/streaming/InputStreamsSuite.scala | 19 +- .../streaming/NetworkReceiverSuite.scala | 249 ++++++++++++ .../streaming/StreamingContextSuite.scala | 42 +- .../streaming/StreamingListenerSuite.scala | 84 +++- 55 files changed, 1836 insertions(+), 731 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala delete mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala rename streaming/src/main/scala/org/apache/spark/streaming/{receivers => receiver}/ActorReceiver.scala (66%) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala rename streaming/src/main/scala/org/apache/spark/streaming/scheduler/{NetworkInputTracker.scala => ReceiverTracker.scala} (68%) create mode 100644 streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index 99770f28549c9..cf987a1ab02c3 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -121,7 +121,11 @@ private[spark] object UIUtils extends Logging { (records, "") } } - "%.1f%s".formatLocal(Locale.US, value, unit) + if (unit.isEmpty) { + "%d".formatLocal(Locale.US, value) + } else { + "%.1f%s".formatLocal(Locale.US, value, unit) + } } // Yarn has to go through a proxy so the base uri is provided and has to be on all links diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index a22e64ca3ce45..eb44768b9c101 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -26,8 +26,8 @@ import akka.actor.{Actor, ActorRef, Props, actorRef2Scala} import org.apache.spark.{SparkConf, SecurityManager} import org.apache.spark.streaming.{Seconds, StreamingContext} import org.apache.spark.streaming.StreamingContext.toPairDStreamFunctions -import org.apache.spark.streaming.receivers.Receiver import org.apache.spark.util.AkkaUtils +import org.apache.spark.streaming.receiver.ActorHelper case class SubscribeReceiver(receiverActor: ActorRef) case class UnsubscribeReceiver(receiverActor: ActorRef) @@ -81,14 +81,14 @@ class FeederActor extends Actor { * @see [[org.apache.spark.streaming.examples.FeederActor]] */ class SampleActorReceiver[T: ClassTag](urlOfPublisher: String) -extends Actor with Receiver { +extends Actor with ActorHelper { lazy private val remotePublisher = context.actorSelection(urlOfPublisher) override def preStart = remotePublisher ! SubscribeReceiver(context.self) def receive = { - case msg => pushBlock(msg.asInstanceOf[T]) + case msg => store(msg.asInstanceOf[T]) } override def postStop() = remotePublisher ! UnsubscribeReceiver(context.self) diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala index 34012b846e21e..df7605fe579f8 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeInputDStream.scala @@ -34,6 +34,8 @@ import org.apache.spark.util.Utils import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ +import org.apache.spark.Logging +import org.apache.spark.streaming.receiver.Receiver private[streaming] class FlumeInputDStream[T: ClassTag]( @@ -41,9 +43,9 @@ class FlumeInputDStream[T: ClassTag]( host: String, port: Int, storageLevel: StorageLevel -) extends NetworkInputDStream[SparkFlumeEvent](ssc_) { +) extends ReceiverInputDStream[SparkFlumeEvent](ssc_) { - override def getReceiver(): NetworkReceiver[SparkFlumeEvent] = { + override def getReceiver(): Receiver[SparkFlumeEvent] = { new FlumeReceiver(host, port, storageLevel) } } @@ -115,13 +117,13 @@ private[streaming] object SparkFlumeEvent { private[streaming] class FlumeEventServer(receiver : FlumeReceiver) extends AvroSourceProtocol { override def append(event : AvroFlumeEvent) : Status = { - receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event) + receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event)) Status.OK } override def appendBatch(events : java.util.List[AvroFlumeEvent]) : Status = { events.foreach (event => - receiver.blockGenerator += SparkFlumeEvent.fromAvroFlumeEvent(event)) + receiver.store(SparkFlumeEvent.fromAvroFlumeEvent(event))) Status.OK } } @@ -133,23 +135,21 @@ class FlumeReceiver( host: String, port: Int, storageLevel: StorageLevel - ) extends NetworkReceiver[SparkFlumeEvent] { + ) extends Receiver[SparkFlumeEvent](storageLevel) with Logging { - lazy val blockGenerator = new BlockGenerator(storageLevel) + lazy val responder = new SpecificResponder( + classOf[AvroSourceProtocol], new FlumeEventServer(this)) + lazy val server = new NettyServer(responder, new InetSocketAddress(host, port)) - protected override def onStart() { - val responder = new SpecificResponder( - classOf[AvroSourceProtocol], new FlumeEventServer(this)) - val server = new NettyServer(responder, new InetSocketAddress(host, port)) - blockGenerator.start() + def onStart() { server.start() logInfo("Flume receiver started") } - protected override def onStop() { - blockGenerator.stop() + def onStop() { + server.close() logInfo("Flume receiver stopped") } - override def getLocationPreference = Some(host) + override def preferredLocation = Some(host) } diff --git a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala index 654ba451e72fb..499f3560ef768 100644 --- a/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala +++ b/external/flume/src/main/scala/org/apache/spark/streaming/flume/FlumeUtils.scala @@ -19,8 +19,8 @@ package org.apache.spark.streaming.flume import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaInputDStream, JavaStreamingContext, JavaDStream} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object FlumeUtils { /** @@ -35,7 +35,7 @@ object FlumeUtils { hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[SparkFlumeEvent] = { + ): ReceiverInputDStream[SparkFlumeEvent] = { val inputStream = new FlumeInputDStream[SparkFlumeEvent](ssc, hostname, port, storageLevel) inputStream } @@ -50,7 +50,7 @@ object FlumeUtils { jssc: JavaStreamingContext, hostname: String, port: Int - ): JavaDStream[SparkFlumeEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createStream(jssc.ssc, hostname, port) } @@ -65,7 +65,7 @@ object FlumeUtils { hostname: String, port: Int, storageLevel: StorageLevel - ): JavaDStream[SparkFlumeEvent] = { + ): JavaReceiverInputDStream[SparkFlumeEvent] = { createStream(jssc.ssc, hostname, port, storageLevel) } } diff --git a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java index 733389b98d22d..e0ad4f1015205 100644 --- a/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java +++ b/external/flume/src/test/java/org/apache/spark/streaming/flume/JavaFlumeStreamSuite.java @@ -19,16 +19,16 @@ import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.junit.Test; public class JavaFlumeStreamSuite extends LocalJavaStreamingContext { @Test public void testFlumeStream() { // tests the API, does not actually test data receiving - JavaDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); - JavaDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, + JavaReceiverInputDStream test1 = FlumeUtils.createStream(ssc, "localhost", 12345); + JavaReceiverInputDStream test2 = FlumeUtils.createStream(ssc, "localhost", 12345, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 8bc43972ab6a0..78603200d2d8a 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -31,6 +31,7 @@ import org.apache.flume.source.avro.{AvroFlumeEvent, AvroSourceProtocol} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{TestOutputStream, StreamingContext, TestSuiteBase} import org.apache.spark.streaming.util.ManualClock +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream class FlumeStreamSuite extends TestSuiteBase { @@ -39,10 +40,11 @@ class FlumeStreamSuite extends TestSuiteBase { test("flume input stream") { // Set up the streaming context and input streams val ssc = new StreamingContext(conf, batchDuration) - val flumeStream = FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) + val flumeStream: JavaReceiverInputDStream[SparkFlumeEvent] = + FlumeUtils.createStream(ssc, "localhost", testPort, StorageLevel.MEMORY_AND_DISK) val outputBuffer = new ArrayBuffer[Seq[SparkFlumeEvent]] with SynchronizedBuffer[Seq[SparkFlumeEvent]] - val outputStream = new TestOutputStream(flumeStream, outputBuffer) + val outputStream = new TestOutputStream(flumeStream.receiverInputDStream, outputBuffer) outputStream.register() ssc.start() diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala index c2d9dcbfaac7a..21443ebbbfb0e 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaInputDStream.scala @@ -33,6 +33,7 @@ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver /** * Input stream that pulls messages from a Kafka Broker. @@ -53,11 +54,11 @@ class KafkaInputDStream[ kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ) extends NetworkInputDStream[(K, V)](ssc_) with Logging { + ) extends ReceiverInputDStream[(K, V)](ssc_) with Logging { - def getReceiver(): NetworkReceiver[(K, V)] = { + def getReceiver(): Receiver[(K, V)] = { new KafkaReceiver[K, V, U, T](kafkaParams, topics, storageLevel) - .asInstanceOf[NetworkReceiver[(K, V)]] + .asInstanceOf[Receiver[(K, V)]] } } @@ -70,21 +71,15 @@ class KafkaReceiver[ kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ) extends NetworkReceiver[Any] { + ) extends Receiver[Any](storageLevel) with Logging { - // Handles pushing data into the BlockManager - lazy protected val blockGenerator = new BlockGenerator(storageLevel) // Connection to Kafka var consumerConnector : ConsumerConnector = null - def onStop() { - blockGenerator.stop() - } + def onStop() { } def onStart() { - blockGenerator.start() - // In case we are using multiple Threads to handle Kafka Messages val executorPool = Executors.newFixedThreadPool(topics.values.reduce(_ + _)) @@ -130,7 +125,7 @@ class KafkaReceiver[ def run() { logInfo("Starting MessageHandler.") for (msgAndMetadata <- stream) { - blockGenerator += (msgAndMetadata.key, msgAndMetadata.message) + store((msgAndMetadata.key, msgAndMetadata.message)) } } } diff --git a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala index 5472d0cd04a94..86bb91f362d29 100644 --- a/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala +++ b/external/kafka/src/main/scala/org/apache/spark/streaming/kafka/KafkaUtils.scala @@ -27,8 +27,8 @@ import kafka.serializer.{Decoder, StringDecoder} import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaPairDStream} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaPairReceiverInputDStream, JavaStreamingContext, JavaPairDStream} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object KafkaUtils { @@ -48,7 +48,7 @@ object KafkaUtils { groupId: String, topics: Map[String, Int], storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[(String, String)] = { + ): ReceiverInputDStream[(String, String)] = { val kafkaParams = Map[String, String]( "zookeeper.connect" -> zkQuorum, "group.id" -> groupId, "zookeeper.connection.timeout.ms" -> "10000") @@ -70,7 +70,7 @@ object KafkaUtils { kafkaParams: Map[String, String], topics: Map[String, Int], storageLevel: StorageLevel - ): DStream[(K, V)] = { + ): ReceiverInputDStream[(K, V)] = { new KafkaInputDStream[K, V, U, T](ssc, kafkaParams, topics, storageLevel) } @@ -88,7 +88,7 @@ object KafkaUtils { zkQuorum: String, groupId: String, topics: JMap[String, JInt] - ): JavaPairDStream[String, String] = { + ): JavaPairReceiverInputDStream[String, String] = { implicit val cmt: ClassTag[String] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*)) @@ -110,7 +110,7 @@ object KafkaUtils { groupId: String, topics: JMap[String, JInt], storageLevel: StorageLevel - ): JavaPairDStream[String, String] = { + ): JavaPairReceiverInputDStream[String, String] = { implicit val cmt: ClassTag[String] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, zkQuorum, groupId, Map(topics.mapValues(_.intValue()).toSeq: _*), @@ -139,7 +139,7 @@ object KafkaUtils { kafkaParams: JMap[String, String], topics: JMap[String, JInt], storageLevel: StorageLevel - ): JavaPairDStream[K, V] = { + ): JavaPairReceiverInputDStream[K, V] = { implicit val keyCmt: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val valueCmt: ClassTag[V] = diff --git a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java index 7b4999447ee69..9f8046bf00f8f 100644 --- a/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java +++ b/external/kafka/src/test/java/org/apache/spark/streaming/kafka/JavaKafkaStreamSuite.java @@ -18,12 +18,13 @@ package org.apache.spark.streaming.kafka; import java.util.HashMap; + +import org.apache.spark.streaming.api.java.JavaPairReceiverInputDStream; import org.junit.Test; import com.google.common.collect.Maps; import kafka.serializer.StringDecoder; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaPairDStream; public class JavaKafkaStreamSuite extends LocalJavaStreamingContext { @Test @@ -31,14 +32,15 @@ public void testKafkaStream() { HashMap topics = Maps.newHashMap(); // tests the API, does not actually test data receiving - JavaPairDStream test1 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); - JavaPairDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, + JavaPairReceiverInputDStream test1 = + KafkaUtils.createStream(ssc, "localhost:12345", "group", topics); + JavaPairReceiverInputDStream test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2()); HashMap kafkaParams = Maps.newHashMap(); kafkaParams.put("zookeeper.connect", "localhost:12345"); kafkaParams.put("group.id","consumer-group"); - JavaPairDStream test3 = KafkaUtils.createStream(ssc, + JavaPairReceiverInputDStream test3 = KafkaUtils.createStream(ssc, String.class, String.class, StringDecoder.class, StringDecoder.class, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2()); } diff --git a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala index d9809f6409d44..e6f2c4a5cf5d1 100644 --- a/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala +++ b/external/kafka/src/test/scala/org/apache/spark/streaming/kafka/KafkaStreamSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming.kafka import kafka.serializer.StringDecoder import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.ReceiverInputDStream class KafkaStreamSuite extends TestSuiteBase { @@ -28,10 +29,13 @@ class KafkaStreamSuite extends TestSuiteBase { val topics = Map("my-topic" -> 1) // tests the API, does not actually test data receiving - val test1 = KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) - val test2 = KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1: ReceiverInputDStream[(String, String)] = + KafkaUtils.createStream(ssc, "localhost:1234", "group", topics) + val test2: ReceiverInputDStream[(String, String)] = + KafkaUtils.createStream(ssc, "localhost:12345", "group", topics, StorageLevel.MEMORY_AND_DISK_SER_2) val kafkaParams = Map("zookeeper.connect"->"localhost:12345","group.id"->"consumer-group") - val test3 = KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( + val test3: ReceiverInputDStream[(String, String)] = + KafkaUtils.createStream[String, String, StringDecoder, StringDecoder]( ssc, kafkaParams, topics, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 1204cfba39f77..0beee8b4157ae 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -39,6 +39,7 @@ import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext import org.apache.spark.streaming.dstream._ +import org.apache.spark.streaming.receiver.Receiver /** * Input stream that subscribe messages from a Mqtt Broker. @@ -49,38 +50,36 @@ import org.apache.spark.streaming.dstream._ */ private[streaming] -class MQTTInputDStream[T: ClassTag]( +class MQTTInputDStream( @transient ssc_ : StreamingContext, brokerUrl: String, topic: String, storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_) with Logging { - - def getReceiver(): NetworkReceiver[T] = { - new MQTTReceiver(brokerUrl, topic, storageLevel).asInstanceOf[NetworkReceiver[T]] + ) extends ReceiverInputDStream[String](ssc_) with Logging { + + def getReceiver(): Receiver[String] = { + new MQTTReceiver(brokerUrl, topic, storageLevel) } } -private[streaming] -class MQTTReceiver(brokerUrl: String, - topic: String, - storageLevel: StorageLevel - ) extends NetworkReceiver[Any] { - lazy protected val blockGenerator = new BlockGenerator(storageLevel) +private[streaming] +class MQTTReceiver( + brokerUrl: String, + topic: String, + storageLevel: StorageLevel + ) extends Receiver[String](storageLevel) { def onStop() { - blockGenerator.stop() - } + } + def onStart() { - blockGenerator.start() - - // Set up persistence for messages - var peristance: MqttClientPersistence = new MemoryPersistence() + // Set up persistence for messages + val persistence = new MemoryPersistence() // Initializing Mqtt Client specifying brokerUrl, clientID and MqttClientPersistance - var client: MqttClient = new MqttClient(brokerUrl, MqttClient.generateClientId(), peristance) + val client = new MqttClient(brokerUrl, MqttClient.generateClientId(), persistence) // Connect to MqttBroker client.connect() @@ -89,18 +88,18 @@ class MQTTReceiver(brokerUrl: String, client.subscribe(topic) // Callback automatically triggers as and when new message arrives on specified topic - var callback: MqttCallback = new MqttCallback() { + val callback: MqttCallback = new MqttCallback() { // Handles Mqtt message override def messageArrived(arg0: String, arg1: MqttMessage) { - blockGenerator += new String(arg1.getPayload()) + store(new String(arg1.getPayload())) } override def deliveryComplete(arg0: IMqttDeliveryToken) { } override def connectionLost(arg0: Throwable) { - logInfo("Connection lost " + arg0) + restart("Connection lost ", arg0) } } diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala index 1b09ee5dc8f65..c5ffe51f9986c 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTUtils.scala @@ -19,9 +19,9 @@ package org.apache.spark.streaming.mqtt import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext, JavaDStream} import scala.reflect.ClassTag -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object MQTTUtils { /** @@ -36,8 +36,8 @@ object MQTTUtils { brokerUrl: String, topic: String, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[String] = { - new MQTTInputDStream[String](ssc, brokerUrl, topic, storageLevel) + ): ReceiverInputDStream[String] = { + new MQTTInputDStream(ssc, brokerUrl, topic, storageLevel) } /** @@ -51,7 +51,7 @@ object MQTTUtils { jssc: JavaStreamingContext, brokerUrl: String, topic: String - ): JavaDStream[String] = { + ): JavaReceiverInputDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, brokerUrl, topic) } @@ -68,7 +68,7 @@ object MQTTUtils { brokerUrl: String, topic: String, storageLevel: StorageLevel - ): JavaDStream[String] = { + ): JavaReceiverInputDStream[String] = { implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[String]] createStream(jssc.ssc, brokerUrl, topic, storageLevel) } diff --git a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java index 44743aaecf986..ce5aa1e0cdda4 100644 --- a/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java +++ b/external/mqtt/src/test/java/org/apache/spark/streaming/mqtt/JavaMQTTStreamSuite.java @@ -18,7 +18,7 @@ package org.apache.spark.streaming.mqtt; import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.JavaDStream; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.junit.Test; import org.apache.spark.streaming.LocalJavaStreamingContext; @@ -30,8 +30,8 @@ public void testMQTTStream() { String topic = "def"; // tests the API, does not actually test data receiving - JavaDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); - JavaDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, + JavaReceiverInputDStream test1 = MQTTUtils.createStream(ssc, brokerUrl, topic); + JavaReceiverInputDStream test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2()); } } diff --git a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala index 89c40ad4619c9..467fd263e2d64 100644 --- a/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala +++ b/external/mqtt/src/test/scala/org/apache/spark/streaming/mqtt/MQTTStreamSuite.scala @@ -19,6 +19,7 @@ package org.apache.spark.streaming.mqtt import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel +import org.apache.spark.streaming.dstream.ReceiverInputDStream class MQTTStreamSuite extends TestSuiteBase { @@ -28,8 +29,9 @@ class MQTTStreamSuite extends TestSuiteBase { val topic = "def" // tests the API, does not actually test data receiving - val test1 = MQTTUtils.createStream(ssc, brokerUrl, topic) - val test2 = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) + val test1: ReceiverInputDStream[String] = MQTTUtils.createStream(ssc, brokerUrl, topic) + val test2: ReceiverInputDStream[String] = + MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_AND_DISK_SER_2) // TODO: Actually test receiving data ssc.stop() diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala index 843a4a7a9ad72..7bca1407116fa 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterInputDStream.scala @@ -25,6 +25,8 @@ import twitter4j.auth.OAuthAuthorization import org.apache.spark.streaming._ import org.apache.spark.streaming.dstream._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.Logging +import org.apache.spark.streaming.receiver.Receiver /* A stream of Twitter statuses, potentially filtered by one or more keywords. * @@ -41,7 +43,7 @@ class TwitterInputDStream( twitterAuth: Option[Authorization], filters: Seq[String], storageLevel: StorageLevel - ) extends NetworkInputDStream[Status](ssc_) { + ) extends ReceiverInputDStream[Status](ssc_) { private def createOAuthAuthorization(): Authorization = { new OAuthAuthorization(new ConfigurationBuilder().build()) @@ -49,7 +51,7 @@ class TwitterInputDStream( private val authorization = twitterAuth.getOrElse(createOAuthAuthorization()) - override def getReceiver(): NetworkReceiver[Status] = { + override def getReceiver(): Receiver[Status] = { new TwitterReceiver(authorization, filters, storageLevel) } } @@ -59,27 +61,27 @@ class TwitterReceiver( twitterAuth: Authorization, filters: Seq[String], storageLevel: StorageLevel - ) extends NetworkReceiver[Status] { + ) extends Receiver[Status](storageLevel) with Logging { var twitterStream: TwitterStream = _ - lazy val blockGenerator = new BlockGenerator(storageLevel) - protected override def onStart() { - blockGenerator.start() + def onStart() { twitterStream = new TwitterStreamFactory().getInstance(twitterAuth) twitterStream.addListener(new StatusListener { def onStatus(status: Status) = { - blockGenerator += status + store(status) } // Unimplemented def onDeletionNotice(statusDeletionNotice: StatusDeletionNotice) {} def onTrackLimitationNotice(i: Int) {} def onScrubGeo(l: Long, l1: Long) {} def onStallWarning(stallWarning: StallWarning) {} - def onException(e: Exception) { stopOnError(e) } + def onException(e: Exception) { + restart("Error receiving tweets", e) + } }) - val query: FilterQuery = new FilterQuery + val query = new FilterQuery if (filters.size > 0) { query.track(filters.toArray) twitterStream.filter(query) @@ -89,8 +91,7 @@ class TwitterReceiver( logInfo("Twitter receiver started") } - protected override def onStop() { - blockGenerator.stop() + def onStop() { twitterStream.shutdown() logInfo("Twitter receiver stopped") } diff --git a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala index e8433b7e9f6fa..c6a9a2b73714f 100644 --- a/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala +++ b/external/twitter/src/main/scala/org/apache/spark/streaming/twitter/TwitterUtils.scala @@ -21,8 +21,8 @@ import twitter4j.Status import twitter4j.auth.Authorization import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaDStream, JavaStreamingContext} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream, DStream} object TwitterUtils { /** @@ -40,7 +40,7 @@ object TwitterUtils { twitterAuth: Option[Authorization], filters: Seq[String] = Nil, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[Status] = { + ): ReceiverInputDStream[Status] = { new TwitterInputDStream(ssc, twitterAuth, filters, storageLevel) } @@ -52,7 +52,7 @@ object TwitterUtils { * Storage level of the data will be the default StorageLevel.MEMORY_AND_DISK_SER_2. * @param jssc JavaStreamingContext object */ - def createStream(jssc: JavaStreamingContext): JavaDStream[Status] = { + def createStream(jssc: JavaStreamingContext): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, None) } @@ -65,7 +65,8 @@ object TwitterUtils { * @param jssc JavaStreamingContext object * @param filters Set of filter strings to get only those tweets that match them */ - def createStream(jssc: JavaStreamingContext, filters: Array[String]): JavaDStream[Status] = { + def createStream(jssc: JavaStreamingContext, filters: Array[String] + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, None, filters) } @@ -82,7 +83,7 @@ object TwitterUtils { jssc: JavaStreamingContext, filters: Array[String], storageLevel: StorageLevel - ): JavaDStream[Status] = { + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, None, filters, storageLevel) } @@ -92,7 +93,8 @@ object TwitterUtils { * @param jssc JavaStreamingContext object * @param twitterAuth Twitter4J Authorization */ - def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization): JavaDStream[Status] = { + def createStream(jssc: JavaStreamingContext, twitterAuth: Authorization + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, Some(twitterAuth)) } @@ -107,7 +109,7 @@ object TwitterUtils { jssc: JavaStreamingContext, twitterAuth: Authorization, filters: Array[String] - ): JavaDStream[Status] = { + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, Some(twitterAuth), filters) } @@ -123,7 +125,7 @@ object TwitterUtils { twitterAuth: Authorization, filters: Array[String], storageLevel: StorageLevel - ): JavaDStream[Status] = { + ): JavaReceiverInputDStream[Status] = { createStream(jssc.ssc, Some(twitterAuth), filters, storageLevel) } } diff --git a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala index 06ab0cdaf3b4e..93741e0375164 100644 --- a/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala +++ b/external/twitter/src/test/scala/org/apache/spark/streaming/twitter/TwitterStreamSuite.scala @@ -20,6 +20,8 @@ package org.apache.spark.streaming.twitter import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} import org.apache.spark.storage.StorageLevel import twitter4j.auth.{NullAuthorization, Authorization} +import org.apache.spark.streaming.dstream.ReceiverInputDStream +import twitter4j.Status class TwitterStreamSuite extends TestSuiteBase { @@ -29,13 +31,17 @@ class TwitterStreamSuite extends TestSuiteBase { val authorization: Authorization = NullAuthorization.getInstance() // tests the API, does not actually test data receiving - val test1 = TwitterUtils.createStream(ssc, None) - val test2 = TwitterUtils.createStream(ssc, None, filters) - val test3 = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) - val test4 = TwitterUtils.createStream(ssc, Some(authorization)) - val test5 = TwitterUtils.createStream(ssc, Some(authorization), filters) - val test6 = TwitterUtils.createStream(ssc, Some(authorization), filters, - StorageLevel.MEMORY_AND_DISK_SER_2) + val test1: ReceiverInputDStream[Status] = TwitterUtils.createStream(ssc, None) + val test2: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, None, filters) + val test3: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_AND_DISK_SER_2) + val test4: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, Some(authorization)) + val test5: ReceiverInputDStream[Status] = + TwitterUtils.createStream(ssc, Some(authorization), filters) + val test6: ReceiverInputDStream[Status] = TwitterUtils.createStream( + ssc, Some(authorization), filters, StorageLevel.MEMORY_AND_DISK_SER_2) // Note that actually testing the data receiving is hard as authentication keys are // necessary for accessing Twitter live stream diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala index a538c38dc4d6f..554705878ee78 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQReceiver.scala @@ -24,7 +24,7 @@ import akka.util.ByteString import akka.zeromq._ import org.apache.spark.Logging -import org.apache.spark.streaming.receivers._ +import org.apache.spark.streaming.receiver.ActorHelper /** * A receiver to subscribe to ZeroMQ stream. @@ -32,7 +32,7 @@ import org.apache.spark.streaming.receivers._ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, subscribe: Subscribe, bytesToObjects: Seq[ByteString] => Iterator[T]) - extends Actor with Receiver with Logging { + extends Actor with ActorHelper with Logging { override def preStart() = ZeroMQExtension(context.system) .newSocket(SocketType.Sub, Listener(self), Connect(publisherUrl), subscribe) @@ -46,9 +46,8 @@ private[streaming] class ZeroMQReceiver[T: ClassTag](publisherUrl: String, // We ignore first frame for processing as it is the topic val bytes = m.frames.tail - pushBlock(bytesToObjects(bytes)) + store(bytesToObjects(bytes)) case Closed => logInfo("received closed ") - } } diff --git a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala index b254e00714621..0469d0af8864a 100644 --- a/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala +++ b/external/zeromq/src/main/scala/org/apache/spark/streaming/zeromq/ZeroMQUtils.scala @@ -24,10 +24,10 @@ import akka.util.ByteString import akka.zeromq.Subscribe import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.receivers.ReceiverSupervisorStrategy import org.apache.spark.streaming.StreamingContext -import org.apache.spark.streaming.api.java.{JavaStreamingContext, JavaDStream} -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.api.java.{JavaReceiverInputDStream, JavaStreamingContext} +import org.apache.spark.streaming.dstream.{ReceiverInputDStream} +import org.apache.spark.streaming.receiver.ActorSupervisorStrategy object ZeroMQUtils { /** @@ -48,8 +48,8 @@ object ZeroMQUtils { subscribe: Subscribe, bytesToObjects: Seq[ByteString] => Iterator[T], storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy - ): DStream[T] = { + supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy + ): ReceiverInputDStream[T] = { ssc.actorStream(Props(new ZeroMQReceiver(publisherUrl, subscribe, bytesToObjects)), "ZeroMQReceiver", storageLevel, supervisorStrategy) } @@ -72,7 +72,7 @@ object ZeroMQUtils { bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], storageLevel: StorageLevel, supervisorStrategy: SupervisorStrategy - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator @@ -96,7 +96,7 @@ object ZeroMQUtils { subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]], storageLevel: StorageLevel - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator @@ -119,7 +119,7 @@ object ZeroMQUtils { publisherUrl: String, subscribe: Subscribe, bytesToObjects: JFunction[Array[Array[Byte]], java.lang.Iterable[T]] - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val fn = (x: Seq[ByteString]) => bytesToObjects.call(x.map(_.toArray).toArray).toIterator diff --git a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java index d2361e14b898a..417b91eecb0ee 100644 --- a/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java +++ b/external/zeromq/src/test/java/org/apache/spark/streaming/zeromq/JavaZeroMQStreamSuite.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming.zeromq; +import org.apache.spark.streaming.api.java.JavaReceiverInputDStream; import org.junit.Test; import akka.actor.SupervisorStrategy; import akka.util.ByteString; @@ -24,7 +25,6 @@ import org.apache.spark.api.java.function.Function; import org.apache.spark.storage.StorageLevel; import org.apache.spark.streaming.LocalJavaStreamingContext; -import org.apache.spark.streaming.api.java.JavaDStream; public class JavaZeroMQStreamSuite extends LocalJavaStreamingContext { @@ -39,11 +39,11 @@ public Iterable call(byte[][] bytes) throws Exception { } }; - JavaDStream test1 = ZeroMQUtils.createStream( + JavaReceiverInputDStream test1 = ZeroMQUtils.createStream( ssc, publishUrl, subscribe, bytesToObjects); - JavaDStream test2 = ZeroMQUtils.createStream( + JavaReceiverInputDStream test2 = ZeroMQUtils.createStream( ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2()); - JavaDStream test3 = ZeroMQUtils.createStream( + JavaReceiverInputDStream test3 = ZeroMQUtils.createStream( ssc,publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2(), SupervisorStrategy.defaultStrategy()); } diff --git a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala index 92d55a7a7b6e4..cc10ff6ae03cd 100644 --- a/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala +++ b/external/zeromq/src/test/scala/org/apache/spark/streaming/zeromq/ZeroMQStreamSuite.scala @@ -23,6 +23,7 @@ import akka.zeromq.Subscribe import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.{StreamingContext, TestSuiteBase} +import org.apache.spark.streaming.dstream.ReceiverInputDStream class ZeroMQStreamSuite extends TestSuiteBase { @@ -33,10 +34,12 @@ class ZeroMQStreamSuite extends TestSuiteBase { val bytesToObjects = (bytes: Seq[ByteString]) => null.asInstanceOf[Iterator[String]] // tests the API, does not actually test data receiving - val test1 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) - val test2 = ZeroMQUtils.createStream( + val test1: ReceiverInputDStream[String] = + ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects) + val test2: ReceiverInputDStream[String] = ZeroMQUtils.createStream( ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2) - val test3 = ZeroMQUtils.createStream(ssc, publishUrl, subscribe, bytesToObjects, + val test3: ReceiverInputDStream[String] = ZeroMQUtils.createStream( + ssc, publishUrl, subscribe, bytesToObjects, StorageLevel.MEMORY_AND_DISK_SER_2, SupervisorStrategy.defaultStrategy) // TODO: Actually test data receiving diff --git a/project/MimaBuild.scala b/project/MimaBuild.scala index 9cb31d70444ff..d540dc0a986e9 100644 --- a/project/MimaBuild.scala +++ b/project/MimaBuild.scala @@ -38,6 +38,7 @@ object MimaBuild { IO.read(excludeFile).split("\n") } + // Exclude a single class and its corresponding object def excludeClass(className: String) = { Seq( excludePackage(className), @@ -48,7 +49,16 @@ object MimaBuild { ProblemFilters.exclude[MissingTypesProblem](className + "$") ) } - def excludeSparkClass(className: String) = excludeClass("org.apache.spark." + className) + + // Exclude a Spark class, that is in the package org.apache.spark + def excludeSparkClass(className: String) = { + excludeClass("org.apache.spark." + className) + } + + // Exclude a Spark package, that is in the package org.apache.spark + def excludeSparkPackage(packageName: String) = { + excludePackage("org.apache.spark." + packageName) + } val packagePrivateExcludes = packagePrivateList.flatMap(excludeClass) @@ -58,10 +68,9 @@ object MimaBuild { SparkBuild.SPARK_VERSION match { case v if v.startsWith("1.0") => Seq( - excludePackage("org.apache.spark.api.java"), - excludePackage("org.apache.spark.streaming.api.java"), - excludePackage("org.apache.spark.streaming.scheduler"), - excludePackage("org.apache.spark.mllib") + excludeSparkPackage("api.java"), + excludeSparkPackage("mllib"), + excludeSparkPackage("streaming") ) ++ excludeSparkClass("rdd.ClassTags") ++ excludeSparkClass("util.XORShiftRandom") ++ @@ -69,14 +78,7 @@ object MimaBuild { excludeSparkClass("mllib.optimization.SquaredGradient") ++ excludeSparkClass("mllib.regression.RidgeRegressionWithSGD") ++ excludeSparkClass("mllib.regression.LassoWithSGD") ++ - excludeSparkClass("mllib.regression.LinearRegressionWithSGD") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#NetworkReceiverActor") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator") ++ - excludeSparkClass("streaming.dstream.NetworkReceiver#BlockGenerator#Block") ++ - excludeSparkClass("streaming.dstream.ReportError") ++ - excludeSparkClass("streaming.dstream.ReportBlock") ++ - excludeSparkClass("streaming.dstream.DStream") + excludeSparkClass("mllib.regression.LinearRegressionWithSGD") case _ => Seq() } @@ -87,5 +89,4 @@ object MimaBuild { previousArtifact := None, binaryIssueFilters ++= ignoredABIProblems(sparkHome) ) - } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala index d3339063cc079..b4adf0e9651a8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/DStreamGraph.scala @@ -21,7 +21,7 @@ import scala.collection.mutable.ArrayBuffer import java.io.{ObjectInputStream, IOException, ObjectOutputStream} import org.apache.spark.Logging import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.streaming.dstream.{DStream, NetworkInputDStream, InputDStream} +import org.apache.spark.streaming.dstream.{DStream, ReceiverInputDStream, InputDStream} final private[streaming] class DStreamGraph extends Serializable with Logging { @@ -103,9 +103,9 @@ final private[streaming] class DStreamGraph extends Serializable with Logging { def getOutputStreams() = this.synchronized { outputStreams.toArray } - def getNetworkInputStreams() = this.synchronized { - inputStreams.filter(_.isInstanceOf[NetworkInputDStream[_]]) - .map(_.asInstanceOf[NetworkInputDStream[_]]) + def getReceiverInputStreams() = this.synchronized { + inputStreams.filter(_.isInstanceOf[ReceiverInputDStream[_]]) + .map(_.asInstanceOf[ReceiverInputDStream[_]]) .toArray } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index e9a4f7ba22576..daa5c69bbadbf 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -31,12 +31,11 @@ import org.apache.hadoop.fs.Path import org.apache.hadoop.io.{LongWritable, Text} import org.apache.hadoop.mapreduce.{InputFormat => NewInputFormat} import org.apache.hadoop.mapreduce.lib.input.TextInputFormat - import org.apache.spark._ import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream._ -import org.apache.spark.streaming.receivers._ +import org.apache.spark.streaming.receiver.{ActorSupervisorStrategy, ActorReceiver, Receiver} import org.apache.spark.streaming.scheduler._ import org.apache.spark.streaming.ui.StreamingTab import org.apache.spark.util.MetadataCleaner @@ -139,7 +138,7 @@ class StreamingContext private[streaming] ( } } - private val nextNetworkInputStreamId = new AtomicInteger(0) + private val nextReceiverInputStreamId = new AtomicInteger(0) private[streaming] var checkpointDir: String = { if (isCheckpointPresent) { @@ -208,15 +207,26 @@ class StreamingContext private[streaming] ( if (isCheckpointPresent) cp_ else null } - private[streaming] def getNewNetworkStreamId() = nextNetworkInputStreamId.getAndIncrement() + private[streaming] def getNewReceiverStreamId() = nextReceiverInputStreamId.getAndIncrement() /** - * Create an input stream with any arbitrary user implemented network receiver. + * Create an input stream with any arbitrary user implemented receiver. * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html - * @param receiver Custom implementation of NetworkReceiver + * @param receiver Custom implementation of Receiver */ + @deprecated("Use receiverStream", "1.0.0") def networkStream[T: ClassTag]( - receiver: NetworkReceiver[T]): DStream[T] = { + receiver: Receiver[T]): ReceiverInputDStream[T] = { + receiverStream(receiver) + } + + /** + * Create an input stream with any arbitrary user implemented receiver. + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * @param receiver Custom implementation of Receiver + */ + def receiverStream[T: ClassTag]( + receiver: Receiver[T]): ReceiverInputDStream[T] = { new PluggableInputDStream[T](this, receiver) } @@ -236,9 +246,9 @@ class StreamingContext private[streaming] ( props: Props, name: String, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2, - supervisorStrategy: SupervisorStrategy = ReceiverSupervisorStrategy.defaultStrategy - ): DStream[T] = { - networkStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) + supervisorStrategy: SupervisorStrategy = ActorSupervisorStrategy.defaultStrategy + ): ReceiverInputDStream[T] = { + receiverStream(new ActorReceiver[T](props, name, storageLevel, supervisorStrategy)) } /** @@ -254,7 +264,7 @@ class StreamingContext private[streaming] ( hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[String] = { + ): ReceiverInputDStream[String] = { socketStream[String](hostname, port, SocketReceiver.bytesToLines, storageLevel) } @@ -273,7 +283,7 @@ class StreamingContext private[streaming] ( port: Int, converter: (InputStream) => Iterator[T], storageLevel: StorageLevel - ): DStream[T] = { + ): ReceiverInputDStream[T] = { new SocketInputDStream[T](this, hostname, port, converter, storageLevel) } @@ -292,7 +302,7 @@ class StreamingContext private[streaming] ( hostname: String, port: Int, storageLevel: StorageLevel = StorageLevel.MEMORY_AND_DISK_SER_2 - ): DStream[T] = { + ): ReceiverInputDStream[T] = { new RawInputDStream[T](this, hostname, port, storageLevel) } @@ -310,7 +320,7 @@ class StreamingContext private[streaming] ( K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]: ClassTag - ] (directory: String): DStream[(K, V)] = { + ] (directory: String): InputDStream[(K, V)] = { new FileInputDStream[K, V, F](this, directory) } @@ -330,7 +340,7 @@ class StreamingContext private[streaming] ( K: ClassTag, V: ClassTag, F <: NewInputFormat[K, V]: ClassTag - ] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): DStream[(K, V)] = { + ] (directory: String, filter: Path => Boolean, newFilesOnly: Boolean): InputDStream[(K, V)] = { new FileInputDStream[K, V, F](this, directory, filter, newFilesOnly) } @@ -356,7 +366,7 @@ class StreamingContext private[streaming] ( def queueStream[T: ClassTag]( queue: Queue[RDD[T]], oneAtATime: Boolean = true - ): DStream[T] = { + ): InputDStream[T] = { queueStream(queue, oneAtATime, sc.makeRDD(Seq[T](), 1)) } @@ -373,7 +383,7 @@ class StreamingContext private[streaming] ( queue: Queue[RDD[T]], oneAtATime: Boolean, defaultRDD: RDD[T] - ): DStream[T] = { + ): InputDStream[T] = { new QueueInputDStream(this, queue, oneAtATime, defaultRDD) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala index 13e2bacc92edc..505e4431e4350 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaDStream.scala @@ -97,6 +97,10 @@ class JavaDStream[T](val dstream: DStream[T])(implicit val classTag: ClassTag[T] } object JavaDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.DStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaDStream]]. + */ implicit def fromDStream[T: ClassTag](dstream: DStream[T]): JavaDStream[T] = new JavaDStream[T](dstream) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala new file mode 100644 index 0000000000000..91f8d342d2bf9 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.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.streaming.api.java + +import scala.reflect.ClassTag + +import org.apache.spark.streaming.dstream.InputDStream + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.InputDStream]]. + */ +class JavaInputDStream[T](val inputDStream: InputDStream[T]) + (implicit override val classTag: ClassTag[T]) extends JavaDStream[T](inputDStream) { +} + +object JavaInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.InputDStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaInputDStream]]. + */ + implicit def fromInputDStream[T: ClassTag]( + inputDStream: InputDStream[T]): JavaInputDStream[T] = { + new JavaInputDStream[T](inputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala new file mode 100644 index 0000000000000..add858530862b --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala @@ -0,0 +1,41 @@ +/* + * 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.streaming.api.java + +import org.apache.spark.streaming.dstream.InputDStream +import scala.reflect.ClassTag + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.InputDStream]] of + * key-value pairs. + */ +class JavaPairInputDStream[K, V](val inputDStream: InputDStream[(K, V)])( + implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V] + ) extends JavaPairDStream[K, V](inputDStream) { +} + +object JavaPairInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.InputDStream]] of pairs to a + * Java-friendly [[org.apache.spark.streaming.api.java.JavaPairInputDStream]]. + */ + implicit def fromInputDStream[K: ClassTag, V: ClassTag]( + inputDStream: InputDStream[(K, V)]): JavaPairInputDStream[K, V] = { + new JavaPairInputDStream[K, V](inputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala new file mode 100644 index 0000000000000..974b3e451642d --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala @@ -0,0 +1,42 @@ +/* + * 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.streaming.api.java + +import scala.reflect.ClassTag + +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.ReceiverInputDStream]], the + * abstract class for defining any input stream that receives data over the network. + */ +class JavaPairReceiverInputDStream[K, V](val receiverInputDStream: ReceiverInputDStream[(K, V)]) + (implicit override val kClassTag: ClassTag[K], override implicit val vClassTag: ClassTag[V]) + extends JavaPairInputDStream[K, V](receiverInputDStream) { +} + +object JavaPairReceiverInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaReceiverInputDStream]]. + */ + implicit def fromReceiverInputDStream[K: ClassTag, V: ClassTag]( + receiverInputDStream: ReceiverInputDStream[(K, V)]): JavaPairReceiverInputDStream[K, V] = { + new JavaPairReceiverInputDStream[K, V](receiverInputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala new file mode 100644 index 0000000000000..340ef979808b9 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala @@ -0,0 +1,41 @@ +/* + * 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.streaming.api.java + +import scala.reflect.ClassTag + +import org.apache.spark.streaming.dstream.ReceiverInputDStream + +/** + * A Java-friendly interface to [[org.apache.spark.streaming.dstream.ReceiverInputDStream]], the + * abstract class for defining any input stream that receives data over the network. + */ +class JavaReceiverInputDStream[T](val receiverInputDStream: ReceiverInputDStream[T]) + (implicit override val classTag: ClassTag[T]) extends JavaInputDStream[T](receiverInputDStream) { +} + +object JavaReceiverInputDStream { + /** + * Convert a scala [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] to a Java-friendly + * [[org.apache.spark.streaming.api.java.JavaReceiverInputDStream]]. + */ + implicit def fromReceiverInputDStream[T: ClassTag]( + receiverInputDStream: ReceiverInputDStream[T]): JavaReceiverInputDStream[T] = { + new JavaReceiverInputDStream[T](receiverInputDStream) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala index c800602d0959b..fbb2e9f85dd12 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaStreamingContext.scala @@ -35,7 +35,8 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming._ import org.apache.spark.streaming.scheduler.StreamingListener import org.apache.hadoop.conf.Configuration -import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.dstream.{PluggableInputDStream, ReceiverInputDStream, DStream} +import org.apache.spark.streaming.receiver.Receiver /** * A Java-friendly version of [[org.apache.spark.streaming.StreamingContext]] which is the main @@ -155,8 +156,10 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param port Port to connect to for receiving data * @param storageLevel Storage level to use for storing the received objects */ - def socketTextStream(hostname: String, port: Int, storageLevel: StorageLevel) - : JavaDStream[String] = { + def socketTextStream( + hostname: String, port: Int, + storageLevel: StorageLevel + ): JavaReceiverInputDStream[String] = { ssc.socketTextStream(hostname, port, storageLevel) } @@ -167,7 +170,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param hostname Hostname to connect to for receiving data * @param port Port to connect to for receiving data */ - def socketTextStream(hostname: String, port: Int): JavaDStream[String] = { + def socketTextStream(hostname: String, port: Int): JavaReceiverInputDStream[String] = { ssc.socketTextStream(hostname, port) } @@ -186,7 +189,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { port: Int, converter: JFunction[InputStream, java.lang.Iterable[T]], storageLevel: StorageLevel) - : JavaDStream[T] = { + : JavaReceiverInputDStream[T] = { def fn = (x: InputStream) => converter.call(x).toIterator implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] @@ -218,10 +221,11 @@ class JavaStreamingContext(val ssc: StreamingContext) { def rawSocketStream[T]( hostname: String, port: Int, - storageLevel: StorageLevel): JavaDStream[T] = { + storageLevel: StorageLevel): JavaReceiverInputDStream[T] = { implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port, storageLevel)) + JavaReceiverInputDStream.fromReceiverInputDStream( + ssc.rawSocketStream(hostname, port, storageLevel)) } /** @@ -233,10 +237,11 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param port Port to connect to for receiving data * @tparam T Type of the objects in the received blocks */ - def rawSocketStream[T](hostname: String, port: Int): JavaDStream[T] = { + def rawSocketStream[T](hostname: String, port: Int): JavaReceiverInputDStream[T] = { implicit val cmt: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] - JavaDStream.fromDStream(ssc.rawSocketStream(hostname, port)) + JavaReceiverInputDStream.fromReceiverInputDStream( + ssc.rawSocketStream(hostname, port)) } /** @@ -249,7 +254,8 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @tparam V Value type for reading HDFS file * @tparam F Input format for reading HDFS file */ - def fileStream[K, V, F <: NewInputFormat[K, V]](directory: String): JavaPairDStream[K, V] = { + def fileStream[K, V, F <: NewInputFormat[K, V]]( + directory: String): JavaPairInputDStream[K, V] = { implicit val cmk: ClassTag[K] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[K]] implicit val cmv: ClassTag[V] = @@ -275,7 +281,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { name: String, storageLevel: StorageLevel, supervisorStrategy: SupervisorStrategy - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name, storageLevel, supervisorStrategy) @@ -296,7 +302,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { props: Props, name: String, storageLevel: StorageLevel - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name, storageLevel) @@ -316,14 +322,14 @@ class JavaStreamingContext(val ssc: StreamingContext) { def actorStream[T]( props: Props, name: String - ): JavaDStream[T] = { + ): JavaReceiverInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] ssc.actorStream[T](props, name) } /** - * Creates an input stream from an queue of RDDs. In each batch, + * Create an input stream from an queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: changes to the queue after the stream is created will not be recognized. @@ -339,7 +345,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Creates an input stream from an queue of RDDs. In each batch, + * Create an input stream from an queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: changes to the queue after the stream is created will not be recognized. @@ -347,7 +353,10 @@ class JavaStreamingContext(val ssc: StreamingContext) { * @param oneAtATime Whether only one RDD should be consumed from the queue in every interval * @tparam T Type of objects in the RDD */ - def queueStream[T](queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean): JavaDStream[T] = { + def queueStream[T]( + queue: java.util.Queue[JavaRDD[T]], + oneAtATime: Boolean + ): JavaInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] @@ -356,7 +365,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { } /** - * Creates an input stream from an queue of RDDs. In each batch, + * Create an input stream from an queue of RDDs. In each batch, * it will process either one or all of the RDDs returned by the queue. * * NOTE: changes to the queue after the stream is created will not be recognized. @@ -368,7 +377,7 @@ class JavaStreamingContext(val ssc: StreamingContext) { def queueStream[T]( queue: java.util.Queue[JavaRDD[T]], oneAtATime: Boolean, - defaultRDD: JavaRDD[T]): JavaDStream[T] = { + defaultRDD: JavaRDD[T]): JavaInputDStream[T] = { implicit val cm: ClassTag[T] = implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] val sQueue = new scala.collection.mutable.Queue[RDD[T]] @@ -376,6 +385,17 @@ class JavaStreamingContext(val ssc: StreamingContext) { ssc.queueStream(sQueue, oneAtATime, defaultRDD.rdd) } + /** + * Create an input stream with any arbitrary user implemented receiver. + * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html + * @param receiver Custom implementation of Receiver + */ + def receiverStream[T](receiver: Receiver[T]): ReceiverInputDStream[T] = { + implicit val cm: ClassTag[T] = + implicitly[ClassTag[AnyRef]].asInstanceOf[ClassTag[T]] + ssc.receiverStream(receiver) + } + /** * Create a unified DStream from multiple DStreams of the same type and same slide duration. */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala index 226844c2284e3..aa1993f0580a8 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/InputDStream.scala @@ -30,7 +30,7 @@ import scala.reflect.ClassTag * FileInputDStream, a subclass of InputDStream, monitors a HDFS directory from the driver for * new files and generates RDDs with the new files. For implementing input streams * that requires running a receiver on the worker nodes, use - * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] as the parent class. + * [[org.apache.spark.streaming.dstream.ReceiverInputDStream]] as the parent class. * * @param ssc_ Streaming context that will execute this input stream */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala deleted file mode 100644 index 5a249706b4d2f..0000000000000 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/NetworkInputDStream.scala +++ /dev/null @@ -1,362 +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.streaming.dstream - -import java.nio.ByteBuffer -import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} - -import scala.collection.mutable.{ArrayBuffer, HashMap} -import scala.concurrent.Await -import scala.reflect.ClassTag - -import akka.actor.{Actor, Props} -import akka.pattern.ask - -import org.apache.spark.{Logging, SparkEnv} -import org.apache.spark.rdd.{BlockRDD, RDD} -import org.apache.spark.storage.{BlockId, StorageLevel, StreamBlockId} -import org.apache.spark.streaming._ -import org.apache.spark.streaming.scheduler.{AddBlock, DeregisterReceiver, ReceivedBlockInfo, RegisterReceiver} -import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} -import org.apache.spark.util.{AkkaUtils, Utils} - -/** - * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] - * that has to start a receiver on worker nodes to receive external data. - * Specific implementations of NetworkInputDStream must - * define the getReceiver() function that gets the receiver object of type - * [[org.apache.spark.streaming.dstream.NetworkReceiver]] that will be sent - * to the workers to receive data. - * @param ssc_ Streaming context that will execute this input stream - * @tparam T Class type of the object of this stream - */ -abstract class NetworkInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) - extends InputDStream[T](ssc_) { - - /** Keeps all received blocks information */ - private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] - - /** This is an unique identifier for the network input stream. */ - val id = ssc.getNewNetworkStreamId() - - /** - * Gets the receiver object that will be sent to the worker nodes - * to receive data. This method needs to defined by any specific implementation - * of a NetworkInputDStream. - */ - def getReceiver(): NetworkReceiver[T] - - // Nothing to start or stop as both taken care of by the NetworkInputTracker. - def start() {} - - def stop() {} - - /** Ask NetworkInputTracker for received data blocks and generates RDDs with them. */ - override def compute(validTime: Time): Option[RDD[T]] = { - // If this is called for any time before the start time of the context, - // then this returns an empty RDD. This may happen when recovering from a - // master failure - if (validTime >= graph.startTime) { - val blockInfo = ssc.scheduler.networkInputTracker.getReceivedBlockInfo(id) - receivedBlockInfo(validTime) = blockInfo - val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) - Some(new BlockRDD[T](ssc.sc, blockIds)) - } else { - Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) - } - } - - /** Get information on received blocks. */ - private[streaming] def getReceivedBlockInfo(time: Time) = { - receivedBlockInfo(time) - } - - /** - * Clear metadata that are older than `rememberDuration` of this DStream. - * This is an internal method that should not be called directly. This - * implementation overrides the default implementation to clear received - * block information. - */ - private[streaming] override def clearMetadata(time: Time) { - super.clearMetadata(time) - val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) - receivedBlockInfo --= oldReceivedBlocks.keys - logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + - (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) - } -} - - -private[streaming] sealed trait NetworkReceiverMessage -private[streaming] case class StopReceiver(msg: String) extends NetworkReceiverMessage - -/** - * Abstract class of a receiver that can be run on worker nodes to receive external data. See - * [[org.apache.spark.streaming.dstream.NetworkInputDStream]] for an explanation. - */ -abstract class NetworkReceiver[T: ClassTag]() extends Serializable with Logging { - - /** Local SparkEnv */ - lazy protected val env = SparkEnv.get - - /** Remote Akka actor for the NetworkInputTracker */ - lazy protected val trackerActor = { - val ip = env.conf.get("spark.driver.host", "localhost") - val port = env.conf.getInt("spark.driver.port", 7077) - val url = "akka.tcp://spark@%s:%s/user/NetworkInputTracker".format(ip, port) - env.actorSystem.actorSelection(url) - } - - /** Akka actor for receiving messages from the NetworkInputTracker in the driver */ - lazy protected val actor = env.actorSystem.actorOf( - Props(new NetworkReceiverActor()), "NetworkReceiver-" + streamId) - - /** Timeout for Akka actor messages */ - lazy protected val askTimeout = AkkaUtils.askTimeout(env.conf) - - /** Thread that starts the receiver and stays blocked while data is being received */ - lazy protected val receivingThread = Thread.currentThread() - - /** Exceptions that occurs while receiving data */ - protected lazy val exceptions = new ArrayBuffer[Exception] - - /** Identifier of the stream this receiver is associated with */ - protected var streamId: Int = -1 - - /** - * This method will be called to start receiving data. All your receiver - * starting code should be implemented by defining this function. - */ - protected def onStart() - - /** This method will be called to stop receiving data. */ - protected def onStop() - - /** Conveys a placement preference (hostname) for this receiver. */ - def getLocationPreference() : Option[String] = None - - /** - * Start the receiver. First is accesses all the lazy members to - * materialize them. Then it calls the user-defined onStart() method to start - * other threads, etc required to receiver the data. - */ - def start() { - try { - // Access the lazy vals to materialize them - env - actor - receivingThread - - // Call user-defined onStart() - logInfo("Starting receiver") - onStart() - - // Wait until interrupt is called on this thread - while(true) Thread.sleep(100000) - } catch { - case ie: InterruptedException => - logInfo("Receiving thread has been interrupted, receiver " + streamId + " stopped") - case e: Exception => - logError("Error receiving data in receiver " + streamId, e) - exceptions += e - } - - // Call user-defined onStop() - logInfo("Stopping receiver") - try { - onStop() - } catch { - case e: Exception => - logError("Error stopping receiver " + streamId, e) - exceptions += e - } - - val message = if (exceptions.isEmpty) { - null - } else if (exceptions.size == 1) { - val e = exceptions.head - "Exception in receiver " + streamId + ": " + e.getMessage + "\n" + e.getStackTraceString - } else { - "Multiple exceptions in receiver " + streamId + "(" + exceptions.size + "):\n" - exceptions.zipWithIndex.map { - case (e, i) => "Exception " + i + ": " + e.getMessage + "\n" + e.getStackTraceString - }.mkString("\n") - } - - logInfo("Deregistering receiver " + streamId) - val future = trackerActor.ask(DeregisterReceiver(streamId, message))(askTimeout) - Await.result(future, askTimeout) - logInfo("Deregistered receiver " + streamId) - env.actorSystem.stop(actor) - logInfo("Stopped receiver " + streamId) - } - - /** - * Stop the receiver. First it interrupts the main receiving thread, - * that is, the thread that called receiver.start(). - */ - def stop() { - // Stop receiving by interrupting the receiving thread - receivingThread.interrupt() - logInfo("Interrupted receiving thread " + receivingThread + " for stopping") - } - - /** - * Stop the receiver and reports exception to the tracker. - * This should be called whenever an exception is to be handled on any thread - * of the receiver. - */ - protected def stopOnError(e: Exception) { - logError("Error receiving data", e) - exceptions += e - stop() - } - - /** - * Push a block (as an ArrayBuffer filled with data) into the block manager. - */ - def pushBlock( - blockId: StreamBlockId, - arrayBuffer: ArrayBuffer[T], - metadata: Any, - level: StorageLevel - ) { - env.blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], level) - trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, arrayBuffer.size, metadata)) - logDebug("Pushed block " + blockId) - } - - /** - * Push a block (as bytes) into the block manager. - */ - def pushBlock( - blockId: StreamBlockId, - bytes: ByteBuffer, - metadata: Any, - level: StorageLevel - ) { - env.blockManager.putBytes(blockId, bytes, level) - trackerActor ! AddBlock(ReceivedBlockInfo(streamId, blockId, -1, metadata)) - } - - /** Set the ID of the DStream that this receiver is associated with */ - protected[streaming] def setStreamId(id: Int) { - streamId = id - } - - /** A helper actor that communicates with the NetworkInputTracker */ - private class NetworkReceiverActor extends Actor { - - override def preStart() { - val msg = RegisterReceiver( - streamId, NetworkReceiver.this.getClass.getSimpleName, Utils.localHostName(), self) - val future = trackerActor.ask(msg)(askTimeout) - Await.result(future, askTimeout) - logInfo("Registered receiver " + streamId) - } - - override def receive() = { - case StopReceiver => - logInfo("Received stop signal") - stop() - } - } - - /** - * Batches objects created by a [[org.apache.spark.streaming.dstream.NetworkReceiver]] and puts - * them into appropriately named blocks at regular intervals. This class starts two threads, - * one to periodically start a new batch and prepare the previous batch of as a block, - * the other to push the blocks into the block manager. - */ - class BlockGenerator(storageLevel: StorageLevel) - extends Serializable with Logging { - - case class Block(id: StreamBlockId, buffer: ArrayBuffer[T], metadata: Any = null) - - val clock = new SystemClock() - val blockInterval = env.conf.getLong("spark.streaming.blockInterval", 200) - val blockIntervalTimer = new RecurringTimer(clock, blockInterval, updateCurrentBuffer, - "BlockGenerator") - val blockStorageLevel = storageLevel - val blocksForPushing = new ArrayBlockingQueue[Block](1000) - val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } - - var currentBuffer = new ArrayBuffer[T] - var stopped = false - - def start() { - blockIntervalTimer.start() - blockPushingThread.start() - logInfo("Started BlockGenerator") - } - - def stop() { - blockIntervalTimer.stop(false) - stopped = true - blockPushingThread.join() - logInfo("Stopped BlockGenerator") - } - - def += (obj: T): Unit = synchronized { - currentBuffer += obj - } - - private def updateCurrentBuffer(time: Long): Unit = synchronized { - try { - val newBlockBuffer = currentBuffer - currentBuffer = new ArrayBuffer[T] - if (newBlockBuffer.size > 0) { - val blockId = StreamBlockId(NetworkReceiver.this.streamId, time - blockInterval) - val newBlock = new Block(blockId, newBlockBuffer) - blocksForPushing.add(newBlock) - } - } catch { - case ie: InterruptedException => - logInfo("Block updating timer thread was interrupted") - case e: Exception => - NetworkReceiver.this.stopOnError(e) - } - } - - private def keepPushingBlocks() { - logInfo("Started block pushing thread") - try { - while(!stopped) { - Option(blocksForPushing.poll(100, TimeUnit.MILLISECONDS)) match { - case Some(block) => - NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) - case None => - } - } - // Push out the blocks that are still left - logInfo("Pushing out the last " + blocksForPushing.size() + " blocks") - while (!blocksForPushing.isEmpty) { - val block = blocksForPushing.take() - NetworkReceiver.this.pushBlock(block.id, block.buffer, block.metadata, storageLevel) - logInfo("Blocks left to push " + blocksForPushing.size()) - } - logInfo("Stopped blocks pushing thread") - } catch { - case ie: InterruptedException => - logInfo("Block pushing thread was interrupted") - case e: Exception => - NetworkReceiver.this.stopOnError(e) - } - } - } -} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala index 6f9477020a459..186e1bf03a944 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PluggableInputDStream.scala @@ -19,13 +19,14 @@ package org.apache.spark.streaming.dstream import org.apache.spark.streaming.StreamingContext import scala.reflect.ClassTag +import org.apache.spark.streaming.receiver.Receiver private[streaming] class PluggableInputDStream[T: ClassTag]( @transient ssc_ : StreamingContext, - receiver: NetworkReceiver[T]) extends NetworkInputDStream[T](ssc_) { + receiver: Receiver[T]) extends ReceiverInputDStream[T](ssc_) { - def getReceiver(): NetworkReceiver[T] = { + def getReceiver(): Receiver[T] = { receiver } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala index dea0f26f908fb..e2925b9e03ec3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/RawInputDStream.scala @@ -17,7 +17,7 @@ package org.apache.spark.streaming.dstream -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.storage.{StorageLevel, StreamBlockId} import org.apache.spark.streaming.StreamingContext @@ -28,6 +28,7 @@ import java.nio.ByteBuffer import java.nio.channels.{ReadableByteChannel, SocketChannel} import java.io.EOFException import java.util.concurrent.ArrayBlockingQueue +import org.apache.spark.streaming.receiver.Receiver /** @@ -42,21 +43,19 @@ class RawInputDStream[T: ClassTag]( host: String, port: Int, storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_ ) with Logging { + ) extends ReceiverInputDStream[T](ssc_ ) with Logging { - def getReceiver(): NetworkReceiver[T] = { - new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[NetworkReceiver[T]] + def getReceiver(): Receiver[T] = { + new RawNetworkReceiver(host, port, storageLevel).asInstanceOf[Receiver[T]] } } private[streaming] class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel) - extends NetworkReceiver[Any] { + extends Receiver[Any](storageLevel) with Logging { var blockPushingThread: Thread = null - override def getLocationPreference = None - def onStart() { // Open a socket to the target address and keep reading from it logInfo("Connecting to " + host + ":" + port) @@ -73,9 +72,8 @@ class RawNetworkReceiver(host: String, port: Int, storageLevel: StorageLevel) var nextBlockNumber = 0 while (true) { val buffer = queue.take() - val blockId = StreamBlockId(streamId, nextBlockNumber) nextBlockNumber += 1 - pushBlock(blockId, buffer, null, storageLevel) + store(buffer) } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala new file mode 100644 index 0000000000000..75cabdbf8da26 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/ReceiverInputDStream.scala @@ -0,0 +1,94 @@ +/* + * 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.streaming.dstream + +import scala.collection.mutable.HashMap +import scala.reflect.ClassTag + +import org.apache.spark.rdd.{BlockRDD, RDD} +import org.apache.spark.storage.BlockId +import org.apache.spark.streaming._ +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler.ReceivedBlockInfo + +/** + * Abstract class for defining any [[org.apache.spark.streaming.dstream.InputDStream]] + * that has to start a receiver on worker nodes to receive external data. + * Specific implementations of NetworkInputDStream must + * define `the getReceiver()` function that gets the receiver object of type + * [[org.apache.spark.streaming.receiver.Receiver]] that will be sent + * to the workers to receive data. + * @param ssc_ Streaming context that will execute this input stream + * @tparam T Class type of the object of this stream + */ +abstract class ReceiverInputDStream[T: ClassTag](@transient ssc_ : StreamingContext) + extends InputDStream[T](ssc_) { + + /** Keeps all received blocks information */ + private lazy val receivedBlockInfo = new HashMap[Time, Array[ReceivedBlockInfo]] + + /** This is an unique identifier for the network input stream. */ + val id = ssc.getNewReceiverStreamId() + + /** + * Gets the receiver object that will be sent to the worker nodes + * to receive data. This method needs to defined by any specific implementation + * of a NetworkInputDStream. + */ + def getReceiver(): Receiver[T] + + // Nothing to start or stop as both taken care of by the ReceiverInputTracker. + def start() {} + + def stop() {} + + /** Ask ReceiverInputTracker for received data blocks and generates RDDs with them. */ + override def compute(validTime: Time): Option[RDD[T]] = { + // If this is called for any time before the start time of the context, + // then this returns an empty RDD. This may happen when recovering from a + // master failure + if (validTime >= graph.startTime) { + val blockInfo = ssc.scheduler.receiverTracker.getReceivedBlockInfo(id) + receivedBlockInfo(validTime) = blockInfo + val blockIds = blockInfo.map(_.blockId.asInstanceOf[BlockId]) + Some(new BlockRDD[T](ssc.sc, blockIds)) + } else { + Some(new BlockRDD[T](ssc.sc, Array[BlockId]())) + } + } + + /** Get information on received blocks. */ + private[streaming] def getReceivedBlockInfo(time: Time) = { + receivedBlockInfo(time) + } + + /** + * Clear metadata that are older than `rememberDuration` of this DStream. + * This is an internal method that should not be called directly. This + * implementation overrides the default implementation to clear received + * block information. + */ + private[streaming] override def clearMetadata(time: Time) { + super.clearMetadata(time) + val oldReceivedBlocks = receivedBlockInfo.filter(_._1 <= (time - rememberDuration)) + receivedBlockInfo --= oldReceivedBlocks.keys + logDebug("Cleared " + oldReceivedBlocks.size + " RDDs that were older than " + + (time - rememberDuration) + ": " + oldReceivedBlocks.keys.mkString(", ")) + } +} + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 63d94d1cc670a..1e32727eacfa3 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -24,7 +24,9 @@ import org.apache.spark.util.NextIterator import scala.reflect.ClassTag import java.io._ -import java.net.Socket +import java.net.{UnknownHostException, Socket} +import org.apache.spark.Logging +import org.apache.spark.streaming.receiver.Receiver private[streaming] class SocketInputDStream[T: ClassTag]( @@ -33,9 +35,9 @@ class SocketInputDStream[T: ClassTag]( port: Int, bytesToObjects: InputStream => Iterator[T], storageLevel: StorageLevel - ) extends NetworkInputDStream[T](ssc_) { + ) extends ReceiverInputDStream[T](ssc_) { - def getReceiver(): NetworkReceiver[T] = { + def getReceiver(): Receiver[T] = { new SocketReceiver(host, port, bytesToObjects, storageLevel) } } @@ -46,26 +48,52 @@ class SocketReceiver[T: ClassTag]( port: Int, bytesToObjects: InputStream => Iterator[T], storageLevel: StorageLevel - ) extends NetworkReceiver[T] { + ) extends Receiver[T](storageLevel) with Logging { - lazy protected val blockGenerator = new BlockGenerator(storageLevel) + var socket: Socket = null + var receivingThread: Thread = null - override def getLocationPreference = None + def onStart() { + receivingThread = new Thread("Socket Receiver") { + override def run() { + connect() + receive() + } + } + receivingThread.start() + } - protected def onStart() { - logInfo("Connecting to " + host + ":" + port) - val socket = new Socket(host, port) - logInfo("Connected to " + host + ":" + port) - blockGenerator.start() - val iterator = bytesToObjects(socket.getInputStream()) - while(iterator.hasNext) { - val obj = iterator.next - blockGenerator += obj + def onStop() { + if (socket != null) { + socket.close() + } + socket = null + if (receivingThread != null) { + receivingThread.join() } } - protected def onStop() { - blockGenerator.stop() + def connect() { + try { + logInfo("Connecting to " + host + ":" + port) + socket = new Socket(host, port) + } catch { + case e: Exception => + restart("Could not connect to " + host + ":" + port, e) + } + } + + def receive() { + try { + logInfo("Connected to " + host + ":" + port) + val iterator = bytesToObjects(socket.getInputStream()) + while(!isStopped && iterator.hasNext) { + store(iterator.next) + } + } catch { + case e: Exception => + restart("Error receiving data from socket", e) + } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala similarity index 66% rename from streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala rename to streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index da0d364ae7bdb..821cf19481d44 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receivers/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -15,26 +15,22 @@ * limitations under the License. */ -package org.apache.spark.streaming.receivers +package org.apache.spark.streaming.receiver -import akka.actor.{ Actor, PoisonPill, Props, SupervisorStrategy } -import akka.actor.{ actorRef2Scala, ActorRef } -import akka.actor.{ PossiblyHarmful, OneForOneStrategy } -import akka.actor.SupervisorStrategy._ +import java.util.concurrent.atomic.AtomicInteger import scala.concurrent.duration._ import scala.language.postfixOps import scala.reflect.ClassTag -import org.apache.spark.storage.{StorageLevel, StreamBlockId} -import org.apache.spark.streaming.dstream.NetworkReceiver - -import java.util.concurrent.atomic.AtomicInteger - -import scala.collection.mutable.ArrayBuffer +import akka.actor._ +import akka.actor.SupervisorStrategy.{Escalate, Restart} +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.storage.StorageLevel +import java.nio.ByteBuffer /** A helper with set of defaults for supervisor strategy */ -object ReceiverSupervisorStrategy { +object ActorSupervisorStrategy { val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = 15 millis) { @@ -50,9 +46,9 @@ object ReceiverSupervisorStrategy { * Find more details at: http://spark.apache.org/docs/latest/streaming-custom-receivers.html * * @example {{{ - * class MyActor extends Actor with Receiver{ + * class MyActor extends Actor with ActorHelper{ * def receive { - * case anything: String => pushBlock(anything) + * case anything: String => store(anything) * } * } * @@ -65,29 +61,40 @@ object ReceiverSupervisorStrategy { * to ensure the type safety, i.e parametrized type of push block and InputDStream * should be same. */ -trait Receiver { +trait ActorHelper { self: Actor => // to ensure that this can be added to Actor classes only + /** Store an iterator of received data as a data block into Spark's memory. */ + def store[T](iter: Iterator[T]) { + println("Storing iterator") + context.parent ! IteratorData(iter) + } + /** - * Push an iterator received data into Spark Streaming for processing + * Store the bytes of received data as a data block into Spark's memory. Note + * that the data in the ByteBuffer must be serialized using the same serializer + * that Spark is configured to use. */ - def pushBlock[T: ClassTag](iter: Iterator[T]) { - context.parent ! Data(iter) + def store(bytes: ByteBuffer) { + context.parent ! ByteBufferData(bytes) } /** - * Push a single item of received data into Spark Streaming for processing + * Store a single item of received data to Spark's memory. + * These single items will be aggregated together into data blocks before + * being pushed into Spark's memory. */ - def pushBlock[T: ClassTag](data: T) { - context.parent ! Data(data) + def store[T](item: T) { + println("Storing item") + context.parent ! SingleItemData(item) } } /** * Statistics for querying the supervisor about state of workers. Used in * conjunction with `StreamingContext.actorStream` and - * [[org.apache.spark.streaming.receivers.Receiver]]. + * [[org.apache.spark.streaming.receiver.ActorHelper]]. */ case class Statistics(numberOfMsgs: Int, numberOfWorkers: Int, @@ -95,7 +102,10 @@ case class Statistics(numberOfMsgs: Int, otherInfo: String) /** Case class to receive data sent by child actors */ -private[streaming] case class Data[T: ClassTag](data: T) +private[streaming] sealed trait ActorReceiverData +private[streaming] case class SingleItemData[T](item: T) extends ActorReceiverData +private[streaming] case class IteratorData[T](iterator: Iterator[T]) extends ActorReceiverData +private[streaming] case class ByteBufferData(bytes: ByteBuffer) extends ActorReceiverData /** * Provides Actors as receivers for receiving stream. @@ -117,16 +127,13 @@ private[streaming] case class Data[T: ClassTag](data: T) * }}} */ private[streaming] class ActorReceiver[T: ClassTag]( - props: Props, - name: String, - storageLevel: StorageLevel, - receiverSupervisorStrategy: SupervisorStrategy) - extends NetworkReceiver[T] { + props: Props, + name: String, + storageLevel: StorageLevel, + receiverSupervisorStrategy: SupervisorStrategy + ) extends Receiver[T](storageLevel) with Logging { - protected lazy val blocksGenerator: BlockGenerator = - new BlockGenerator(storageLevel) - - protected lazy val supervisor = env.actorSystem.actorOf(Props(new Supervisor), + protected lazy val supervisor = SparkEnv.get.actorSystem.actorOf(Props(new Supervisor), "Supervisor" + streamId) class Supervisor extends Actor { @@ -140,12 +147,18 @@ private[streaming] class ActorReceiver[T: ClassTag]( def receive = { - case Data(iter: Iterator[_]) => pushBlock(iter.asInstanceOf[Iterator[T]]) + case IteratorData(iterator) => + println("received iterator") + store(iterator.asInstanceOf[Iterator[T]]) - case Data(msg) => - blocksGenerator += msg.asInstanceOf[T] + case SingleItemData(msg) => + println("received single") + store(msg.asInstanceOf[T]) n.incrementAndGet + case ByteBufferData(bytes) => + store(bytes) + case props: Props => val worker = context.actorOf(props) logInfo("Started receiver worker at:" + worker.path) @@ -165,20 +178,14 @@ private[streaming] class ActorReceiver[T: ClassTag]( } } - protected def pushBlock(iter: Iterator[T]) { - val buffer = new ArrayBuffer[T] - buffer ++= iter - pushBlock(StreamBlockId(streamId, System.nanoTime()), buffer, null, storageLevel) - } - - protected def onStart() = { - blocksGenerator.start() + def onStart() = { supervisor logInfo("Supervision tree for receivers initialized at:" + supervisor.path) } - protected def onStop() = { + def onStop() = { supervisor ! PoisonPill } } + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala new file mode 100644 index 0000000000000..78cc2daa56e53 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/BlockGenerator.scala @@ -0,0 +1,142 @@ +/* + * 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.streaming.receiver + +import java.util.concurrent.{ArrayBlockingQueue, TimeUnit} + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.util.{RecurringTimer, SystemClock} + +/** Listener object for BlockGenerator events */ +private[streaming] trait BlockGeneratorListener { + /** Called when a new block needs to be pushed */ + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) + /** Called when an error has occurred in BlockGenerator */ + def onError(message: String, throwable: Throwable) +} + +/** + * Generates batches of objects received by a + * [[org.apache.spark.streaming.receiver.Receiver]] and puts them into appropriately + * named blocks at regular intervals. This class starts two threads, + * one to periodically start a new batch and prepare the previous batch of as a block, + * the other to push the blocks into the block manager. + */ +private[streaming] class BlockGenerator( + listener: BlockGeneratorListener, + receiverId: Int, + conf: SparkConf + ) extends Logging { + + private case class Block(id: StreamBlockId, buffer: ArrayBuffer[Any]) + + private val clock = new SystemClock() + private val blockInterval = conf.getLong("spark.streaming.blockInterval", 200) + private val blockIntervalTimer = + new RecurringTimer(clock, blockInterval, updateCurrentBuffer, "BlockGenerator") + private val blockQueueSize = conf.getInt("spark.streaming.blockQueueSize", 10) + private val blocksForPushing = new ArrayBlockingQueue[Block](blockQueueSize) + private val blockPushingThread = new Thread() { override def run() { keepPushingBlocks() } } + + @volatile private var currentBuffer = new ArrayBuffer[Any] + @volatile private var stopped = false + + /** Start block generating and pushing threads. */ + def start() { + blockIntervalTimer.start() + blockPushingThread.start() + logInfo("Started BlockGenerator") + } + + /** Stop all threads. */ + def stop() { + logInfo("Stopping BlockGenerator") + blockIntervalTimer.stop(interruptTimer = false) + stopped = true + logInfo("Waiting for block pushing thread") + blockPushingThread.join() + logInfo("Stopped BlockGenerator") + } + + /** + * Push a single data item into the buffer. All received data items + * will be periodically pushed into BlockManager. + */ + def += (data: Any): Unit = synchronized { + currentBuffer += data + } + + /** Change the buffer to which single records are added to. */ + private def updateCurrentBuffer(time: Long): Unit = synchronized { + try { + val newBlockBuffer = currentBuffer + currentBuffer = new ArrayBuffer[Any] + if (newBlockBuffer.size > 0) { + val blockId = StreamBlockId(receiverId, time - blockInterval) + val newBlock = new Block(blockId, newBlockBuffer) + blocksForPushing.put(newBlock) // put is blocking when queue is full + logDebug("Last element in " + blockId + " is " + newBlockBuffer.last) + } + } catch { + case ie: InterruptedException => + logInfo("Block updating timer thread was interrupted") + case t: Throwable => + reportError("Error in block updating thread", t) + } + } + + /** Keep pushing blocks to the BlockManager. */ + private def keepPushingBlocks() { + logInfo("Started block pushing thread") + try { + while(!stopped) { + Option(blocksForPushing.poll(100, TimeUnit.MILLISECONDS)) match { + case Some(block) => pushBlock(block) + case None => + } + } + // Push out the blocks that are still left + logInfo("Pushing out the last " + blocksForPushing.size() + " blocks") + while (!blocksForPushing.isEmpty) { + logDebug("Getting block ") + val block = blocksForPushing.take() + pushBlock(block) + logInfo("Blocks left to push " + blocksForPushing.size()) + } + logInfo("Stopped block pushing thread") + } catch { + case ie: InterruptedException => + logInfo("Block pushing thread was interrupted") + case t: Throwable => + reportError("Error in block pushing thread", t) + } + } + + private def reportError(message: String, t: Throwable) { + logError(message, t) + listener.onError(message, t) + } + + private def pushBlock(block: Block) { + listener.onPushBlock(block.id, block.buffer) + logInfo("Pushed block " + block.id) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala new file mode 100644 index 0000000000000..44eecf1dd2567 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -0,0 +1,236 @@ +/* + * 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.streaming.receiver + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer +import scala.collection.JavaConversions._ + +import org.apache.spark.storage.StorageLevel + +/** + * Abstract class of a receiver that can be run on worker nodes to receive external data. A + * custom receiver can be defined by defining the functions onStart() and onStop(). onStart() + * should define the setup steps necessary to start receiving data, + * and onStop() should define the cleanup steps necessary to stop receiving data. A custom + * receiver would look something like this. + * + * @example {{{ + * class MyReceiver(storageLevel: StorageLevel) extends NetworkReceiver[String](storageLevel) { + * def onStart() { + * // Setup stuff (start threads, open sockets, etc.) to start receiving data. + * // Must start new thread to receive data, as onStart() must be non-blocking. + * + * // Call store(...) in those threads to store received data into Spark's memory. + * + * // Call stop(...), restart() or reportError(...) on any thread based on how + * // different errors should be handled. + * + * // See corresponding method documentation for more details + * } + * + * def onStop() { + * // Cleanup stuff (stop threads, close sockets, etc.) to stop receiving data. + * } + * } + * }}} + */ +abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable { + + /** + * This method is called by the system when the receiver is started. This function + * must initialize all resources (threads, buffers, etc.) necessary for receiving data. + * This function must be non-blocking, so receiving the data must occur on a different + * thread. Received data can be stored with Spark by calling `store(data)`. + * + * If there are errors in threads started here, then following options can be done + * (i) `reportError(...)` can be called to report the error to the driver. + * The receiving of data will continue uninterrupted. + * (ii) `stop(...)` can be called to stop receiving data. This will call `onStop()` to + * clear up all resources allocated (threads, buffers, etc.) during `onStart()`. + * (iii) `restart(...)` can be called to restart the receiver. This will call `onStop()` + * immediately, and then `onStart()` after a delay. + */ + def onStart() + + /** + * This method is called by the system when the receiver is stopped. All resources + * (threads, buffers, etc.) setup in `onStart()` must be cleaned up in this method. + */ + def onStop() + + /** Override this to specify a preferred location (hostname). */ + def preferredLocation : Option[String] = None + + /** + * Store a single item of received data to Spark's memory. + * These single items will be aggregated together into data blocks before + * being pushed into Spark's memory. + */ + def store(dataItem: T) { + executor.pushSingle(dataItem) + } + + /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ + def store(dataBuffer: ArrayBuffer[T]) { + executor.pushArrayBuffer(dataBuffer, None, None) + } + + /** + * Store an ArrayBuffer of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(dataBuffer: ArrayBuffer[T], metadata: Any) { + executor.pushArrayBuffer(dataBuffer, Some(metadata), None) + } + + /** Store an iterator of received data as a data block into Spark's memory. */ + def store(dataIterator: Iterator[T]) { + executor.pushIterator(dataIterator, None, None) + } + + /** + * Store an iterator of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(dataIterator: java.util.Iterator[T], metadata: Any) { + executor.pushIterator(dataIterator, Some(metadata), None) + } + + /** Store an iterator of received data as a data block into Spark's memory. */ + def store(dataIterator: java.util.Iterator[T]) { + executor.pushIterator(dataIterator, None, None) + } + + /** + * Store an iterator of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(dataIterator: Iterator[T], metadata: Any) { + executor.pushIterator(dataIterator, Some(metadata), None) + } + + /** + * Store the bytes of received data as a data block into Spark's memory. Note + * that the data in the ByteBuffer must be serialized using the same serializer + * that Spark is configured to use. + */ + def store(bytes: ByteBuffer) { + executor.pushBytes(bytes, None, None) + } + + /** + * Store the bytes of received data as a data block into Spark's memory. + * The metadata will be associated with this block of data + * for being used in the corresponding InputDStream. + */ + def store(bytes: ByteBuffer, metadata: Any) { + executor.pushBytes(bytes, Some(metadata), None) + } + + /** Report exceptions in receiving data. */ + def reportError(message: String, throwable: Throwable) { + executor.reportError(message, throwable) + } + + /** + * Restart the receiver. This will call `onStop()` immediately and return. + * Asynchronously, after a delay, `onStart()` will be called. + * The `message` will be reported to the driver. + * The delay is defined by the Spark configuration + * `spark.streaming.receiverRestartDelay`. + */ + def restart(message: String) { + executor.restartReceiver(message) + } + + /** + * Restart the receiver. This will call `onStop()` immediately and return. + * Asynchronously, after a delay, `onStart()` will be called. + * The `message` and `exception` will be reported to the driver. + * The delay is defined by the Spark configuration + * `spark.streaming.receiverRestartDelay`. + */ + def restart(message: String, error: Throwable) { + executor.restartReceiver(message, Some(error)) + } + + /** + * Restart the receiver. This will call `onStop()` immediately and return. + * Asynchronously, after the given delay, `onStart()` will be called. + */ + def restart(message: String, error: Throwable, millisecond: Int) { + executor.restartReceiver(message, Some(error), millisecond) + } + + /** Stop the receiver completely. */ + def stop(message: String) { + executor.stop(message, None) + } + + /** Stop the receiver completely due to an exception */ + def stop(message: String, error: Throwable) { + executor.stop(message, Some(error)) + } + + def isStarted(): Boolean = { + executor.isReceiverStarted() + } + + /** Check if receiver has been marked for stopping. */ + def isStopped(): Boolean = { + !executor.isReceiverStarted() + } + + /** Get unique identifier of this receiver. */ + def streamId = id + + /* + * ================= + * Private methods + * ================= + */ + + /** Identifier of the stream this receiver is associated with. */ + private var id: Int = -1 + + /** Handler object that runs the receiver. This is instantiated lazily in the worker. */ + private[streaming] var executor_ : ReceiverSupervisor = null + + /** Set the ID of the DStream that this receiver is associated with. */ + private[streaming] def setReceiverId(id_ : Int) { + id = id_ + } + + /** Attach Network Receiver executor to this receiver. */ + private[streaming] def attachExecutor(exec: ReceiverSupervisor) { + assert(executor_ == null) + executor_ = exec + } + + /** Get the attached executor. */ + private def executor = { + assert(executor_ != null, "Executor has not been attached to this receiver") + executor_ + } +} + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala new file mode 100644 index 0000000000000..6ab3ca6ea5fa6 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala @@ -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. + */ + +package org.apache.spark.streaming.receiver + +/** Messages sent to the NetworkReceiver. */ +private[streaming] sealed trait NetworkReceiverMessage +private[streaming] object StopReceiver extends NetworkReceiverMessage + diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala new file mode 100644 index 0000000000000..256b3335e49aa --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -0,0 +1,180 @@ +/* + * 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.streaming.receiver + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.{Logging, SparkConf} +import org.apache.spark.storage.StreamBlockId +import java.util.concurrent.CountDownLatch +import scala.concurrent._ +import ExecutionContext.Implicits.global + +/** + * Abstract class that is responsible for supervising a Receiver in the worker. + * It provides all the necessary interfaces for handling the data received by the receiver. + */ +private[streaming] abstract class ReceiverSupervisor( + receiver: Receiver[_], + conf: SparkConf + ) extends Logging { + + /** Enumeration to identify current state of the StreamingContext */ + object ReceiverState extends Enumeration { + type CheckpointState = Value + val Initialized, Started, Stopped = Value + } + import ReceiverState._ + + // Attach the executor to the receiver + receiver.attachExecutor(this) + + /** Receiver id */ + protected val streamId = receiver.streamId + + /** Has the receiver been marked for stop. */ + private val stopLatch = new CountDownLatch(1) + + /** Time between a receiver is stopped and started again */ + private val defaultRestartDelay = conf.getInt("spark.streaming.receiverRestartDelay", 2000) + + /** Exception associated with the stopping of the receiver */ + @volatile protected var stoppingError: Throwable = null + + /** State of the receiver */ + @volatile private[streaming] var receiverState = Initialized + + /** Push a single data item to backend data store. */ + def pushSingle(data: Any) + + /** Store the bytes of received data as a data block into Spark's memory. */ + def pushBytes( + bytes: ByteBuffer, + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) + + /** Store a iterator of received data as a data block into Spark's memory. */ + def pushIterator( + iterator: Iterator[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) + + /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ + def pushArrayBuffer( + arrayBuffer: ArrayBuffer[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) + + /** Report errors. */ + def reportError(message: String, throwable: Throwable) + + /** Start the executor */ + def start() { + startReceiver() + } + + /** Mark the executor and the receiver for stopping */ + def stop(message: String, error: Option[Throwable]) { + stoppingError = error.orNull + stopReceiver(message, error) + stopLatch.countDown() + } + + /** Start receiver */ + def startReceiver(): Unit = synchronized { + try { + logInfo("Starting receiver") + onReceiverStart() + receiverState = Started + } catch { + case t: Throwable => + stop("Error starting receiver " + streamId, Some(t)) + } + } + + /** Stop receiver */ + def stopReceiver(message: String, error: Option[Throwable]): Unit = synchronized { + try { + receiverState = Stopped + onReceiverStop(message, error) + } catch { + case t: Throwable => + stop("Error stopping receiver " + streamId, Some(t)) + } + } + + /** Restart receiver with delay */ + def restartReceiver(message: String, error: Option[Throwable] = None) { + restartReceiver(message, error, defaultRestartDelay) + } + + /** Restart receiver with delay */ + def restartReceiver(message: String, error: Option[Throwable], delay: Int) { + logWarning("Restarting receiver with delay " + delay + " ms: " + message, + error.getOrElse(null)) + stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) + future { + logDebug("Sleeping for " + delay) + Thread.sleep(delay) + logDebug("Starting receiver again") + startReceiver() + logInfo("Receiver started again") + } + } + + /** Called when the receiver needs to be started */ + protected def onReceiverStart(): Unit = synchronized { + // Call user-defined onStart() + logInfo("Calling receiver onStart") + receiver.onStart() + logInfo("Called receiver onStart") + } + + /** Called when the receiver needs to be stopped */ + protected def onReceiverStop(message: String, error: Option[Throwable]): Unit = synchronized { + // Call user-defined onStop() + logInfo("Calling receiver onStop") + receiver.onStop() + logInfo("Called receiver onStop") + } + + /** Check if receiver has been marked for stopping */ + def isReceiverStarted() = { + logDebug("state = " + receiverState) + receiverState == Started + } + + /** Wait the thread until the executor is stopped */ + def awaitTermination() { + stopLatch.await() + logInfo("Waiting for executor stop is over") + if (stoppingError != null) { + logError("Stopped executor with error: " + stoppingError) + } else { + logWarning("Stopped executor without error") + } + if (stoppingError != null) { + throw stoppingError + } + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala new file mode 100644 index 0000000000000..2a3521bd46ae7 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -0,0 +1,180 @@ +/* + * 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.streaming.receiver + +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicLong + +import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} +import scala.concurrent.Await + +import akka.actor.{Actor, Props} +import akka.pattern.ask + +import org.apache.spark.{Logging, SparkEnv} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.scheduler._ +import org.apache.spark.util.{Utils, AkkaUtils} +import org.apache.spark.storage.StreamBlockId +import org.apache.spark.streaming.scheduler.DeregisterReceiver +import org.apache.spark.streaming.scheduler.AddBlock +import scala.Some +import org.apache.spark.streaming.scheduler.RegisterReceiver +import com.google.common.base.Throwables + +/** + * Concrete implementation of [[org.apache.spark.streaming.receiver.ReceiverSupervisor]] + * which provides all the necessary functionality for handling the data received by + * the receiver. Specifically, it creates a [[org.apache.spark.streaming.receiver.BlockGenerator]] + * object that is used to divide the received data stream into blocks of data. + */ +private[streaming] class ReceiverSupervisorImpl( + receiver: Receiver[_], + env: SparkEnv + ) extends ReceiverSupervisor(receiver, env.conf) with Logging { + + private val blockManager = env.blockManager + + private val storageLevel = receiver.storageLevel + + /** Remote Akka actor for the ReceiverTracker */ + private val trackerActor = { + val ip = env.conf.get("spark.driver.host", "localhost") + val port = env.conf.getInt("spark.driver.port", 7077) + val url = "akka.tcp://spark@%s:%s/user/ReceiverTracker".format(ip, port) + env.actorSystem.actorSelection(url) + } + + /** Timeout for Akka actor messages */ + private val askTimeout = AkkaUtils.askTimeout(env.conf) + + /** Akka actor for receiving messages from the ReceiverTracker in the driver */ + private val actor = env.actorSystem.actorOf( + Props(new Actor { + override def preStart() { + logInfo("Registered receiver " + streamId) + val msg = RegisterReceiver( + streamId, receiver.getClass.getSimpleName, Utils.localHostName(), self) + val future = trackerActor.ask(msg)(askTimeout) + Await.result(future, askTimeout) + } + + override def receive() = { + case StopReceiver => + logInfo("Received stop signal") + stop("Stopped by driver", None) + } + }), "Receiver-" + streamId + "-" + System.currentTimeMillis()) + + /** Unique block ids if one wants to add blocks directly */ + private val newBlockId = new AtomicLong(System.currentTimeMillis()) + + /** Divides received data records into data blocks for pushing in BlockManager. */ + private val blockGenerator = new BlockGenerator(new BlockGeneratorListener { + def onError(message: String, throwable: Throwable) { + reportError(message, throwable) + } + + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) { + pushArrayBuffer(arrayBuffer, None, Some(blockId)) + } + }, streamId, env.conf) + + /** Push a single record of received data into block generator. */ + def pushSingle(data: Any) { + blockGenerator += (data) + } + + /** Store an ArrayBuffer of received data as a data block into Spark's memory. */ + def pushArrayBuffer( + arrayBuffer: ArrayBuffer[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + val blockId = optionalBlockId.getOrElse(nextBlockId) + val time = System.currentTimeMillis + blockManager.put(blockId, arrayBuffer.asInstanceOf[ArrayBuffer[Any]], + storageLevel, tellMaster = true) + logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") + reportPushedBlock(blockId, arrayBuffer.size, optionalMetadata) + } + + /** Store a iterator of received data as a data block into Spark's memory. */ + def pushIterator( + iterator: Iterator[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + val blockId = optionalBlockId.getOrElse(nextBlockId) + val time = System.currentTimeMillis + blockManager.put(blockId, iterator, storageLevel, tellMaster = true) + logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") + reportPushedBlock(blockId, -1, optionalMetadata) + } + + /** Store the bytes of received data as a data block into Spark's memory. */ + def pushBytes( + bytes: ByteBuffer, + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + val blockId = optionalBlockId.getOrElse(nextBlockId) + val time = System.currentTimeMillis + blockManager.putBytes(blockId, bytes, storageLevel, tellMaster = true) + logDebug("Pushed block " + blockId + " in " + (System.currentTimeMillis - time) + " ms") + reportPushedBlock(blockId, -1, optionalMetadata) + } + + /** Report pushed block */ + def reportPushedBlock(blockId: StreamBlockId, numRecords: Long, optionalMetadata: Option[Any]) { + val blockInfo = ReceivedBlockInfo(streamId, blockId, numRecords, optionalMetadata.orNull) + trackerActor ! AddBlock(blockInfo) + logDebug("Reported block " + blockId) + } + + /** Report error to the receiver tracker */ + def reportError(message: String, error: Throwable) { + val errorString = Option(error).map(Throwables.getStackTraceAsString).getOrElse("") + trackerActor ! ReportError(streamId, message, errorString) + logWarning("Reported error " + message + " - " + error) + } + + override def onReceiverStart() { + blockGenerator.start() + super.onReceiverStart() + } + + override def onReceiverStop(message: String, error: Option[Throwable]) { + super.onReceiverStop(message, error) + blockGenerator.stop() + logInfo("Deregistering receiver " + streamId) + val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") + val future = trackerActor.ask( + DeregisterReceiver(streamId, message, errorString))(askTimeout) + Await.result(future, askTimeout) + logInfo("Stopped receiver " + streamId) + } + + override def stop(message: String, error: Option[Throwable]) { + super.stop(message, error) + env.actorSystem.stop(actor) + } + + /** Generate new block ID */ + private def nextBlockId = StreamBlockId(streamId, newBlockId.getAndIncrement) +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala index e564eccba2df5..374848358e700 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobGenerator.scala @@ -38,6 +38,7 @@ private[streaming] class JobGenerator(jobScheduler: JobScheduler) extends Logging { private val ssc = jobScheduler.ssc + private val conf = ssc.conf private val graph = ssc.graph val clock = { @@ -93,26 +94,31 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { if (processReceivedData) { logInfo("Stopping JobGenerator gracefully") val timeWhenStopStarted = System.currentTimeMillis() - val stopTimeout = 10 * ssc.graph.batchDuration.milliseconds + val stopTimeout = conf.getLong( + "spark.streaming.gracefulStopTimeout", + 10 * ssc.graph.batchDuration.milliseconds + ) val pollTime = 100 // To prevent graceful stop to get stuck permanently def hasTimedOut = { val timedOut = System.currentTimeMillis() - timeWhenStopStarted > stopTimeout - if (timedOut) logWarning("Timed out while stopping the job generator") + if (timedOut) { + logWarning("Timed out while stopping the job generator (timeout = " + stopTimeout + ")") + } timedOut } // Wait until all the received blocks in the network input tracker has // been consumed by network input DStreams, and jobs have been generated with them logInfo("Waiting for all received blocks to be consumed for job generation") - while(!hasTimedOut && jobScheduler.networkInputTracker.hasMoreReceivedBlockIds) { + while(!hasTimedOut && jobScheduler.receiverTracker.hasMoreReceivedBlockIds) { Thread.sleep(pollTime) } logInfo("Waited for all received blocks to be consumed for job generation") // Stop generating jobs - val stopTime = timer.stop(false) + val stopTime = timer.stop(interruptTimer = false) graph.stop() logInfo("Stopped generation timer") @@ -214,7 +220,7 @@ class JobGenerator(jobScheduler: JobScheduler) extends Logging { SparkEnv.set(ssc.env) Try(graph.generateJobs(time)) match { case Success(jobs) => - val receivedBlockInfo = graph.getNetworkInputStreams.map { stream => + val receivedBlockInfo = graph.getReceiverInputStreams.map { stream => val streamId = stream.id val receivedBlockInfo = stream.getReceivedBlockInfo(time) (streamId, receivedBlockInfo) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala index d9ada99b472ac..1b034b9fb187c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/JobScheduler.scala @@ -46,7 +46,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { // These two are created only when scheduler starts. // eventActor not being null means the scheduler has been started and not stopped - var networkInputTracker: NetworkInputTracker = null + var receiverTracker: ReceiverTracker = null private var eventActor: ActorRef = null @@ -61,8 +61,8 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { }), "JobScheduler") listenerBus.start() - networkInputTracker = new NetworkInputTracker(ssc) - networkInputTracker.start() + receiverTracker = new ReceiverTracker(ssc) + receiverTracker.start() jobGenerator.start() logInfo("Started JobScheduler") } @@ -72,7 +72,7 @@ class JobScheduler(val ssc: StreamingContext) extends Logging { logDebug("Stopping JobScheduler") // First, stop receiving - networkInputTracker.stop() + receiverTracker.stop() // Second, stop generating jobs. If it has to process all received data, // then this will wait for all the processing through JobScheduler to be over. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala similarity index 68% rename from streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala rename to streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 438e72a7ced89..3d2537f6f23dc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/NetworkInputTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -21,12 +21,11 @@ import scala.collection.mutable.{HashMap, SynchronizedMap, SynchronizedQueue} import scala.language.existentials import akka.actor._ - import org.apache.spark.{Logging, SparkEnv, SparkException} import org.apache.spark.SparkContext._ import org.apache.spark.storage.StreamBlockId import org.apache.spark.streaming.{StreamingContext, Time} -import org.apache.spark.streaming.dstream.{NetworkReceiver, StopReceiver} +import org.apache.spark.streaming.receiver.{Receiver, ReceiverSupervisorImpl, StopReceiver} import org.apache.spark.util.AkkaUtils /** Information about receiver */ @@ -34,7 +33,7 @@ case class ReceiverInfo(streamId: Int, typ: String, location: String) { override def toString = s"$typ-$streamId" } -/** Information about blocks received by the network receiver */ +/** Information about blocks received by the receiver */ case class ReceivedBlockInfo( streamId: Int, blockId: StreamBlockId, @@ -43,20 +42,21 @@ case class ReceivedBlockInfo( ) /** - * Messages used by the NetworkReceiver and the NetworkInputTracker to communicate + * Messages used by the NetworkReceiver and the ReceiverTracker to communicate * with each other. */ -private[streaming] sealed trait NetworkInputTrackerMessage +private[streaming] sealed trait ReceiverTrackerMessage private[streaming] case class RegisterReceiver( streamId: Int, typ: String, host: String, receiverActor: ActorRef - ) extends NetworkInputTrackerMessage + ) extends ReceiverTrackerMessage private[streaming] case class AddBlock(receivedBlockInfo: ReceivedBlockInfo) - extends NetworkInputTrackerMessage -private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) - extends NetworkInputTrackerMessage + extends ReceiverTrackerMessage +private[streaming] case class ReportError(streamId: Int, message: String, error: String) +private[streaming] case class DeregisterReceiver(streamId: Int, msg: String, error: String) + extends ReceiverTrackerMessage /** * This class manages the execution of the receivers of NetworkInputDStreams. Instance of @@ -64,11 +64,11 @@ private[streaming] case class DeregisterReceiver(streamId: Int, msg: String) * has been called because it needs the final set of input streams at the time of instantiation. */ private[streaming] -class NetworkInputTracker(ssc: StreamingContext) extends Logging { +class ReceiverTracker(ssc: StreamingContext) extends Logging { - val networkInputStreams = ssc.graph.getNetworkInputStreams() - val networkInputStreamMap = Map(networkInputStreams.map(x => (x.id, x)): _*) - val receiverExecutor = new ReceiverExecutor() + val receiverInputStreams = ssc.graph.getReceiverInputStreams() + val receiverInputStreamMap = Map(receiverInputStreams.map(x => (x.id, x)): _*) + val receiverExecutor = new ReceiverLauncher() val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] @@ -83,27 +83,27 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { /** Start the actor and receiver execution thread. */ def start() = synchronized { if (actor != null) { - throw new SparkException("NetworkInputTracker already started") + throw new SparkException("ReceiverTracker already started") } - if (!networkInputStreams.isEmpty) { - actor = ssc.env.actorSystem.actorOf(Props(new NetworkInputTrackerActor), - "NetworkInputTracker") + if (!receiverInputStreams.isEmpty) { + actor = ssc.env.actorSystem.actorOf(Props(new ReceiverTrackerActor), + "ReceiverTracker") receiverExecutor.start() - logInfo("NetworkInputTracker started") + logInfo("ReceiverTracker started") } } /** Stop the receiver execution thread. */ def stop() = synchronized { - if (!networkInputStreams.isEmpty && actor != null) { + if (!receiverInputStreams.isEmpty && actor != null) { // First, stop the receivers receiverExecutor.stop() // Finally, stop the actor ssc.env.actorSystem.stop(actor) actor = null - logInfo("NetworkInputTracker stopped") + logInfo("ReceiverTracker stopped") } } @@ -126,20 +126,26 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { receiverActor: ActorRef, sender: ActorRef ) { - if (!networkInputStreamMap.contains(streamId)) { + if (!receiverInputStreamMap.contains(streamId)) { throw new Exception("Register received for unexpected id " + streamId) } receiverInfo += ((streamId, receiverActor)) ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted( ReceiverInfo(streamId, typ, host) )) - logInfo("Registered receiver for network stream " + streamId + " from " + sender.path.address) + logInfo("Registered receiver for stream " + streamId + " from " + sender.path.address) } /** Deregister a receiver */ - def deregisterReceiver(streamId: Int, message: String) { + def deregisterReceiver(streamId: Int, message: String, error: String) { receiverInfo -= streamId - logError("Deregistered receiver for network stream " + streamId + " with message:\n" + message) + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(streamId, message, error)) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logError(s"Deregistered receiver for stream $streamId: $messageWithError") } /** Add new blocks for the given stream */ @@ -149,27 +155,40 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { receivedBlockInfo.blockId) } + /** Report error sent by a receiver */ + def reportError(streamId: Int, message: String, error: String) { + ssc.scheduler.listenerBus.post(StreamingListenerReceiverError(streamId, message, error)) + val messageWithError = if (error != null && !error.isEmpty) { + s"$message - $error" + } else { + s"$message" + } + logWarning(s"Error reported by receiver for stream $streamId: $messageWithError") + } + /** Check if any blocks are left to be processed */ def hasMoreReceivedBlockIds: Boolean = { !receivedBlockInfo.values.forall(_.isEmpty) } /** Actor to receive messages from the receivers. */ - private class NetworkInputTrackerActor extends Actor { + private class ReceiverTrackerActor extends Actor { def receive = { case RegisterReceiver(streamId, typ, host, receiverActor) => registerReceiver(streamId, typ, host, receiverActor, sender) sender ! true case AddBlock(receivedBlockInfo) => addBlocks(receivedBlockInfo) - case DeregisterReceiver(streamId, message) => - deregisterReceiver(streamId, message) + case ReportError(streamId, message, error) => + reportError(streamId, message, error) + case DeregisterReceiver(streamId, message, error) => + deregisterReceiver(streamId, message, error) sender ! true } } /** This thread class runs all the receivers on the cluster. */ - class ReceiverExecutor { + class ReceiverLauncher { @transient val env = ssc.env @transient val thread = new Thread() { override def run() { @@ -177,7 +196,7 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { SparkEnv.set(env) startReceivers() } catch { - case ie: InterruptedException => logInfo("ReceiverExecutor interrupted") + case ie: InterruptedException => logInfo("ReceiverLauncher interrupted") } } } @@ -203,37 +222,39 @@ class NetworkInputTracker(ssc: StreamingContext) extends Logging { } /** - * Get the receivers from the NetworkInputDStreams, distributes them to the + * Get the receivers from the ReceiverInputDStreams, distributes them to the * worker nodes as a parallel collection, and runs them. */ private def startReceivers() { - val receivers = networkInputStreams.map(nis => { + val receivers = receiverInputStreams.map(nis => { val rcvr = nis.getReceiver() - rcvr.setStreamId(nis.id) + rcvr.setReceiverId(nis.id) rcvr }) // Right now, we only honor preferences if all receivers have them - val hasLocationPreferences = receivers.map(_.getLocationPreference().isDefined) - .reduce(_ && _) + val hasLocationPreferences = receivers.map(_.preferredLocation.isDefined).reduce(_ && _) // Create the parallel collection of receivers to distributed them on the worker nodes val tempRDD = if (hasLocationPreferences) { - val receiversWithPreferences = - receivers.map(r => (r, Seq(r.getLocationPreference().toString))) - ssc.sc.makeRDD[NetworkReceiver[_]](receiversWithPreferences) + val receiversWithPreferences = receivers.map(r => (r, Seq(r.preferredLocation.get))) + ssc.sc.makeRDD[Receiver[_]](receiversWithPreferences) } else { ssc.sc.makeRDD(receivers, receivers.size) } // Function to start the receiver on the worker node - val startReceiver = (iterator: Iterator[NetworkReceiver[_]]) => { + val startReceiver = (iterator: Iterator[Receiver[_]]) => { if (!iterator.hasNext) { - throw new Exception("Could not start receiver as details not found.") + throw new SparkException( + "Could not start receiver as object not found.") } - iterator.next().start() + val receiver = iterator.next() + val executor = new ReceiverSupervisorImpl(receiver, SparkEnv.get) + executor.start() + executor.awaitTermination() } // Run the dummy Spark job to ensure that all slaves have registered. // This avoids all the receivers to be scheduled on the same node. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 5db40ebbeb1de..9d6ec1fa33549 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -18,6 +18,7 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.Queue + import org.apache.spark.util.Distribution /** Base trait for events related to StreamingListener */ @@ -26,8 +27,13 @@ sealed trait StreamingListenerEvent case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent + case class StreamingListenerReceiverStarted(receiverInfo: ReceiverInfo) extends StreamingListenerEvent +case class StreamingListenerReceiverError(streamId: Int, message: String, error: String) + extends StreamingListenerEvent +case class StreamingListenerReceiverStopped(streamId: Int, message: String, error: String) + extends StreamingListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object StreamingListenerShutdown extends StreamingListenerEvent @@ -41,14 +47,20 @@ trait StreamingListener { /** Called when a receiver has been started */ def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { } + /** Called when a receiver has reported an error */ + def onReceiverError(receiverError: StreamingListenerReceiverError) { } + + /** Called when a receiver has been stopped */ + def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { } + /** Called when a batch of jobs has been submitted for processing. */ def onBatchSubmitted(batchSubmitted: StreamingListenerBatchSubmitted) { } - /** Called when processing of a batch of jobs has completed. */ - def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } - /** Called when processing of a batch of jobs has started. */ def onBatchStarted(batchStarted: StreamingListenerBatchStarted) { } + + /** Called when processing of a batch of jobs has completed. */ + def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala index ea03dfc7bfeea..398724d9e8130 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListenerBus.scala @@ -40,6 +40,10 @@ private[spark] class StreamingListenerBus() extends Logging { event match { case receiverStarted: StreamingListenerReceiverStarted => listeners.foreach(_.onReceiverStarted(receiverStarted)) + case receiverError: StreamingListenerReceiverError => + listeners.foreach(_.onReceiverError(receiverError)) + case receiverStopped: StreamingListenerReceiverStopped => + listeners.foreach(_.onReceiverStopped(receiverStopped)) case batchSubmitted: StreamingListenerBatchSubmitted => listeners.foreach(_.onBatchSubmitted(batchSubmitted)) case batchStarted: StreamingListenerBatchStarted => diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 8b025b09ed34d..bf637c1446314 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -62,8 +62,8 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St totalCompletedBatches += 1L } - def numNetworkReceivers = synchronized { - ssc.graph.getNetworkInputStreams().size + def numReceivers = synchronized { + ssc.graph.getReceiverInputStreams().size } def numTotalCompletedBatches: Long = synchronized { @@ -101,7 +101,7 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St def receivedRecordsDistributions: Map[Int, Option[Distribution]] = synchronized { val latestBatchInfos = retainedBatches.reverse.take(batchInfoLimit) val latestBlockInfos = latestBatchInfos.map(_.receivedBlockInfo) - (0 until numNetworkReceivers).map { receiverId => + (0 until numReceivers).map { receiverId => val blockInfoOfParticularReceiver = latestBlockInfos.map { batchInfo => batchInfo.get(receiverId).getOrElse(Array.empty) } @@ -117,11 +117,11 @@ private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends St def lastReceivedBatchRecords: Map[Int, Long] = { val lastReceivedBlockInfoOption = lastReceivedBatch.map(_.receivedBlockInfo) lastReceivedBlockInfoOption.map { lastReceivedBlockInfo => - (0 until numNetworkReceivers).map { receiverId => + (0 until numReceivers).map { receiverId => (receiverId, lastReceivedBlockInfo(receiverId).map(_.numRecords).sum) }.toMap }.getOrElse { - (0 until numNetworkReceivers).map(receiverId => (receiverId, 0L)).toMap + (0 until numReceivers).map(receiverId => (receiverId, 0L)).toMap } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 6607437db560a..8fe1219356cdc 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -40,7 +40,7 @@ private[ui] class StreamingPage(parent: StreamingTab) val content = generateBasicStats() ++

      ++

      Statistics over last {listener.retainedCompletedBatches.size} processed batches

      ++ - generateNetworkStatsTable() ++ + generateReceiverStats() ++ generateBatchStatsTable() UIUtils.headerSparkPage( content, parent.basePath, parent.appName, "Streaming", parent.headerTabs, parent, Some(5000)) @@ -57,7 +57,7 @@ private[ui] class StreamingPage(parent: StreamingTab) Time since start: {formatDurationVerbose(timeSinceStart)}
    • - Network receivers: {listener.numNetworkReceivers} + Network receivers: {listener.numReceivers}
    • Batch interval: {formatDurationVerbose(listener.batchDuration)} @@ -71,8 +71,8 @@ private[ui] class StreamingPage(parent: StreamingTab) } - /** Generate stats of data received over the network the streaming program */ - private def generateNetworkStatsTable(): Seq[Node] = { + /** Generate stats of data received by the receivers in the streaming program */ + private def generateReceiverStats(): Seq[Node] = { val receivedRecordDistributions = listener.receivedRecordsDistributions val lastBatchReceivedRecord = listener.lastReceivedBatchRecords val table = if (receivedRecordDistributions.size > 0) { @@ -86,13 +86,13 @@ private[ui] class StreamingPage(parent: StreamingTab) "75th percentile rate\n[records/sec]", "Maximum rate\n[records/sec]" ) - val dataRows = (0 until listener.numNetworkReceivers).map { receiverId => + val dataRows = (0 until listener.numReceivers).map { receiverId => val receiverInfo = listener.receiverInfo(receiverId) val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) - val receiverLastBatchRecords = formatDurationVerbose(lastBatchReceivedRecord(receiverId)) + val receiverLastBatchRecords = formatNumber(lastBatchReceivedRecord(receiverId)) val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => - d.getQuantiles().map(r => formatDurationVerbose(r.toLong)) + d.getQuantiles().map(r => formatNumber(r.toLong)) }.getOrElse { Seq(emptyCell, emptyCell, emptyCell, emptyCell, emptyCell) } @@ -104,8 +104,8 @@ private[ui] class StreamingPage(parent: StreamingTab) } val content = -
      Network Input Statistics
      ++ -
      {table.getOrElse("No network receivers")}
      +
      Receiver Statistics
      ++ +
      {table.getOrElse("No receivers")}
      content } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala index e016377c94c0d..1a616a0434f2c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/util/RecurringTimer.scala @@ -77,7 +77,9 @@ class RecurringTimer(clock: Clock, period: Long, callback: (Long) => Unit, name: def stop(interruptTimer: Boolean): Long = synchronized { if (!stopped) { stopped = true - if (interruptTimer) thread.interrupt() + if (interruptTimer) { + thread.interrupt() + } thread.join() logInfo("Stopped timer for " + name + " after time " + prevTime) } diff --git a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java index a0b1bbc34fa7c..f9bfb9b7444cc 100644 --- a/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java +++ b/streaming/src/test/java/org/apache/spark/streaming/JavaAPISuite.java @@ -17,6 +17,7 @@ package org.apache.spark.streaming; +import org.apache.spark.streaming.api.java.*; import scala.Tuple2; import org.junit.Assert; @@ -36,10 +37,6 @@ import org.apache.spark.api.java.JavaSparkContext; import org.apache.spark.api.java.function.*; import org.apache.spark.storage.StorageLevel; -import org.apache.spark.streaming.api.java.JavaDStream; -import org.apache.spark.streaming.api.java.JavaDStreamLike; -import org.apache.spark.streaming.api.java.JavaPairDStream; -import org.apache.spark.streaming.api.java.JavaStreamingContext; // The test suite itself is Serializable so that anonymous Function implementations can be // serialized, as an alternative to converting these anonymous classes to static inner classes; @@ -1668,7 +1665,7 @@ public Integer call(String s) throws Exception { // InputStream functionality is deferred to the existing Scala tests. @Test public void testSocketTextStream() { - JavaDStream test = ssc.socketTextStream("localhost", 12345); + JavaReceiverInputDStream test = ssc.socketTextStream("localhost", 12345); } @Test @@ -1701,6 +1698,6 @@ public void testTextFileStream() { @Test public void testRawSocketStream() { - JavaDStream test = ssc.rawSocketStream("localhost", 12345); + JavaReceiverInputDStream test = ssc.rawSocketStream("localhost", 12345); } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 952511d411a8e..46b7f63b6567e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -36,10 +36,9 @@ import org.scalatest.BeforeAndAfter import org.apache.spark.Logging import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.NetworkReceiver -import org.apache.spark.streaming.receivers.Receiver import org.apache.spark.streaming.util.ManualClock import org.apache.spark.util.Utils +import org.apache.spark.streaming.receiver.{ActorHelper, Receiver} class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { @@ -207,7 +206,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { // set up the network stream using the test receiver val ssc = new StreamingContext(conf, batchDuration) - val networkStream = ssc.networkStream[Int](testReceiver) + val networkStream = ssc.receiverStream[Int](testReceiver) val countStream = networkStream.count val outputBuffer = new ArrayBuffer[Seq[Long]] with SynchronizedBuffer[Seq[Long]] val outputStream = new TestOutputStream(countStream, outputBuffer) @@ -301,7 +300,7 @@ object TestServer { } /** This is an actor for testing actor input stream */ -class TestActor(port: Int) extends Actor with Receiver { +class TestActor(port: Int) extends Actor with ActorHelper { def bytesToString(byteString: ByteString) = byteString.utf8String @@ -309,24 +308,22 @@ class TestActor(port: Int) extends Actor with Receiver { def receive = { case IO.Read(socket, bytes) => - pushBlock(bytesToString(bytes)) + store(bytesToString(bytes)) } } /** This is a receiver to test multiple threads inserting data using block generator */ class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int) - extends NetworkReceiver[Int] { + extends Receiver[Int](StorageLevel.MEMORY_ONLY_SER) with Logging { lazy val executorPool = Executors.newFixedThreadPool(numThreads) - lazy val blockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY) lazy val finishCount = new AtomicInteger(0) - protected def onStart() { - blockGenerator.start() + def onStart() { (1 to numThreads).map(threadId => { val runnable = new Runnable { def run() { (1 to numRecordsPerThread).foreach(i => - blockGenerator += (threadId * numRecordsPerThread + i) ) + store(threadId * numRecordsPerThread + i) ) if (finishCount.incrementAndGet == numThreads) { MultiThreadTestReceiver.haveAllThreadsFinished = true } @@ -337,7 +334,7 @@ class MultiThreadTestReceiver(numThreads: Int, numRecordsPerThread: Int) }) } - protected def onStop() { + def onStop() { executorPool.shutdown() } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala new file mode 100644 index 0000000000000..5c0415ad14ebb --- /dev/null +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -0,0 +1,249 @@ +/* + * 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.streaming + +import java.nio.ByteBuffer + +import scala.collection.mutable.ArrayBuffer + +import org.apache.spark.SparkConf +import org.apache.spark.storage.{StorageLevel, StreamBlockId} +import org.apache.spark.streaming.receiver.{BlockGenerator, BlockGeneratorListener, Receiver, ReceiverSupervisor} +import org.scalatest.FunSuite +import org.scalatest.concurrent.Timeouts +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ + +/** Testsuite for testing the network receiver behavior */ +class NetworkReceiverSuite extends FunSuite with Timeouts { + + test("network receiver life cycle") { + + val receiver = new FakeReceiver + val executor = new FakeReceiverSupervisor(receiver) + + assert(executor.isAllEmpty) + + // Thread that runs the executor + val executingThread = new Thread() { + override def run() { + executor.start() + executor.awaitTermination() + } + } + + // Start the receiver + executingThread.start() + + // Verify that the receiver + intercept[Exception] { + failAfter(200 millis) { + executingThread.join() + } + } + + // Verify that receiver was started + assert(receiver.onStartCalled) + assert(executor.isReceiverStarted) + assert(receiver.isStarted) + assert(!receiver.isStopped()) + assert(receiver.otherThread.isAlive) + eventually(timeout(100 millis), interval(10 millis)) { + assert(receiver.receiving) + } + + // Verify whether the data stored by the receiver was sent to the executor + val byteBuffer = ByteBuffer.allocate(100) + val arrayBuffer = new ArrayBuffer[Int]() + val iterator = arrayBuffer.iterator + receiver.store(1) + receiver.store(byteBuffer) + receiver.store(arrayBuffer) + receiver.store(iterator) + assert(executor.singles.size === 1) + assert(executor.singles.head === 1) + assert(executor.byteBuffers.size === 1) + assert(executor.byteBuffers.head.eq(byteBuffer)) + assert(executor.iterators.size === 1) + assert(executor.iterators.head.eq(iterator)) + assert(executor.arrayBuffers.size === 1) + assert(executor.arrayBuffers.head.eq(arrayBuffer)) + + // Verify whether the exceptions reported by the receiver was sent to the executor + val exception = new Exception + receiver.reportError("Error", exception) + assert(executor.errors.size === 1) + assert(executor.errors.head.eq(exception)) + + // Verify restarting actually stops and starts the receiver + receiver.restart("restarting", null, 100) + assert(receiver.isStopped) + assert(receiver.onStopCalled) + eventually(timeout(1000 millis), interval(100 millis)) { + assert(receiver.onStartCalled) + assert(executor.isReceiverStarted) + assert(receiver.isStarted) + assert(!receiver.isStopped) + assert(receiver.receiving) + } + + // Verify that stopping actually stops the thread + failAfter(100 millis) { + receiver.stop("test") + assert(receiver.isStopped) + assert(!receiver.otherThread.isAlive) + + // The thread that started the executor should complete + // as stop() stops everything + executingThread.join() + } + } + + test("block generator") { + val blockGeneratorListener = new FakeBlockGeneratorListener + val blockInterval = 200 + val conf = new SparkConf().set("spark.streaming.blockInterval", blockInterval.toString) + val blockGenerator = new BlockGenerator(blockGeneratorListener, 1, conf) + val expectedBlocks = 5 + val waitTime = expectedBlocks * blockInterval + (blockInterval / 2) + val generatedData = new ArrayBuffer[Int] + + // Generate blocks + val startTime = System.currentTimeMillis() + blockGenerator.start() + var count = 0 + while(System.currentTimeMillis - startTime < waitTime) { + blockGenerator += count + generatedData += count + count += 1 + Thread.sleep(10) + } + blockGenerator.stop() + + val recordedData = blockGeneratorListener.arrayBuffers.flatten + assert(blockGeneratorListener.arrayBuffers.size > 0) + assert(recordedData.toSet === generatedData.toSet) + } + + /** + * An implementation of NetworkReceiver that is used for testing a receiver's life cycle. + */ + class FakeReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) { + var otherThread: Thread = null + var receiving = false + var onStartCalled = false + var onStopCalled = false + + def onStart() { + otherThread = new Thread() { + override def run() { + receiving = true + while(!isStopped()) { + Thread.sleep(10) + } + } + } + onStartCalled = true + otherThread.start() + + } + + def onStop() { + onStopCalled = true + otherThread.join() + } + + def reset() { + receiving = false + onStartCalled = false + onStopCalled = false + } + } + + /** + * An implementation of NetworkReceiverExecutor used for testing a NetworkReceiver. + * Instead of storing the data in the BlockManager, it stores all the data in a local buffer + * that can used for verifying that the data has been forwarded correctly. + */ + class FakeReceiverSupervisor(receiver: FakeReceiver) + extends ReceiverSupervisor(receiver, new SparkConf()) { + val singles = new ArrayBuffer[Any] + val byteBuffers = new ArrayBuffer[ByteBuffer] + val iterators = new ArrayBuffer[Iterator[_]] + val arrayBuffers = new ArrayBuffer[ArrayBuffer[_]] + val errors = new ArrayBuffer[Throwable] + + /** Check if all data structures are clean */ + def isAllEmpty = { + singles.isEmpty && byteBuffers.isEmpty && iterators.isEmpty && + arrayBuffers.isEmpty && errors.isEmpty + } + + def pushSingle(data: Any) { + singles += data + } + + def pushBytes( + bytes: ByteBuffer, + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + byteBuffers += bytes + } + + def pushIterator( + iterator: Iterator[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + iterators += iterator + } + + def pushArrayBuffer( + arrayBuffer: ArrayBuffer[_], + optionalMetadata: Option[Any], + optionalBlockId: Option[StreamBlockId] + ) { + arrayBuffers += arrayBuffer + } + + def reportError(message: String, throwable: Throwable) { + errors += throwable + } + } + + /** + * An implementation of BlockGeneratorListener that is used to test the BlockGenerator. + */ + class FakeBlockGeneratorListener(pushDelay: Long = 0) extends BlockGeneratorListener { + // buffer of data received as ArrayBuffers + val arrayBuffers = new ArrayBuffer[ArrayBuffer[Int]] + val errors = new ArrayBuffer[Throwable] + + def onPushBlock(blockId: StreamBlockId, arrayBuffer: ArrayBuffer[_]) { + val bufferOfInts = arrayBuffer.map(_.asInstanceOf[Int]) + arrayBuffers += bufferOfInts + Thread.sleep(0) + } + + def onError(message: String, throwable: Throwable) { + errors += throwable + } + } +} + diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index ad5367ab941a2..6d14b1f785db3 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -21,7 +21,8 @@ import java.util.concurrent.atomic.AtomicInteger import org.apache.spark.{Logging, SparkConf, SparkContext, SparkException} import org.apache.spark.storage.StorageLevel -import org.apache.spark.streaming.dstream.{DStream, NetworkReceiver} +import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.receiver.Receiver import org.apache.spark.util.{MetadataCleaner, Utils} import org.scalatest.{BeforeAndAfter, FunSuite} import org.scalatest.concurrent.Timeouts @@ -181,15 +182,15 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w conf.set("spark.cleaner.ttl", "3600") sc = new SparkContext(conf) for (i <- 1 to 4) { - logInfo("==================================") - ssc = new StreamingContext(sc, batchDuration) + logInfo("==================================\n\n\n") + ssc = new StreamingContext(sc, Milliseconds(100)) var runningCount = 0 TestReceiver.counter.set(1) val input = ssc.networkStream(new TestReceiver) input.count.foreachRDD(rdd => { val count = rdd.first() - logInfo("Count = " + count) runningCount += count.toInt + logInfo("Count = " + count + ", Running count = " + runningCount) }) ssc.start() ssc.awaitTermination(500) @@ -216,12 +217,12 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc.start() } - // test whether waitForStop() exits after give amount of time + // test whether awaitTermination() exits after give amount of time failAfter(1000 millis) { ssc.awaitTermination(500) } - // test whether waitForStop() does not exit if not time is given + // test whether awaitTermination() does not exit if not time is given val exception = intercept[Exception] { failAfter(1000 millis) { ssc.awaitTermination() @@ -276,23 +277,26 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w class TestException(msg: String) extends Exception(msg) /** Custom receiver for testing whether all data received by a receiver gets processed or not */ -class TestReceiver extends NetworkReceiver[Int] { - protected lazy val blockGenerator = new BlockGenerator(StorageLevel.MEMORY_ONLY) - protected def onStart() { - blockGenerator.start() - logInfo("BlockGenerator started on thread " + receivingThread) - try { - while(true) { - blockGenerator += TestReceiver.counter.getAndIncrement - Thread.sleep(0) +class TestReceiver extends Receiver[Int](StorageLevel.MEMORY_ONLY) with Logging { + + var receivingThreadOption: Option[Thread] = None + + def onStart() { + val thread = new Thread() { + override def run() { + logInfo("Receiving started") + while (!isStopped) { + store(TestReceiver.counter.getAndIncrement) + } + logInfo("Receiving stopped at count value of " + TestReceiver.counter.get()) } - } finally { - logInfo("Receiving stopped at count value of " + TestReceiver.counter.get()) } + receivingThreadOption = Some(thread) + thread.start() } - protected def onStop() { - blockGenerator.stop() + def onStop() { + // no cleanup to be done, the receiving thread should stop on it own } } diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 9e0f2c900e8ba..542c697ae3127 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -17,10 +17,19 @@ package org.apache.spark.streaming -import org.apache.spark.streaming.scheduler._ import scala.collection.mutable.ArrayBuffer -import org.scalatest.matchers.ShouldMatchers +import scala.concurrent.Future +import scala.concurrent.ExecutionContext.Implicits.global + +import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream +import org.apache.spark.streaming.receiver.Receiver +import org.apache.spark.streaming.scheduler._ + +import org.scalatest.matchers.ShouldMatchers +import org.scalatest.concurrent.Eventually._ +import org.scalatest.time.SpanSugar._ +import org.apache.spark.Logging class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { @@ -32,7 +41,7 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { override def batchDuration = Milliseconds(100) override def actuallyWait = true - test("basic BatchInfo generation") { + test("batch info reporting") { val ssc = setupStreams(input, operation) val collector = new BatchInfoCollector ssc.addStreamingListener(collector) @@ -54,6 +63,31 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { isInIncreasingOrder(batchInfos.map(_.processingEndTime.get)) should be (true) } + test("receiver info reporting") { + val ssc = new StreamingContext("local[2]", "test", Milliseconds(1000)) + val inputStream = ssc.networkStream(new StreamingListenerSuiteReceiver) + inputStream.foreachRDD(_.count) + + val collector = new ReceiverInfoCollector + ssc.addStreamingListener(collector) + + ssc.start() + try { + eventually(timeout(1000 millis), interval(20 millis)) { + collector.startedReceiverInfo should have size 1 + collector.startedReceiverInfo(0).streamId should equal (0) + collector.stoppedReceiverStreamIds should have size 1 + collector.stoppedReceiverStreamIds(0) should equal (0) + collector.receiverErrors should have size 1 + collector.receiverErrors(0)._1 should equal (0) + collector.receiverErrors(0)._2 should include ("report error") + collector.receiverErrors(0)._3 should include ("report exception") + } + } finally { + ssc.stop() + } + } + /** Check if a sequence of numbers is in increasing order */ def isInIncreasingOrder(seq: Seq[Long]): Boolean = { for(i <- 1 until seq.size) { @@ -61,12 +95,46 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { } true } +} + +/** Listener that collects information on processed batches */ +class BatchInfoCollector extends StreamingListener { + val batchInfos = new ArrayBuffer[BatchInfo] + override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { + batchInfos += batchCompleted.batchInfo + } +} + +/** Listener that collects information on processed batches */ +class ReceiverInfoCollector extends StreamingListener { + val startedReceiverInfo = new ArrayBuffer[ReceiverInfo] + val stoppedReceiverStreamIds = new ArrayBuffer[Int]() + val receiverErrors = new ArrayBuffer[(Int, String, String)]() + + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { + startedReceiverInfo += receiverStarted.receiverInfo + } + + override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { + stoppedReceiverStreamIds += receiverStopped.streamId + } + + override def onReceiverError(receiverError: StreamingListenerReceiverError) { + receiverErrors += ((receiverError.streamId, receiverError.message, receiverError.error)) + } +} - /** Listener that collects information on processed batches */ - class BatchInfoCollector extends StreamingListener { - val batchInfos = new ArrayBuffer[BatchInfo] - override def onBatchCompleted(batchCompleted: StreamingListenerBatchCompleted) { - batchInfos += batchCompleted.batchInfo +class StreamingListenerSuiteReceiver extends Receiver[Any](StorageLevel.MEMORY_ONLY) with Logging { + def onStart() { + Future { + logInfo("Started receiver and sleeping") + Thread.sleep(10) + logInfo("Reporting error and sleeping") + reportError("test report error", new Exception("test report exception")) + Thread.sleep(10) + logInfo("Stopping") + stop("test stop error") } } + def onStop() { } } From fc7838470465474f777bd17791c1bb5f9c348521 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Mon, 21 Apr 2014 21:57:40 -0700 Subject: [PATCH 328/397] [SPARK-1439, SPARK-1440] Generate unified Scaladoc across projects and Javadocs I used the sbt-unidoc plugin (https://github.com/sbt/sbt-unidoc) to create a unified Scaladoc of our public packages, and generate Javadocs as well. One limitation is that I haven't found an easy way to exclude packages in the Javadoc; there is a SBT task that identifies Java sources to run javadoc on, but it's been very difficult to modify it from outside to change what is set in the unidoc package. Some SBT-savvy people should help with this. The Javadoc site also lacks package-level descriptions and things like that, so we may want to look into that. We may decide not to post these right now if it's too limited compared to the Scala one. Example of the built doc site: http://people.csail.mit.edu/matei/spark-unified-docs/ Author: Matei Zaharia This patch had conflicts when merged, resolved by Committer: Patrick Wendell Closes #457 from mateiz/better-docs and squashes the following commits: a63d4a3 [Matei Zaharia] Skip Java/Scala API docs for Python package 5ea1f43 [Matei Zaharia] Fix links to Java classes in Java guide, fix some JS for scrolling to anchors on page load f05abc0 [Matei Zaharia] Don't include java.lang package names 995e992 [Matei Zaharia] Skip internal packages and class names with $ in JavaDoc a14a93c [Matei Zaharia] typo 76ce64d [Matei Zaharia] Add groups to Javadoc index page, and a first package-info.java ed6f994 [Matei Zaharia] Generate JavaDoc as well, add titles, update doc site to use unified docs acb993d [Matei Zaharia] Add Unidoc plugin for the projects we want Unidoced --- .../java/org/apache/spark/package-info.java | 23 ++++ .../main/scala/org/apache/spark/package.scala | 2 +- docs/_layouts/global.html | 105 +++--------------- docs/_plugins/copy_api_dirs.rb | 65 +++++------ docs/api.md | 13 +-- docs/configuration.md | 8 +- docs/graphx-programming-guide.md | 62 +++++------ docs/index.md | 10 +- docs/java-programming-guide.md | 55 ++++----- docs/js/main.js | 18 +++ docs/mllib-classification-regression.md | 14 +-- docs/mllib-clustering.md | 2 +- docs/mllib-collaborative-filtering.md | 2 +- docs/mllib-guide.md | 10 +- docs/mllib-optimization.md | 8 +- docs/python-programming-guide.md | 4 +- docs/quick-start.md | 6 +- docs/scala-programming-guide.md | 10 +- docs/sql-programming-guide.md | 20 ++-- docs/streaming-custom-receivers.md | 4 +- docs/streaming-programming-guide.md | 56 +++++----- docs/tuning.md | 4 +- project/SparkBuild.scala | 74 ++++++++++-- project/plugins.sbt | 1 + python/epydoc.conf | 4 +- 25 files changed, 296 insertions(+), 284 deletions(-) create mode 100644 core/src/main/java/org/apache/spark/package-info.java diff --git a/core/src/main/java/org/apache/spark/package-info.java b/core/src/main/java/org/apache/spark/package-info.java new file mode 100644 index 0000000000000..4426c7afcebdd --- /dev/null +++ b/core/src/main/java/org/apache/spark/package-info.java @@ -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. + */ + +/** + * Core Spark classes in Scala. A few classes here, such as {@link org.apache.spark.Accumulator} + * and {@link org.apache.spark.storage.StorageLevel}, are also used in Java, but the + * {@link org.apache.spark.api.java} package contains the main Java API. + */ +package org.apache.spark; diff --git a/core/src/main/scala/org/apache/spark/package.scala b/core/src/main/scala/org/apache/spark/package.scala index 59bbb1171f239..5cdbc306e56a0 100644 --- a/core/src/main/scala/org/apache/spark/package.scala +++ b/core/src/main/scala/org/apache/spark/package.scala @@ -30,7 +30,7 @@ package org.apache * type (e.g. RDD[(Int, Int)] through implicit conversions when you * `import org.apache.spark.SparkContext._`. * - * Java programmers should reference the [[spark.api.java]] package + * Java programmers should reference the [[org.apache.spark.api.java]] package * for Spark programming APIs in Java. * * Classes and methods marked with diff --git a/docs/_layouts/global.html b/docs/_layouts/global.html index 5d4dbb7a9c3a9..8b543de574622 100755 --- a/docs/_layouts/global.html +++ b/docs/_layouts/global.html @@ -76,32 +76,9 @@
    • @@ -140,33 +117,6 @@

      {{ page.title }}

      {{ content }} - - - - - @@ -174,42 +124,23 @@

      Heading

      - + + + - - - - - diff --git a/docs/_plugins/copy_api_dirs.rb b/docs/_plugins/copy_api_dirs.rb index 05f0bd47a88a5..2dbbbf6feb4b8 100644 --- a/docs/_plugins/copy_api_dirs.rb +++ b/docs/_plugins/copy_api_dirs.rb @@ -20,47 +20,48 @@ if not (ENV['SKIP_API'] == '1' or ENV['SKIP_SCALADOC'] == '1') # Build Scaladoc for Java/Scala - core_projects = ["core", "examples", "repl", "bagel", "graphx", "streaming", "mllib"] - external_projects = ["flume", "kafka", "mqtt", "twitter", "zeromq"] - sql_projects = ["catalyst", "core", "hive"] - projects = core_projects - projects = projects + external_projects.map { |project_name| "external/" + project_name } - projects = projects + sql_projects.map { |project_name| "sql/" + project_name } - - puts "Moving to project root and building scaladoc." + puts "Moving to project root and building API docs." curr_dir = pwd cd("..") - puts "Running 'sbt/sbt doc hive/doc' from " + pwd + "; this may take a few minutes..." - puts `sbt/sbt doc hive/doc` + puts "Running 'sbt/sbt compile unidoc' from " + pwd + "; this may take a few minutes..." + puts `sbt/sbt compile unidoc` puts "Moving back into docs dir." cd("docs") - # Copy over the scaladoc from each project into the docs directory. + # Copy over the unified ScalaDoc for all projects to api/scala. # This directory will be copied over to _site when `jekyll` command is run. - projects.each do |project_name| - source = "../" + project_name + "/target/scala-2.10/api" - dest = "api/" + project_name + source = "../target/scala-2.10/unidoc" + dest = "api/scala" + + puts "Making directory " + dest + mkdir_p dest + + # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. + puts "cp -r " + source + "/. " + dest + cp_r(source + "/.", dest) + + # Append custom JavaScript + js = File.readlines("./js/api-docs.js") + js_file = dest + "/lib/template.js" + File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } - puts "making directory " + dest - mkdir_p dest + # Append custom CSS + css = File.readlines("./css/api-docs.css") + css_file = dest + "/lib/template.css" + File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } - # From the rubydoc: cp_r('src', 'dest') makes src/dest, but this doesn't. - puts "cp -r " + source + "/. " + dest - cp_r(source + "/.", dest) + # Copy over the unified JavaDoc for all projects to api/java. + source = "../target/javaunidoc" + dest = "api/java" - # Append custom JavaScript - js = File.readlines("./js/api-docs.js") - js_file = dest + "/lib/template.js" - File.open(js_file, 'a') { |f| f.write("\n" + js.join()) } + puts "Making directory " + dest + mkdir_p dest - # Append custom CSS - css = File.readlines("./css/api-docs.css") - css_file = dest + "/lib/template.css" - File.open(css_file, 'a') { |f| f.write("\n" + css.join()) } - end + puts "cp -r " + source + "/. " + dest + cp_r(source + "/.", dest) # Build Epydoc for Python puts "Moving to python directory and building epydoc." @@ -70,11 +71,11 @@ puts "Moving back into docs dir." cd("../docs") - puts "echo making directory pyspark" - mkdir_p "pyspark" + puts "Making directory api/python" + mkdir_p "api/python" - puts "cp -r ../python/docs/. api/pyspark" - cp_r("../python/docs/.", "api/pyspark") + puts "cp -r ../python/docs/. api/python" + cp_r("../python/docs/.", "api/python") cd("..") end diff --git a/docs/api.md b/docs/api.md index 91c8e51d26c15..03460383335e8 100644 --- a/docs/api.md +++ b/docs/api.md @@ -1,13 +1,10 @@ --- layout: global -title: Spark API documentation (Scaladoc) +title: Spark API Documentation --- -Here you can find links to the Scaladoc generated for the Spark sbt subprojects. If the following links don't work, try running `sbt/sbt doc` from the Spark project home directory. +Here you can API docs for Spark and its submodules. -- [Spark](api/core/index.html) -- [Spark Examples](api/examples/index.html) -- [Spark Streaming](api/streaming/index.html) -- [Bagel](api/bagel/index.html) -- [GraphX](api/graphx/index.html) -- [PySpark](api/pyspark/index.html) +- [Spark Scala API (Scaladoc)](api/scala/index.html) +- [Spark Java API (Javadoc)](api/java/index.html) +- [Spark Python API (Epydoc)](api/python/index.html) diff --git a/docs/configuration.md b/docs/configuration.md index 5a4abca2646b6..e7e1dd56cf124 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -6,7 +6,7 @@ title: Spark Configuration Spark provides three locations to configure the system: * [Spark properties](#spark-properties) control most application parameters and can be set by passing - a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java + a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) object to SparkContext, or through Java system properties. * [Environment variables](#environment-variables) can be used to set per-machine settings, such as the IP address, through the `conf/spark-env.sh` script on each node. @@ -16,7 +16,7 @@ Spark provides three locations to configure the system: # Spark Properties Spark properties control most application settings and are configured separately for each application. -The preferred way to set them is by passing a [SparkConf](api/core/index.html#org.apache.spark.SparkConf) +The preferred way to set them is by passing a [SparkConf](api/scala/index.html#org.apache.spark.SparkConf) class to your SparkContext constructor. Alternatively, Spark will also load them from Java system properties, for compatibility with old versions of Spark. @@ -53,7 +53,7 @@ there are at least five properties that you will commonly want to control: in serialized form. The default of Java serialization works with any Serializable Java object but is quite slow, so we recommend using org.apache.spark.serializer.KryoSerializer and configuring Kryo serialization when speed is necessary. Can be any subclass of - org.apache.spark.Serializer. + org.apache.spark.Serializer.
      @@ -62,7 +62,7 @@ there are at least five properties that you will commonly want to control: diff --git a/docs/graphx-programming-guide.md b/docs/graphx-programming-guide.md index 1238e3e0a4e7d..07be8ba58efa3 100644 --- a/docs/graphx-programming-guide.md +++ b/docs/graphx-programming-guide.md @@ -17,7 +17,7 @@ title: GraphX Programming Guide # Overview GraphX is the new (alpha) Spark API for graphs and graph-parallel computation. At a high-level, -GraphX extends the Spark [RDD](api/core/index.html#org.apache.spark.rdd.RDD) by introducing the +GraphX extends the Spark [RDD](api/scala/index.html#org.apache.spark.rdd.RDD) by introducing the [Resilient Distributed Property Graph](#property_graph): a directed multigraph with properties attached to each vertex and edge. To support graph computation, GraphX exposes a set of fundamental operators (e.g., [subgraph](#structural_operators), [joinVertices](#join_operators), and @@ -82,7 +82,7 @@ Prior to the release of GraphX, graph computation in Spark was expressed using B implementation of Pregel. GraphX improves upon Bagel by exposing a richer property graph API, a more streamlined version of the Pregel abstraction, and system optimizations to improve performance and reduce memory overhead. While we plan to eventually deprecate Bagel, we will continue to -support the [Bagel API](api/bagel/index.html#org.apache.spark.bagel.package) and +support the [Bagel API](api/scala/index.html#org.apache.spark.bagel.package) and [Bagel programming guide](bagel-programming-guide.html). However, we encourage Bagel users to explore the new GraphX API and comment on issues that may complicate the transition from Bagel. @@ -103,7 +103,7 @@ getting started with Spark refer to the [Spark Quick Start Guide](quick-start.ht # The Property Graph -The [property graph](api/graphx/index.html#org.apache.spark.graphx.Graph) is a directed multigraph +The [property graph](api/scala/index.html#org.apache.spark.graphx.Graph) is a directed multigraph with user defined objects attached to each vertex and edge. A directed multigraph is a directed graph with potentially multiple parallel edges sharing the same source and destination vertex. The ability to support parallel edges simplifies modeling scenarios where there can be multiple @@ -179,7 +179,7 @@ val userGraph: Graph[(String, String), String] There are numerous ways to construct a property graph from raw files, RDDs, and even synthetic generators and these are discussed in more detail in the section on [graph builders](#graph_builders). Probably the most general method is to use the -[Graph object](api/graphx/index.html#org.apache.spark.graphx.Graph$). For example the following +[Graph object](api/scala/index.html#org.apache.spark.graphx.Graph$). For example the following code constructs a graph from a collection of RDDs: {% highlight scala %} @@ -203,7 +203,7 @@ In the above example we make use of the [`Edge`][Edge] case class. Edges have a `dstId` corresponding to the source and destination vertex identifiers. In addition, the `Edge` class has an `attr` member which stores the edge property. -[Edge]: api/graphx/index.html#org.apache.spark.graphx.Edge +[Edge]: api/scala/index.html#org.apache.spark.graphx.Edge We can deconstruct a graph into the respective vertex and edge views by using the `graph.vertices` and `graph.edges` members respectively. @@ -229,7 +229,7 @@ The triplet view logically joins the vertex and edge properties yielding an `RDD[EdgeTriplet[VD, ED]]` containing instances of the [`EdgeTriplet`][EdgeTriplet] class. This *join* can be expressed in the following SQL expression: -[EdgeTriplet]: api/graphx/index.html#org.apache.spark.graphx.EdgeTriplet +[EdgeTriplet]: api/scala/index.html#org.apache.spark.graphx.EdgeTriplet {% highlight sql %} SELECT src.id, dst.id, src.attr, e.attr, dst.attr @@ -270,8 +270,8 @@ core operators are defined in [`GraphOps`][GraphOps]. However, thanks to Scala operators in `GraphOps` are automatically available as members of `Graph`. For example, we can compute the in-degree of each vertex (defined in `GraphOps`) by the following: -[Graph]: api/graphx/index.html#org.apache.spark.graphx.Graph -[GraphOps]: api/graphx/index.html#org.apache.spark.graphx.GraphOps +[Graph]: api/scala/index.html#org.apache.spark.graphx.Graph +[GraphOps]: api/scala/index.html#org.apache.spark.graphx.GraphOps {% highlight scala %} val graph: Graph[(String, String), String] @@ -382,7 +382,7 @@ val newGraph = Graph(newVertices, graph.edges) val newGraph = graph.mapVertices((id, attr) => mapUdf(id, attr)) {% endhighlight %} -[Graph.mapVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexId,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED] +[Graph.mapVertices]: api/scala/index.html#org.apache.spark.graphx.Graph@mapVertices[VD2]((VertexId,VD)⇒VD2)(ClassTag[VD2]):Graph[VD2,ED] These operators are often used to initialize the graph for a particular computation or project away unnecessary properties. For example, given a graph with the out-degrees as the vertex properties @@ -419,7 +419,7 @@ This can be useful when, for example, trying to compute the inverse PageRank. B operation does not modify vertex or edge properties or change the number of edges, it can be implemented efficiently without data-movement or duplication. -[Graph.reverse]: api/graphx/index.html#org.apache.spark.graphx.Graph@reverse:Graph[VD,ED] +[Graph.reverse]: api/scala/index.html#org.apache.spark.graphx.Graph@reverse:Graph[VD,ED] The [`subgraph`][Graph.subgraph] operator takes vertex and edge predicates and returns the graph containing only the vertices that satisfy the vertex predicate (evaluate to true) and edges that @@ -427,7 +427,7 @@ satisfy the edge predicate *and connect vertices that satisfy the vertex predica operator can be used in number of situations to restrict the graph to the vertices and edges of interest or eliminate broken links. For example in the following code we remove broken links: -[Graph.subgraph]: api/graphx/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexId,VD)⇒Boolean):Graph[VD,ED] +[Graph.subgraph]: api/scala/index.html#org.apache.spark.graphx.Graph@subgraph((EdgeTriplet[VD,ED])⇒Boolean,(VertexId,VD)⇒Boolean):Graph[VD,ED] {% highlight scala %} // Create an RDD for the vertices @@ -467,7 +467,7 @@ vertices and edges that are also found in the input graph. This can be used in example, we might run connected components using the graph with missing vertices and then restrict the answer to the valid subgraph. -[Graph.mask]: api/graphx/index.html#org.apache.spark.graphx.Graph@mask[VD2,ED2](Graph[VD2,ED2])(ClassTag[VD2],ClassTag[ED2]):Graph[VD,ED] +[Graph.mask]: api/scala/index.html#org.apache.spark.graphx.Graph@mask[VD2,ED2](Graph[VD2,ED2])(ClassTag[VD2],ClassTag[ED2]):Graph[VD,ED] {% highlight scala %} // Run Connected Components @@ -482,7 +482,7 @@ The [`groupEdges`][Graph.groupEdges] operator merges parallel edges (i.e., dupli pairs of vertices) in the multigraph. In many numerical applications, parallel edges can be *added* (their weights combined) into a single edge thereby reducing the size of the graph. -[Graph.groupEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph@groupEdges((ED,ED)⇒ED):Graph[VD,ED] +[Graph.groupEdges]: api/scala/index.html#org.apache.spark.graphx.Graph@groupEdges((ED,ED)⇒ED):Graph[VD,ED] ## Join Operators @@ -506,7 +506,7 @@ returns a new graph with the vertex properties obtained by applying the user def to the result of the joined vertices. Vertices without a matching value in the RDD retain their original value. -[GraphOps.joinVertices]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexId,U)])((VertexId,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED] +[GraphOps.joinVertices]: api/scala/index.html#org.apache.spark.graphx.GraphOps@joinVertices[U](RDD[(VertexId,U)])((VertexId,VD,U)⇒VD)(ClassTag[U]):Graph[VD,ED] > Note that if the RDD contains more than one value for a given vertex only one will be used. It > is therefore recommended that the input RDD be first made unique using the following which will @@ -525,7 +525,7 @@ property type. Because not all vertices may have a matching value in the input function takes an `Option` type. For example, we can setup a graph for PageRank by initializing vertex properties with their `outDegree`. -[Graph.outerJoinVertices]: api/graphx/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexId,U)])((VertexId,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED] +[Graph.outerJoinVertices]: api/scala/index.html#org.apache.spark.graphx.Graph@outerJoinVertices[U,VD2](RDD[(VertexId,U)])((VertexId,VD,Option[U])⇒VD2)(ClassTag[U],ClassTag[VD2]):Graph[VD2,ED] {% highlight scala %} @@ -559,7 +559,7 @@ PageRank Value, shortest path to the source, and smallest reachable vertex id). ### Map Reduce Triplets (mapReduceTriplets) -[Graph.mapReduceTriplets]: api/graphx/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=>Iterator[(org.apache.spark.graphx.VertexId,A)],reduceFunc:(A,A)=>A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A] +[Graph.mapReduceTriplets]: api/scala/index.html#org.apache.spark.graphx.Graph@mapReduceTriplets[A](mapFunc:org.apache.spark.graphx.EdgeTriplet[VD,ED]=>Iterator[(org.apache.spark.graphx.VertexId,A)],reduceFunc:(A,A)=>A,activeSetOpt:Option[(org.apache.spark.graphx.VertexRDD[_],org.apache.spark.graphx.EdgeDirection)])(implicitevidence$10:scala.reflect.ClassTag[A]):org.apache.spark.graphx.VertexRDD[A] The core (heavily optimized) aggregation primitive in GraphX is the [`mapReduceTriplets`][Graph.mapReduceTriplets] operator: @@ -665,8 +665,8 @@ attributes at each vertex. This can be easily accomplished using the [`collectNeighborIds`][GraphOps.collectNeighborIds] and the [`collectNeighbors`][GraphOps.collectNeighbors] operators. -[GraphOps.collectNeighborIds]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexId]] -[GraphOps.collectNeighbors]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexId,VD)]] +[GraphOps.collectNeighborIds]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighborIds(EdgeDirection):VertexRDD[Array[VertexId]] +[GraphOps.collectNeighbors]: api/scala/index.html#org.apache.spark.graphx.GraphOps@collectNeighbors(EdgeDirection):VertexRDD[Array[(VertexId,VD)]] {% highlight scala %} @@ -685,7 +685,7 @@ class GraphOps[VD, ED] { In Spark, RDDs are not persisted in memory by default. To avoid recomputation, they must be explicitly cached when using them multiple times (see the [Spark Programming Guide][RDD Persistence]). Graphs in GraphX behave the same way. **When using a graph multiple times, make sure to call [`Graph.cache()`][Graph.cache] on it first.** [RDD Persistence]: scala-programming-guide.html#rdd-persistence -[Graph.cache]: api/graphx/index.html#org.apache.spark.graphx.Graph@cache():Graph[VD,ED] +[Graph.cache]: api/scala/index.html#org.apache.spark.graphx.Graph@cache():Graph[VD,ED] In iterative computations, *uncaching* may also be necessary for best performance. By default, cached RDDs and graphs will remain in memory until memory pressure forces them to be evicted in LRU order. For iterative computation, intermediate results from previous iterations will fill up the cache. Though they will eventually be evicted, the unnecessary data stored in memory will slow down garbage collection. It would be more efficient to uncache intermediate results as soon as they are no longer necessary. This involves materializing (caching and forcing) a graph or RDD every iteration, uncaching all other datasets, and only using the materialized dataset in future iterations. However, because graphs are composed of multiple RDDs, it can be difficult to unpersist them correctly. **For iterative computation we recommend using the Pregel API, which correctly unpersists intermediate results.** @@ -716,7 +716,7 @@ messages remaining. The following is the type signature of the [Pregel operator][GraphOps.pregel] as well as a *sketch* of its implementation (note calls to graph.cache have been removed): -[GraphOps.pregel]: api/graphx/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexId,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexId,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED] +[GraphOps.pregel]: api/scala/index.html#org.apache.spark.graphx.GraphOps@pregel[A](A,Int,EdgeDirection)((VertexId,VD,A)⇒VD,(EdgeTriplet[VD,ED])⇒Iterator[(VertexId,A)],(A,A)⇒A)(ClassTag[A]):Graph[VD,ED] {% highlight scala %} class GraphOps[VD, ED] { @@ -840,12 +840,12 @@ object Graph { [`Graph.fromEdgeTuples`][Graph.fromEdgeTuples] allows creating a graph from only an RDD of edge tuples, assigning the edges the value 1, and automatically creating any vertices mentioned by edges and assigning them the default value. It also supports deduplicating the edges; to deduplicate, pass `Some` of a [`PartitionStrategy`][PartitionStrategy] as the `uniqueEdges` parameter (for example, `uniqueEdges = Some(PartitionStrategy.RandomVertexCut)`). A partition strategy is necessary to colocate identical edges on the same partition so they can be deduplicated. -[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy$ +[PartitionStrategy]: api/scala/index.html#org.apache.spark.graphx.PartitionStrategy$ -[GraphLoader.edgeListFile]: api/graphx/index.html#org.apache.spark.graphx.GraphLoader$@edgeListFile(SparkContext,String,Boolean,Int):Graph[Int,Int] -[Graph.apply]: api/graphx/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexId,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] -[Graph.fromEdgeTuples]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexId,VertexId)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int] -[Graph.fromEdges]: api/graphx/index.html#org.apache.spark.graphx.Graph$@fromEdges[VD,ED](RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] +[GraphLoader.edgeListFile]: api/scala/index.html#org.apache.spark.graphx.GraphLoader$@edgeListFile(SparkContext,String,Boolean,Int):Graph[Int,Int] +[Graph.apply]: api/scala/index.html#org.apache.spark.graphx.Graph$@apply[VD,ED](RDD[(VertexId,VD)],RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] +[Graph.fromEdgeTuples]: api/scala/index.html#org.apache.spark.graphx.Graph$@fromEdgeTuples[VD](RDD[(VertexId,VertexId)],VD,Option[PartitionStrategy])(ClassTag[VD]):Graph[VD,Int] +[Graph.fromEdges]: api/scala/index.html#org.apache.spark.graphx.Graph$@fromEdges[VD,ED](RDD[Edge[ED]],VD)(ClassTag[VD],ClassTag[ED]):Graph[VD,ED] # Vertex and Edge RDDs @@ -913,7 +913,7 @@ of the various partitioning strategies defined in [`PartitionStrategy`][Partitio each partition, edge attributes and adjacency structure, are stored separately enabling maximum reuse when changing attribute values. -[PartitionStrategy]: api/graphx/index.html#org.apache.spark.graphx.PartitionStrategy +[PartitionStrategy]: api/scala/index.html#org.apache.spark.graphx.PartitionStrategy The three additional functions exposed by the `EdgeRDD` are: {% highlight scala %} @@ -952,7 +952,7 @@ the [`Graph.partitionBy`][Graph.partitionBy] operator. The default partitioning the initial partitioning of the edges as provided on graph construction. However, users can easily switch to 2D-partitioning or other heuristics included in GraphX. -[Graph.partitionBy]: api/graphx/index.html#org.apache.spark.graphx.Graph$@partitionBy(partitionStrategy:org.apache.spark.graphx.PartitionStrategy):org.apache.spark.graphx.Graph[VD,ED] +[Graph.partitionBy]: api/scala/index.html#org.apache.spark.graphx.Graph$@partitionBy(partitionStrategy:org.apache.spark.graphx.PartitionStrategy):org.apache.spark.graphx.Graph[VD,ED]

      @@ -81,7 +81,7 @@ interface has a single abstract method, `call()`, that must be implemented. ## Storage Levels RDD [storage level](scala-programming-guide.html#rdd-persistence) constants, such as `MEMORY_AND_DISK`, are -declared in the [org.apache.spark.api.java.StorageLevels](api/core/index.html#org.apache.spark.api.java.StorageLevels) class. To +declared in the [org.apache.spark.api.java.StorageLevels](api/java/index.html?org/apache/spark/api/java/StorageLevels.html) class. To define your own storage level, you can use StorageLevels.create(...). # Other Features @@ -101,11 +101,11 @@ the following changes: classes to interfaces. This means that concrete implementations of these `Function` classes will need to use `implements` rather than `extends`. * Certain transformation functions now have multiple versions depending - on the return type. In Spark core, the map functions (map, flatMap, - mapPartitons) have type-specific versions, e.g. - [`mapToPair`](api/core/index.html#org.apache.spark.api.java.JavaRDD@mapToPair[K2,V2](f:org.apache.spark.api.java.function.PairFunction[T,K2,V2]):org.apache.spark.api.java.JavaPairRDD[K2,V2]) - and [`mapToDouble`](api/core/index.html#org.apache.spark.api.java.JavaRDD@mapToDouble[R](f:org.apache.spark.api.java.function.DoubleFunction[T]):org.apache.spark.api.java.JavaDoubleRDD). - Spark Streaming also uses the same approach, e.g. [`transformToPair`](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaDStream@transformToPair[K2,V2](transformFunc:org.apache.spark.api.java.function.Function[R,org.apache.spark.api.java.JavaPairRDD[K2,V2]]):org.apache.spark.streaming.api.java.JavaPairDStream[K2,V2]). + on the return type. In Spark core, the map functions (`map`, `flatMap`, and + `mapPartitons`) have type-specific versions, e.g. + [`mapToPair`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToPair(org.apache.spark.api.java.function.PairFunction)) + and [`mapToDouble`](api/java/org/apache/spark/api/java/JavaRDDLike.html#mapToDouble(org.apache.spark.api.java.function.DoubleFunction)). + Spark Streaming also uses the same approach, e.g. [`transformToPair`](api/java/org/apache/spark/streaming/api/java/JavaDStreamLike.html#transformToPair(org.apache.spark.api.java.function.Function)). # Example @@ -205,16 +205,9 @@ JavaPairRDD counts = lines.flatMapToPair( There is no performance difference between these approaches; the choice is just a matter of style. -# Javadoc - -We currently provide documentation for the Java API as Scaladoc, in the -[`org.apache.spark.api.java` package](api/core/index.html#org.apache.spark.api.java.package), because -some of the classes are implemented in Scala. It is important to note that the types and function -definitions show Scala syntax (for example, `def reduce(func: Function2[T, T]): T` instead of -`T reduce(Function2 func)`). In addition, the Scala `trait` modifier is used for Java -interface classes. We hope to generate documentation with Java-style syntax in the future to -avoid these quirks. +# API Docs +[API documentation](api/java/index.html) for Spark in Java is available in Javadoc format. # Where to Go from Here diff --git a/docs/js/main.js b/docs/js/main.js index 0bd2286cced19..59055467110bf 100755 --- a/docs/js/main.js +++ b/docs/js/main.js @@ -73,8 +73,26 @@ function viewSolution() { }); } +// A script to fix internal hash links because we have an overlapping top bar. +// Based on https://github.com/twitter/bootstrap/issues/193#issuecomment-2281510 +function maybeScrollToHash() { + console.log("HERE"); + if (window.location.hash && $(window.location.hash).length) { + console.log("HERE2", $(window.location.hash), $(window.location.hash).offset().top); + var newTop = $(window.location.hash).offset().top - 57; + $(window).scrollTop(newTop); + } +} $(function() { codeTabs(); viewSolution(); + + $(window).bind('hashchange', function() { + maybeScrollToHash(); + }); + + // Scroll now too in case we had opened the page on a hash, but wait a bit because some browsers + // will try to do *their* initial scroll after running the onReady handler. + $(window).load(function() { setTimeout(function() { maybeScrollToHash(); }, 25); }); }); diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md index 2c42f60c2ecce..2e0fa093dccaa 100644 --- a/docs/mllib-classification-regression.md +++ b/docs/mllib-classification-regression.md @@ -316,26 +316,26 @@ For each of them, we support all 3 possible regularizations (none, L1 or L2). Available algorithms for binary classification: -* [SVMWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.SVMWithSGD) -* [LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) +* [SVMWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.SVMWithSGD) +* [LogisticRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) Available algorithms for linear regression: -* [LinearRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) -* [RidgeRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -* [LassoWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD) +* [LinearRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) +* [RidgeRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +* [LassoWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) Behind the scenes, all above methods use the SGD implementation from the gradient descent primitive in MLlib, see the optimization part: -* [GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) +* [GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) #### Tree-based Methods The decision tree algorithm supports binary classification and regression: -* [DecisionTee](api/mllib/index.html#org.apache.spark.mllib.tree.DecisionTree) +* [DecisionTee](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) # Usage in Scala diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 50a8671560737..0359c67157168 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -33,7 +33,7 @@ a given dataset, the algorithm returns the best clustering result). Available algorithms for clustering: -* [KMeans](api/mllib/index.html#org.apache.spark.mllib.clustering.KMeans) +* [KMeans](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index aa22f67b303ce..2f1f5f3856efe 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -42,7 +42,7 @@ for an item. Available algorithms for collaborative filtering: -* [ALS](api/mllib/index.html#org.apache.spark.mllib.recommendation.ALS) +* [ALS](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS) # Usage in Scala diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 4236b0c8b6c99..0963a99881c9d 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -36,15 +36,15 @@ The following links provide a detailed explanation of the methods and usage exam # Data Types Most MLlib algorithms operate on RDDs containing vectors. In Java and Scala, the -[Vector](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) class is used to +[Vector](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) class is used to represent vectors. You can create either dense or sparse vectors using the -[Vectors](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) factory. +[Vectors](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) factory. In Python, MLlib can take the following vector types: * [NumPy](http://www.numpy.org) arrays * Standard Python lists (e.g. `[1, 2, 3]`) -* The MLlib [SparseVector](api/pyspark/pyspark.mllib.linalg.SparseVector-class.html) class +* The MLlib [SparseVector](api/python/pyspark.mllib.linalg.SparseVector-class.html) class * [SciPy sparse matrices](http://docs.scipy.org/doc/scipy/reference/sparse.html) For efficiency, we recommend using NumPy arrays over lists, and using the @@ -52,8 +52,8 @@ For efficiency, we recommend using NumPy arrays over lists, and using the for SciPy matrices, or MLlib's own SparseVector class. Several other simple data types are used throughout the library, e.g. the LabeledPoint -class ([Java/Scala](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint), -[Python](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html)) for labeled data. +class ([Java/Scala](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint), +[Python](api/python/pyspark.mllib.regression.LabeledPoint-class.html)) for labeled data. # Dependencies MLlib uses the [jblas](https://github.com/mikiobraun/jblas) linear algebra library, which itself diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index 396b98d52a994..c79cc3d944f32 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -95,12 +95,12 @@ As an alternative to just use the subgradient `$R'(\wv)$` of the regularizer in direction, an improved update for some cases can be obtained by using the proximal operator instead. For the L1-regularizer, the proximal operator is given by soft thresholding, as implemented in -[L1Updater](api/mllib/index.html#org.apache.spark.mllib.optimization.L1Updater). +[L1Updater](api/scala/index.html#org.apache.spark.mllib.optimization.L1Updater). ## Update Schemes for Distributed SGD The SGD implementation in -[GradientDescent](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) uses +[GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) uses a simple (distributed) sampling of the data examples. We recall that the loss part of the optimization problem `$\eqref{eq:regPrimal}$` is `$\frac1n \sum_{i=1}^n L(\wv;\x_i,y_i)$`, and therefore `$\frac1n \sum_{i=1}^n L'_{\wv,i}$` would @@ -138,7 +138,7 @@ are developed, see the section for example. The SGD method -[GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) +[GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) has the following parameters: * `gradient` is a class that computes the stochastic gradient of the function @@ -161,6 +161,6 @@ each iteration, to compute the gradient direction. Available algorithms for gradient descent: -* [GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) +* [GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) diff --git a/docs/python-programming-guide.md b/docs/python-programming-guide.md index 39de603b29f87..98233bf556b79 100644 --- a/docs/python-programming-guide.md +++ b/docs/python-programming-guide.md @@ -134,7 +134,7 @@ Files listed here will be added to the `PYTHONPATH` and shipped to remote worker Code dependencies can be added to an existing SparkContext using its `addPyFile()` method. You can set [configuration properties](configuration.html#spark-properties) by passing a -[SparkConf](api/pyspark/pyspark.conf.SparkConf-class.html) object to SparkContext: +[SparkConf](api/python/pyspark.conf.SparkConf-class.html) object to SparkContext: {% highlight python %} from pyspark import SparkConf, SparkContext @@ -147,7 +147,7 @@ sc = SparkContext(conf = conf) # API Docs -[API documentation](api/pyspark/index.html) for PySpark is available as Epydoc. +[API documentation](api/python/index.html) for PySpark is available as Epydoc. Many of the methods also contain [doctests](http://docs.python.org/2/library/doctest.html) that provide additional usage examples. # Libraries diff --git a/docs/quick-start.md b/docs/quick-start.md index 6b4f4ba4254a2..68afa6e1bff95 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -138,7 +138,9 @@ Spark README. Note that you'll need to replace YOUR_SPARK_HOME with the location installed. Unlike the earlier examples with the Spark shell, which initializes its own SparkContext, we initialize a SparkContext as part of the program. -We pass the SparkContext constructor a SparkConf object which contains information about our +We pass the SparkContext constructor a +[SparkConf](api/scala/index.html#org.apache.spark.SparkConf) +object which contains information about our application. We also call sc.addJar to make sure that when our application is launched in cluster mode, the jar file containing it will be shipped automatically to worker nodes. @@ -327,4 +329,4 @@ Congratulations on running your first Spark application! * For an in-depth overview of the API see "Programming Guides" menu section. * For running applications on a cluster head to the [deployment overview](cluster-overview.html). -* For configuration options available to Spark applications see the [configuration page](configuration.html). \ No newline at end of file +* For configuration options available to Spark applications see the [configuration page](configuration.html). diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index 4431da0721ac7..a3171709ff05b 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -147,7 +147,7 @@ All transformations in Spark are lazy, in that they do not compute their By default, each transformed RDD is recomputed each time you run an action on it. However, you may also *persist* an RDD in memory using the `persist` (or `cache`) method, in which case Spark will keep the elements around on the cluster for much faster access the next time you query it. There is also support for persisting datasets on disk, or replicated across the cluster. The next section in this document describes these options. -The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD) for details): +The following tables list the transformations and actions currently supported (see also the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD) for details): ### Transformations @@ -216,7 +216,7 @@ The following tables list the transformations and actions currently supported (s
      Stage IdPool NameDescriptionSubmittedDurationTasks: Succeeded/TotalShuffle ReadShuffle Write
      {s.stageId} - - {poolName.get} - - {makeDescription(s)}{submissionTime}{formattedDuration} - {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} + {s.stageId} + + {poolName.get} + {shuffleRead}{shuffleWrite}
      {makeDescription(s)}{submissionTime}{formattedDuration} + {makeProgressBar(startedTasks, completedTasks, failedTasks, totalTasks)} + {shuffleRead}{shuffleWrite}
      Failure Reason{s.failureReason.getOrElse("")}
      If you use Kryo serialization, set this class to register your custom classes with Kryo. It should be set to a class that extends - KryoRegistrator. + KryoRegistrator. See the tuning guide for more details.
      ClassFunction Type
      -A complete list of transformations is available in the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD). +A complete list of transformations is available in the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD). ### Actions @@ -264,7 +264,7 @@ A complete list of transformations is available in the [RDD API doc](api/core/in -A complete list of actions is available in the [RDD API doc](api/core/index.html#org.apache.spark.rdd.RDD). +A complete list of actions is available in the [RDD API doc](api/scala/index.html#org.apache.spark.rdd.RDD). ## RDD Persistence @@ -283,7 +283,7 @@ In addition, each RDD can be stored using a different *storage level*, allowing persist the dataset on disk, or persist it in memory but as serialized Java objects (to save space), or replicate it across nodes, or store the data in off-heap memory in [Tachyon](http://tachyon-project.org/). These levels are chosen by passing a -[`org.apache.spark.storage.StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel) +[`org.apache.spark.storage.StorageLevel`](api/scala/index.html#org.apache.spark.storage.StorageLevel) object to `persist()`. The `cache()` method is a shorthand for using the default storage level, which is `StorageLevel.MEMORY_ONLY` (store deserialized objects in memory). The complete set of available storage levels is: @@ -355,7 +355,7 @@ waiting to recompute a lost partition. If you want to define your own storage level (say, with replication factor of 3 instead of 2), then use the function factor method `apply()` of the -[`StorageLevel`](api/core/index.html#org.apache.spark.storage.StorageLevel$) singleton object. +[`StorageLevel`](api/scala/index.html#org.apache.spark.storage.StorageLevel$) singleton object. Spark has a block manager inside the Executors that let you chose memory, disk, or off-heap. The latter is for storing RDDs off-heap outside the Executor JVM on top of the memory management system diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 8e98cc0c80a34..e25379bd7614d 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -14,8 +14,8 @@ title: Spark SQL Programming Guide Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed -[Row](api/sql/catalyst/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with +[SchemaRDD](api/scala/index.html#org.apache.spark.sql.SchemaRDD). SchemaRDDs are composed +[Row](api/scala/index.html#org.apache.spark.sql.catalyst.expressions.Row) objects along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). @@ -27,8 +27,8 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac
      Spark SQL allows relational queries expressed in SQL, HiveQL, or Scala to be executed using Spark. At the core of this component is a new type of RDD, -[JavaSchemaRDD](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed -[Row](api/sql/catalyst/index.html#org.apache.spark.sql.api.java.Row) objects along with +[JavaSchemaRDD](api/scala/index.html#org.apache.spark.sql.api.java.JavaSchemaRDD). JavaSchemaRDDs are composed +[Row](api/scala/index.html#org.apache.spark.sql.api.java.Row) objects along with a schema that describes the data types of each column in the row. A JavaSchemaRDD is similar to a table in a traditional relational database. A JavaSchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). @@ -38,8 +38,8 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac Spark SQL allows relational queries expressed in SQL or HiveQL to be executed using Spark. At the core of this component is a new type of RDD, -[SchemaRDD](api/pyspark/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed -[Row](api/pyspark/pyspark.sql.Row-class.html) objects along with +[SchemaRDD](api/python/pyspark.sql.SchemaRDD-class.html). SchemaRDDs are composed +[Row](api/python/pyspark.sql.Row-class.html) objects along with a schema that describes the data types of each column in the row. A SchemaRDD is similar to a table in a traditional relational database. A SchemaRDD can be created from an existing RDD, parquet file, or by running HiveQL against data stored in [Apache Hive](http://hive.apache.org/). @@ -56,7 +56,7 @@ file, or by running HiveQL against data stored in [Apache Hive](http://hive.apac
      The entry point into all relational functionality in Spark is the -[SQLContext](api/sql/core/index.html#org.apache.spark.sql.SQLContext) class, or one of its +[SQLContext](api/scala/index.html#org.apache.spark.sql.SQLContext) class, or one of its descendants. To create a basic SQLContext, all you need is a SparkContext. {% highlight scala %} @@ -72,7 +72,7 @@ import sqlContext._
      The entry point into all relational functionality in Spark is the -[JavaSQLContext](api/sql/core/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one +[JavaSQLContext](api/scala/index.html#org.apache.spark.sql.api.java.JavaSQLContext) class, or one of its descendants. To create a basic JavaSQLContext, all you need is a JavaSparkContext. {% highlight java %} @@ -85,7 +85,7 @@ JavaSQLContext sqlCtx = new org.apache.spark.sql.api.java.JavaSQLContext(ctx);
      The entry point into all relational functionality in Spark is the -[SQLContext](api/pyspark/pyspark.sql.SQLContext-class.html) class, or one +[SQLContext](api/python/pyspark.sql.SQLContext-class.html) class, or one of its decedents. To create a basic SQLContext, all you need is a SparkContext. {% highlight python %} @@ -331,7 +331,7 @@ val teenagers = people.where('age >= 10).where('age <= 19).select('name) The DSL uses Scala symbols to represent columns in the underlying table, which are identifiers prefixed with a tick (`'`). Implicit conversions turn these symbols into expressions that are evaluated by the SQL execution engine. A full list of the functions supported can be found in the -[ScalaDoc](api/sql/core/index.html#org.apache.spark.sql.SchemaRDD). +[ScalaDoc](api/scala/index.html#org.apache.spark.sql.SchemaRDD). diff --git a/docs/streaming-custom-receivers.md b/docs/streaming-custom-receivers.md index 3fb540c9fb560..3cfa4516cc12d 100644 --- a/docs/streaming-custom-receivers.md +++ b/docs/streaming-custom-receivers.md @@ -9,7 +9,7 @@ This guide shows the programming model and features by walking through a simple ### Writing a Simple Receiver -This starts with implementing [NetworkReceiver](api/streaming/index.html#org.apache.spark.streaming.dstream.NetworkReceiver). +This starts with implementing [NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver). The following is a simple socket text-stream receiver. @@ -125,4 +125,4 @@ _A more comprehensive example is provided in the spark streaming examples_ ## References 1.[Akka Actor documentation](http://doc.akka.io/docs/akka/2.0.5/scala/actors.html) -2.[NetworkReceiver](api/streaming/index.html#org.apache.spark.streaming.dstream.NetworkReceiver) +2.[NetworkReceiver](api/scala/index.html#org.apache.spark.streaming.dstream.NetworkReceiver) diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index f9904d45013f6..946d6c4879cf3 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -40,7 +40,7 @@ Spark Streaming provides a high-level abstraction called *discretized stream* or which represents a continuous stream of data. DStreams can be created either from input data stream from sources such as Kafka and Flume, or by applying high-level operations on other DStreams. Internally, a DStream is represented as a sequence of -[RDDs](api/core/index.html#org.apache.spark.rdd.RDD). +[RDDs](api/scala/index.html#org.apache.spark.rdd.RDD). This guide shows you how to start writing Spark Streaming programs with DStreams. You can write Spark Streaming programs in Scala or Java, both of which are presented in this guide. You @@ -62,7 +62,7 @@ First, we import the names of the Spark Streaming classes, and some implicit conversions from StreamingContext into our environment, to add useful methods to other classes we need (like DStream). -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) is the +[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) is the main entry point for all streaming functionality. {% highlight scala %} @@ -71,7 +71,7 @@ import org.apache.spark.streaming.StreamingContext._ {% endhighlight %} Then we create a -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) object. +[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object. Besides Spark's configuration, we specify that any DStream will be processed in 1 second batches. @@ -132,7 +132,7 @@ The complete code can be found in the Spark Streaming example
      First, we create a -[JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object, +[JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object, which is the main entry point for all streaming functionality. Besides Spark's configuration, we specify that any DStream would be processed in 1 second batches. @@ -168,7 +168,7 @@ JavaDStream words = lines.flatMap( generating multiple new records from each record in the source DStream. In this case, each line will be split into multiple words and the stream of words is represented as the `words` DStream. Note that we defined the transformation using a -[FlatMapFunction](api/core/index.html#org.apache.spark.api.java.function.FlatMapFunction) object. +[FlatMapFunction](api/scala/index.html#org.apache.spark.api.java.function.FlatMapFunction) object. As we will discover along the way, there are a number of such convenience classes in the Java API that help define DStream transformations. @@ -192,9 +192,9 @@ wordCounts.print(); // Print a few of the counts to the console {% endhighlight %} The `words` DStream is further mapped (one-to-one transformation) to a DStream of `(word, -1)` pairs, using a [PairFunction](api/core/index.html#org.apache.spark.api.java.function.PairFunction) +1)` pairs, using a [PairFunction](api/scala/index.html#org.apache.spark.api.java.function.PairFunction) object. Then, it is reduced to get the frequency of words in each batch of data, -using a [Function2](api/core/index.html#org.apache.spark.api.java.function.Function2) object. +using a [Function2](api/scala/index.html#org.apache.spark.api.java.function.Function2) object. Finally, `wordCounts.print()` will print a few of the counts generated every second. Note that when these lines are executed, Spark Streaming only sets up the computation it @@ -333,7 +333,7 @@ for the full list of supported sources and artifacts.
      To initialize a Spark Streaming program in Scala, a -[`StreamingContext`](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) +[`StreamingContext`](api/scala/index.html#org.apache.spark.streaming.StreamingContext) object has to be created, which is the main entry point of all Spark Streaming functionality. A `StreamingContext` object can be created by using @@ -344,7 +344,7 @@ new StreamingContext(master, appName, batchDuration, [sparkHome], [jars])
      To initialize a Spark Streaming program in Java, a -[`JavaStreamingContext`](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) +[`JavaStreamingContext`](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) object has to be created, which is the main entry point of all Spark Streaming functionality. A `JavaStreamingContext` object can be created by using @@ -431,8 +431,8 @@ and process any files created in that directory. Note that For more details on streams from files, Akka actors and sockets, see the API documentations of the relevant functions in -[StreamingContext](api/streaming/index.html#org.apache.spark.streaming.StreamingContext) for -Scala and [JavaStreamingContext](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) +[StreamingContext](api/scala/index.html#org.apache.spark.streaming.StreamingContext) for +Scala and [JavaStreamingContext](api/scala/index.html#org.apache.spark.streaming.api.java.JavaStreamingContext) for Java. Additional functionality for creating DStreams from sources such as Kafka, Flume, and Twitter @@ -802,10 +802,10 @@ output operators are defined: The complete list of DStream operations is available in the API documentation. For the Scala API, -see [DStream](api/streaming/index.html#org.apache.spark.streaming.dstream.DStream) -and [PairDStreamFunctions](api/streaming/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). -For the Java API, see [JavaDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.dstream.DStream) -and [JavaPairDStream](api/streaming/index.html#org.apache.spark.streaming.api.java.JavaPairDStream). +see [DStream](api/scala/index.html#org.apache.spark.streaming.dstream.DStream) +and [PairDStreamFunctions](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions). +For the Java API, see [JavaDStream](api/scala/index.html#org.apache.spark.streaming.api.java.dstream.DStream) +and [JavaPairDStream](api/scala/index.html#org.apache.spark.streaming.api.java.JavaPairDStream). Specifically for the Java API, see [Spark's Java programming guide](java-programming-guide.html) for more information. @@ -881,7 +881,7 @@ Cluster resources maybe under-utilized if the number of parallel tasks used in a computation is not high enough. For example, for distributed reduce operations like `reduceByKey` and `reduceByKeyAndWindow`, the default number of parallel tasks is 8. You can pass the level of parallelism as an argument (see the -[`PairDStreamFunctions`](api/streaming/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) +[`PairDStreamFunctions`](api/scala/index.html#org.apache.spark.streaming.dstream.PairDStreamFunctions) documentation), or set the [config property](configuration.html#spark-properties) `spark.default.parallelism` to change the default. @@ -925,7 +925,7 @@ A good approach to figure out the right batch size for your application is to te conservative batch size (say, 5-10 seconds) and a low data rate. To verify whether the system is able to keep up with data rate, you can check the value of the end-to-end delay experienced by each processed batch (either look for "Total delay" in Spark driver log4j logs, or use the -[StreamingListener](api/streaming/index.html#org.apache.spark.streaming.scheduler.StreamingListener) +[StreamingListener](api/scala/index.html#org.apache.spark.streaming.scheduler.StreamingListener) interface). If the delay is maintained to be comparable to the batch size, then system is stable. Otherwise, if the delay is continuously increasing, it means that the system is unable to keep up and it @@ -952,7 +952,7 @@ exception saying so. ## Monitoring Besides Spark's in-built [monitoring capabilities](monitoring.html), the progress of a Spark Streaming program can also be monitored using the [StreamingListener] -(api/streaming/index.html#org.apache.spark.scheduler.StreamingListener) interface, +(api/scala/index.html#org.apache.spark.scheduler.StreamingListener) interface, which allows you to get statistics of batch processing times, queueing delays, and total end-to-end delays. Note that this is still an experimental API and it is likely to be improved upon (i.e., more information reported) in the future. @@ -965,9 +965,9 @@ in Spark Streaming applications and achieving more consistent batch processing t * **Default persistence level of DStreams**: Unlike RDDs, the default persistence level of DStreams serializes the data in memory (that is, -[StorageLevel.MEMORY_ONLY_SER](api/core/index.html#org.apache.spark.storage.StorageLevel$) for +[StorageLevel.MEMORY_ONLY_SER](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for DStream compared to -[StorageLevel.MEMORY_ONLY](api/core/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). +[StorageLevel.MEMORY_ONLY](api/scala/index.html#org.apache.spark.storage.StorageLevel$) for RDDs). Even though keeping the data serialized incurs higher serialization/deserialization overheads, it significantly reduces GC pauses. @@ -1244,15 +1244,15 @@ and output 30 after recovery. # Where to Go from Here * API documentation - - Main docs of StreamingContext and DStreams in [Scala](api/streaming/index.html#org.apache.spark.streaming.package) - and [Java](api/streaming/index.html#org.apache.spark.streaming.api.java.package) + - Main docs of StreamingContext and DStreams in [Scala](api/scala/index.html#org.apache.spark.streaming.package) + and [Java](api/scala/index.html#org.apache.spark.streaming.api.java.package) - Additional docs for - [Kafka](api/external/kafka/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), - [Flume](api/external/flume/index.html#org.apache.spark.streaming.flume.FlumeUtils$), - [Twitter](api/external/twitter/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), - [ZeroMQ](api/external/zeromq/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and - [MQTT](api/external/mqtt/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) + [Kafka](api/scala/index.html#org.apache.spark.streaming.kafka.KafkaUtils$), + [Flume](api/scala/index.html#org.apache.spark.streaming.flume.FlumeUtils$), + [Twitter](api/scala/index.html#org.apache.spark.streaming.twitter.TwitterUtils$), + [ZeroMQ](api/scala/index.html#org.apache.spark.streaming.zeromq.ZeroMQUtils$), and + [MQTT](api/scala/index.html#org.apache.spark.streaming.mqtt.MQTTUtils$) * More examples in [Scala]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/scala/org/apache/spark/streaming/examples) and [Java]({{site.SPARK_GITHUB_URL}}/tree/master/examples/src/main/java/org/apache/spark/streaming/examples) -* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming +* [Paper](http://www.eecs.berkeley.edu/Pubs/TechRpts/2012/EECS-2012-259.pdf) describing Spark Streaming. diff --git a/docs/tuning.md b/docs/tuning.md index cc069f0e84b9c..78e10770a80b6 100644 --- a/docs/tuning.md +++ b/docs/tuning.md @@ -48,7 +48,7 @@ Spark automatically includes Kryo serializers for the many commonly-used core Sc in the AllScalaRegistrar from the [Twitter chill](https://github.com/twitter/chill) library. To register your own custom classes with Kryo, create a public class that extends -[`org.apache.spark.serializer.KryoRegistrator`](api/core/index.html#org.apache.spark.serializer.KryoRegistrator) and set the +[`org.apache.spark.serializer.KryoRegistrator`](api/scala/index.html#org.apache.spark.serializer.KryoRegistrator) and set the `spark.kryo.registrator` config property to point to it, as follows: {% highlight scala %} @@ -222,7 +222,7 @@ enough. Spark automatically sets the number of "map" tasks to run on each file a (though you can control it through optional parameters to `SparkContext.textFile`, etc), and for distributed "reduce" operations, such as `groupByKey` and `reduceByKey`, it uses the largest parent RDD's number of partitions. You can pass the level of parallelism as a second argument -(see the [`spark.PairRDDFunctions`](api/core/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation), +(see the [`spark.PairRDDFunctions`](api/scala/index.html#org.apache.spark.rdd.PairRDDFunctions) documentation), or set the config property `spark.default.parallelism` to change the default. In general, we recommend 2-3 tasks per CPU core in your cluster. diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 33f9d644ca66d..f115f0d3a041c 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 com.typesafe.tools.mima.plugin.MimaKeys.previousArtifact +import sbtunidoc.Plugin._ +import UnidocKeys._ import scala.collection.JavaConversions._ @@ -31,6 +33,7 @@ import scala.collection.JavaConversions._ object SparkBuild extends Build { val SPARK_VERSION = "1.0.0-SNAPSHOT" + val SPARK_VERSION_SHORT = SPARK_VERSION.replaceAll("-SNAPSHOT", "") // Hadoop version to build against. For example, "1.0.4" for Apache releases, or // "2.0.0-mr1-cdh4.2.0" for Cloudera Hadoop. Note that these variables can be set @@ -184,12 +187,17 @@ object SparkBuild extends Build { // Show full stack trace and duration in test cases. testOptions in Test += Tests.Argument("-oDF"), // Remove certain packages from Scaladoc - scalacOptions in (Compile,doc) := Seq("-groups", "-skip-packages", Seq( - "akka", - "org.apache.spark.network", - "org.apache.spark.deploy", - "org.apache.spark.util.collection" - ).mkString(":")), + scalacOptions in (Compile, doc) := Seq( + "-groups", + "-skip-packages", Seq( + "akka", + "org.apache.spark.api.python", + "org.apache.spark.network", + "org.apache.spark.deploy", + "org.apache.spark.util.collection" + ).mkString(":"), + "-doc-title", "Spark " + SPARK_VERSION_SHORT + " ScalaDoc" + ), // Only allow one test at a time, even across projects, since they run in the same JVM concurrentRestrictions in Global += Tags.limit(Tags.Test, 1), @@ -283,7 +291,7 @@ object SparkBuild extends Build { publishMavenStyle in MavenCompile := true, publishLocal in MavenCompile <<= publishTask(publishLocalConfiguration in MavenCompile, deliverLocal), publishLocalBoth <<= Seq(publishLocal in MavenCompile, publishLocal).dependOn - ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings + ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings ++ genjavadocSettings val akkaVersion = "2.2.3-shaded-protobuf" val chillVersion = "0.3.1" @@ -349,15 +357,57 @@ object SparkBuild extends Build { libraryDependencies ++= maybeAvro ) - def rootSettings = sharedSettings ++ Seq( - publish := {} + // Create a colon-separate package list adding "org.apache.spark" in front of all of them, + // for easier specification of JavaDoc package groups + def packageList(names: String*): String = { + names.map(s => "org.apache.spark." + s).mkString(":") + } + + def rootSettings = sharedSettings ++ scalaJavaUnidocSettings ++ Seq( + publish := {}, + + unidocProjectFilter in (ScalaUnidoc, unidoc) := + inAnyProject -- inProjects(repl, examples, tools, yarn, yarnAlpha), + unidocProjectFilter in (JavaUnidoc, unidoc) := + inAnyProject -- inProjects(repl, examples, bagel, graphx, catalyst, tools, yarn, yarnAlpha), + + // Skip class names containing $ and some internal packages in Javadocs + unidocAllSources in (JavaUnidoc, unidoc) := { + (unidocAllSources in (JavaUnidoc, unidoc)).value + .map(_.filterNot(_.getName.contains("$"))) + .map(_.filterNot(_.getCanonicalPath.contains("akka"))) + .map(_.filterNot(_.getCanonicalPath.contains("deploy"))) + .map(_.filterNot(_.getCanonicalPath.contains("network"))) + .map(_.filterNot(_.getCanonicalPath.contains("executor"))) + .map(_.filterNot(_.getCanonicalPath.contains("python"))) + .map(_.filterNot(_.getCanonicalPath.contains("collection"))) + }, + + // Javadoc options: create a window title, and group key packages on index page + javacOptions in doc := Seq( + "-windowtitle", "Spark " + SPARK_VERSION_SHORT + " JavaDoc", + "-public", + "-group", "Core Java API", packageList("api.java", "api.java.function"), + "-group", "Spark Streaming", packageList( + "streaming.api.java", "streaming.flume", "streaming.kafka", + "streaming.mqtt", "streaming.twitter", "streaming.zeromq" + ), + "-group", "MLlib", packageList( + "mllib.classification", "mllib.clustering", "mllib.evaluation.binary", "mllib.linalg", + "mllib.linalg.distributed", "mllib.optimization", "mllib.rdd", "mllib.recommendation", + "mllib.regression", "mllib.stat", "mllib.tree", "mllib.tree.configuration", + "mllib.tree.impurity", "mllib.tree.model", "mllib.util" + ), + "-group", "Spark SQL", packageList("sql.api.java", "sql.hive.api.java"), + "-noqualifier", "java.lang" + ) ) def replSettings = sharedSettings ++ Seq( name := "spark-repl", - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v ), - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v ), - libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v ) + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-compiler" % v), + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "jline" % v), + libraryDependencies <+= scalaVersion(v => "org.scala-lang" % "scala-reflect" % v) ) def examplesSettings = sharedSettings ++ Seq( diff --git a/project/plugins.sbt b/project/plugins.sbt index c25a25863d6ed..0cd16fd5bedd4 100644 --- a/project/plugins.sbt +++ b/project/plugins.sbt @@ -23,3 +23,4 @@ addSbtPlugin("com.typesafe" % "sbt-mima-plugin" % "0.1.6") addSbtPlugin("com.alpinenow" % "junit_xml_listener" % "0.5.0") +addSbtPlugin("com.eed3si9n" % "sbt-unidoc" % "0.3.0") diff --git a/python/epydoc.conf b/python/epydoc.conf index 081ed215ae60c..b73860bad8263 100644 --- a/python/epydoc.conf +++ b/python/epydoc.conf @@ -18,8 +18,8 @@ # # Information about the project. -name: PySpark -url: http://spark-project.org +name: Spark 1.0.0 Python API Docs +url: http://spark.apache.org # The list of modules to document. Modules can be named using # dotted names, module filenames, or package directory names. From b3e5366f696c463f1c2f033b0d5c7365e5d6b0f8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Mon, 21 Apr 2014 22:24:44 -0700 Subject: [PATCH 329/397] [Fix #274] Document + fix annotation usages ... so that we don't follow an unspoken set of forbidden rules for adding **@AlphaComponent**, **@DeveloperApi**, and **@Experimental** annotations in the code. In addition, this PR (1) removes unnecessary `:: * ::` tags, (2) adds missing `:: * ::` tags, and (3) removes annotations for internal APIs. Author: Andrew Or Closes #470 from andrewor14/annotations-fix and squashes the following commits: 92a7f42 [Andrew Or] Document + fix annotation usages --- core/src/main/scala/org/apache/spark/SparkContext.scala | 1 + .../org/apache/spark/annotation/AlphaComponent.java | 9 ++++++++- .../scala/org/apache/spark/annotation/DeveloperApi.java | 5 +++++ .../scala/org/apache/spark/annotation/Experimental.java | 5 +++++ core/src/main/scala/org/apache/spark/rdd/RDD.scala | 6 ------ .../scala/org/apache/spark/scheduler/SplitInfo.scala | 8 ++++++-- .../scala/org/apache/spark/mllib/clustering/KMeans.scala | 6 +++--- .../apache/spark/mllib/regression/RegressionModel.scala | 3 --- .../src/main/scala/org/apache/spark/sql/SQLContext.scala | 2 -- .../src/main/scala/org/apache/spark/sql/SchemaRDD.scala | 2 +- .../scala/org/apache/spark/sql/hive/HiveContext.scala | 1 - 11 files changed, 29 insertions(+), 19 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 7933d68d67d96..aa49f32ecdf62 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1110,6 +1110,7 @@ class SparkContext(config: SparkConf) extends Logging { } /** + * :: Experimental :: * Submit a job for execution and return a FutureJob holding the result. */ @Experimental diff --git a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java index af01fb7cfbd04..db7b25c727d34 100644 --- a/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java +++ b/core/src/main/scala/org/apache/spark/annotation/AlphaComponent.java @@ -19,7 +19,14 @@ import java.lang.annotation.*; -/** A new component of Spark which may have unstable API's. */ +/** + * A new component of Spark which may have unstable API's. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: AlphaComponent ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. + */ @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, ElementType.CONSTRUCTOR, ElementType.LOCAL_VARIABLE, ElementType.PACKAGE}) diff --git a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java index 5d546e7a63985..0ecef6db0e039 100644 --- a/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java +++ b/core/src/main/scala/org/apache/spark/annotation/DeveloperApi.java @@ -23,6 +23,11 @@ * A lower-level, unstable API intended for developers. * * Developer API's might change or be removed in minor versions of Spark. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: DeveloperApi ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. */ @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, diff --git a/core/src/main/scala/org/apache/spark/annotation/Experimental.java b/core/src/main/scala/org/apache/spark/annotation/Experimental.java index 306b1418d8d0a..ff8120291455f 100644 --- a/core/src/main/scala/org/apache/spark/annotation/Experimental.java +++ b/core/src/main/scala/org/apache/spark/annotation/Experimental.java @@ -24,6 +24,11 @@ * * Experimental API's might change or be removed in minor versions of Spark, or be adopted as * first-class Spark API's. + * + * NOTE: If there exists a Scaladoc comment that immediately precedes this annotation, the first + * line of the comment must be ":: Experimental ::" with no trailing blank line. This is because + * of the known issue that Scaladoc displays only either the annotation or the comment, whichever + * comes first. */ @Retention(RetentionPolicy.RUNTIME) @Target({ElementType.TYPE, ElementType.FIELD, ElementType.METHOD, ElementType.PARAMETER, diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 891efccf23b6a..5d2ed2b402cd8 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -94,26 +94,20 @@ abstract class RDD[T: ClassTag]( def compute(split: Partition, context: TaskContext): Iterator[T] /** - * :: DeveloperApi :: * Implemented by subclasses to return the set of partitions in this RDD. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ - @DeveloperApi protected def getPartitions: Array[Partition] /** - * :: DeveloperApi :: * Implemented by subclasses to return how this RDD depends on parent RDDs. This method will only * be called once, so it is safe to implement a time-consuming computation in it. */ - @DeveloperApi protected def getDependencies: Seq[Dependency[_]] = deps /** - * :: DeveloperApi :: * Optionally overridden by subclasses to specify placement preferences. */ - @DeveloperApi protected def getPreferredLocations(split: Partition): Seq[String] = Nil /** Optionally overridden by subclasses to specify how they are partitioned. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala index b85eabd6bbdbc..1ce83485f024b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SplitInfo.scala @@ -24,8 +24,12 @@ import org.apache.spark.annotation.DeveloperApi // information about a specific split instance : handles both split instances. // So that we do not need to worry about the differences. @DeveloperApi -class SplitInfo(val inputFormatClazz: Class[_], val hostLocation: String, val path: String, - val length: Long, val underlyingSplit: Any) { +class SplitInfo( + val inputFormatClazz: Class[_], + val hostLocation: String, + val path: String, + val length: Long, + val underlyingSplit: Any) { override def toString(): String = { "SplitInfo " + super.toString + " .. inputFormatClazz " + inputFormatClazz + ", hostLocation : " + hostLocation + ", path : " + path + diff --git a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala index 90cf8525df523..dee9ef07e41ed 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/clustering/KMeans.scala @@ -391,9 +391,9 @@ object KMeans { * Returns the squared Euclidean distance between two vectors computed by * [[org.apache.spark.mllib.util.MLUtils#fastSquaredDistance]]. */ - private[clustering] - def fastSquaredDistance(v1: BreezeVectorWithNorm, v2: BreezeVectorWithNorm) - : Double = { + private[clustering] def fastSquaredDistance( + v1: BreezeVectorWithNorm, + v2: BreezeVectorWithNorm): Double = { MLUtils.fastSquaredDistance(v1.vector, v1.norm, v2.vector, v2.norm) } diff --git a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala index 027305aa78926..b27e158b43f9a 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/regression/RegressionModel.scala @@ -21,9 +21,6 @@ import org.apache.spark.rdd.RDD import org.apache.spark.mllib.linalg.Vector import org.apache.spark.annotation.Experimental -/** - * :: Experimental :: - */ @Experimental trait RegressionModel extends Serializable { /** diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 4d216b5cd14cb..e25201a6c1775 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -249,11 +249,9 @@ class SQLContext(@transient val sparkContext: SparkContext) } /** - * :: DeveloperApi :: * The primary workflow for executing relational queries using Spark. Designed to allow easy * access to the intermediate phases of query execution for developers. */ - @DeveloperApi protected abstract class QueryExecution { def logical: LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index f2ae5b0fe612f..6cb0e0f61f2d2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -20,7 +20,7 @@ package org.apache.spark.sql import net.razorvine.pickle.Pickler import org.apache.spark.{Dependency, OneToOneDependency, Partition, TaskContext} -import org.apache.spark.annotation.{AlphaComponent, Experimental, DeveloperApi} +import org.apache.spark.annotation.{AlphaComponent, Experimental} import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.expressions._ 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 c0d8adf43dd07..b21f24dad785d 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 @@ -239,7 +239,6 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1) /** Extends QueryExecution with hive specific features. */ - @DeveloperApi protected[sql] abstract class QueryExecution extends super.QueryExecution { // TODO: Create mixin for the analyzer instead of overriding things here. override lazy val optimizedPlan = From ac164b79d12f5a269d9c05d04dced51a415e3b0e Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Mon, 21 Apr 2014 23:10:53 -0700 Subject: [PATCH 330/397] [SPARK-1459] Use local path (and not complete URL) when opening local lo... ...g file. Author: Marcelo Vanzin Closes #375 from vanzin/event-file and squashes the following commits: f673029 [Marcelo Vanzin] [SPARK-1459] Use local path (and not complete URL) when opening local log file. --- core/src/main/scala/org/apache/spark/util/FileLogger.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 68a12e8ed67d7..7d58d1c765180 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -91,7 +91,7 @@ private[spark] class FileLogger( val dstream = uri.getScheme match { case "file" | null => // Second parameter is whether to append - new FileOutputStream(logPath, !overwrite) + new FileOutputStream(uri.getPath, !overwrite) case _ => val path = new Path(logPath) From 83084d3b7bab09524dd85054c442abaf97d22fea Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 22 Apr 2014 00:42:16 -0700 Subject: [PATCH 331/397] SPARK-1496: Have jarOfClass return Option[String] A simple change, mostly had to change a bunch of example code. Author: Patrick Wendell Closes #438 from pwendell/jar-of-class and squashes the following commits: aa010ff [Patrick Wendell] SPARK-1496: Have jarOfClass return Option[String] --- .../src/main/scala/org/apache/spark/SparkContext.scala | 10 +++++----- .../org/apache/spark/examples/BroadcastTest.scala | 2 +- .../org/apache/spark/examples/CassandraCQLTest.scala | 2 +- .../apache/spark/examples/ExceptionHandlingTest.scala | 2 +- .../scala/org/apache/spark/examples/GroupByTest.scala | 2 +- .../scala/org/apache/spark/examples/HBaseTest.scala | 2 +- .../scala/org/apache/spark/examples/HdfsTest.scala | 2 +- .../scala/org/apache/spark/examples/LogQuery.scala | 2 +- .../org/apache/spark/examples/MultiBroadcastTest.scala | 2 +- .../spark/examples/SimpleSkewedGroupByTest.scala | 2 +- .../org/apache/spark/examples/SkewedGroupByTest.scala | 2 +- .../scala/org/apache/spark/examples/SparkALS.scala | 2 +- .../scala/org/apache/spark/examples/SparkHdfsLR.scala | 2 +- .../scala/org/apache/spark/examples/SparkKMeans.scala | 2 +- .../main/scala/org/apache/spark/examples/SparkLR.scala | 2 +- .../org/apache/spark/examples/SparkPageRank.scala | 2 +- .../main/scala/org/apache/spark/examples/SparkPi.scala | 2 +- .../main/scala/org/apache/spark/examples/SparkTC.scala | 2 +- .../org/apache/spark/examples/SparkTachyonHdfsLR.scala | 2 +- .../org/apache/spark/examples/SparkTachyonPi.scala | 2 +- .../apache/spark/examples/mllib/TallSkinnyPCA.scala | 2 +- .../apache/spark/examples/mllib/TallSkinnySVD.scala | 2 +- .../spark/streaming/examples/ActorWordCount.scala | 2 +- .../spark/streaming/examples/FlumeEventCount.scala | 2 +- .../spark/streaming/examples/HdfsWordCount.scala | 2 +- .../spark/streaming/examples/KafkaWordCount.scala | 2 +- .../spark/streaming/examples/MQTTWordCount.scala | 2 +- .../spark/streaming/examples/NetworkWordCount.scala | 2 +- .../apache/spark/streaming/examples/QueueStream.scala | 2 +- .../spark/streaming/examples/RawNetworkGrep.scala | 2 +- .../examples/RecoverableNetworkWordCount.scala | 2 +- .../streaming/examples/StatefulNetworkWordCount.scala | 2 +- .../spark/streaming/examples/TwitterAlgebirdCMS.scala | 2 +- .../spark/streaming/examples/TwitterAlgebirdHLL.scala | 2 +- .../spark/streaming/examples/TwitterPopularTags.scala | 2 +- .../spark/streaming/examples/ZeroMQWordCount.scala | 2 +- .../examples/clickstream/PageViewStream.scala | 2 +- .../org/apache/spark/streaming/StreamingContext.scala | 2 +- 38 files changed, 42 insertions(+), 42 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index aa49f32ecdf62..25ca650a3a37e 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1346,19 +1346,19 @@ object SparkContext extends Logging { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to SparkContext. */ - def jarOfClass(cls: Class[_]): Seq[String] = { + def jarOfClass(cls: Class[_]): Option[String] = { val uri = cls.getResource("/" + cls.getName.replace('.', '/') + ".class") if (uri != null) { val uriStr = uri.toString if (uriStr.startsWith("jar:file:")) { // URI will be of the form "jar:file:/path/foo.jar!/package/cls.class", // so pull out the /path/foo.jar - List(uriStr.substring("jar:file:".length, uriStr.indexOf('!'))) + Some(uriStr.substring("jar:file:".length, uriStr.indexOf('!'))) } else { - Nil + None } } else { - Nil + None } } @@ -1367,7 +1367,7 @@ object SparkContext extends Logging { * to pass their JARs to SparkContext. In most cases you can call jarOfObject(this) in * your driver program. */ - def jarOfObject(obj: AnyRef): Seq[String] = jarOfClass(obj.getClass) + def jarOfObject(obj: AnyRef): Option[String] = jarOfClass(obj.getClass) /** * Creates a modified version of a SparkConf with the parameters that can be passed separately diff --git a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala index c8c916bb45e00..f6dfd2c4c6217 100644 --- a/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/BroadcastTest.scala @@ -35,7 +35,7 @@ object BroadcastTest { System.setProperty("spark.broadcast.blockSize", blockSize) val sc = new SparkContext(args(0), "Broadcast Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala index 4e787240e912d..3798329fc2f41 100644 --- a/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/CassandraCQLTest.scala @@ -75,7 +75,7 @@ object CassandraCQLTest { val sc = new SparkContext(args(0), "CQLTestApp", System.getenv("SPARK_HOME"), - SparkContext.jarOfClass(this.getClass)) + SparkContext.jarOfClass(this.getClass).toSeq) val cHost: String = args(1) val cPort: String = args(2) val KeySpace = "retail" diff --git a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala index be7d39549a28d..f0dcef431b2e1 100644 --- a/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/ExceptionHandlingTest.scala @@ -27,7 +27,7 @@ object ExceptionHandlingTest { } val sc = new SparkContext(args(0), "ExceptionHandlingTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) sc.parallelize(0 until sc.defaultParallelism).foreach { i => if (math.random > 0.75) { throw new Exception("Testing exception handling") diff --git a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala index 2b7ecdc991325..e67bb29a49405 100644 --- a/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/GroupByTest.scala @@ -36,7 +36,7 @@ object GroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala index cbf78e8e9eba1..adbd1c02fa2ea 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HBaseTest.scala @@ -27,7 +27,7 @@ import org.apache.spark.rdd.NewHadoopRDD object HBaseTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HBaseTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val conf = HBaseConfiguration.create() diff --git a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala index dd6d5205133be..c7a4884af10b7 100644 --- a/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/HdfsTest.scala @@ -22,7 +22,7 @@ import org.apache.spark._ object HdfsTest { def main(args: Array[String]) { val sc = new SparkContext(args(0), "HdfsTest", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val file = sc.textFile(args(1)) val mapped = file.map(s => s.length).cache() for (iter <- 1 to 10) { diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 35758fa003d94..820e87d04f3fa 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -46,7 +46,7 @@ object LogQuery { } val sc = new SparkContext(args(0), "Log Query", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val dataSet = if (args.length == 2) sc.textFile(args(1)) diff --git a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala index 58f26f1e24052..c8985eae33de3 100644 --- a/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/MultiBroadcastTest.scala @@ -28,7 +28,7 @@ object MultiBroadcastTest { } val sc = new SparkContext(args(0), "Multi-Broadcast Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val num = if (args.length > 2) args(2).toInt else 1000000 diff --git a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala index 557a0c1841339..54e8503711e30 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SimpleSkewedGroupByTest.scala @@ -37,7 +37,7 @@ object SimpleSkewedGroupByTest { var ratio = if (args.length > 5) args(5).toInt else 5.0 val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala index 05a74725b875b..1c5f22e1c00bb 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SkewedGroupByTest.scala @@ -36,7 +36,7 @@ object SkewedGroupByTest { var numReducers = if (args.length > 4) args(4).toInt else numMappers val sc = new SparkContext(args(0), "GroupBy Test", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val pairs1 = sc.parallelize(0 until numMappers, numMappers).flatMap { p => val ranGen = new Random diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala index 191c82fd913ee..0dc726aecdd28 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkALS.scala @@ -113,7 +113,7 @@ object SparkALS { printf("Running with M=%d, U=%d, F=%d, iters=%d\n", M, U, F, ITERATIONS) val sc = new SparkContext(host, "SparkALS", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val R = generateR() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala index fd63ba3dbce7d..3a6f18c33ea4b 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkHdfsLR.scala @@ -56,7 +56,7 @@ object SparkHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq, Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala index 8aa31d7e6a2c2..dcae9591b0407 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkKMeans.scala @@ -57,7 +57,7 @@ object SparkKMeans { System.exit(1) } val sc = new SparkContext(args(0), "SparkLocalKMeans", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val lines = sc.textFile(args(1)) val data = lines.map(parseVector _).cache() val K = args(2).toInt diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala index d70ce603bb71d..4f74882ccbea5 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkLR.scala @@ -52,7 +52,7 @@ object SparkLR { System.exit(1) } val sc = new SparkContext(args(0), "SparkLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val numSlices = if (args.length > 1) args(1).toInt else 2 val points = sc.parallelize(generateData, numSlices).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala index 60e4a11a21f69..fa41c5c560943 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPageRank.scala @@ -37,7 +37,7 @@ object SparkPageRank { } var iters = args(2).toInt val ctx = new SparkContext(args(0), "PageRank", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val lines = ctx.textFile(args(1), 1) val links = lines.map{ s => val parts = s.split("\\s+") diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala index d3babc3ed12c8..d8f5720504223 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkPi.scala @@ -29,7 +29,7 @@ object SparkPi { System.exit(1) } val spark = new SparkContext(args(0), "SparkPi", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices val count = spark.parallelize(1 to n, slices).map { i => diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala index 65bd61abda6cd..17d983cd875db 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTC.scala @@ -47,7 +47,7 @@ object SparkTC { System.exit(1) } val spark = new SparkContext(args(0), "SparkTC", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 var tc = spark.parallelize(generateGraph, slices).cache() diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala index 4f558929add51..7e43c384bdb9d 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonHdfsLR.scala @@ -58,7 +58,7 @@ object SparkTachyonHdfsLR { val inputPath = args(1) val conf = SparkHadoopUtil.get.newConfiguration() val sc = new SparkContext(args(0), "SparkTachyonHdfsLR", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass), Map(), + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq, Map(), InputFormatInfo.computePreferredLocations( Seq(new InputFormatInfo(conf, classOf[org.apache.hadoop.mapred.TextInputFormat], inputPath)) )) diff --git a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala index 2b207fd8d3e16..93459110e4e0e 100644 --- a/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala +++ b/examples/src/main/scala/org/apache/spark/examples/SparkTachyonPi.scala @@ -33,7 +33,7 @@ object SparkTachyonPi { System.exit(1) } val spark = new SparkContext(args(0), "SparkTachyonPi", - System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val slices = if (args.length > 1) args(1).toInt else 2 val n = 100000 * slices diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala index 61b9655cd3759..39e71cdab4328 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnyPCA.scala @@ -44,7 +44,7 @@ object TallSkinnyPCA { .setMaster(args(0)) .setAppName("TallSkinnyPCA") .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass)) + .setJars(SparkContext.jarOfClass(this.getClass).toSeq) val sc = new SparkContext(conf) // Load and parse the data file. diff --git a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala index 9aeebf58eabfb..2b7de2acc65d6 100644 --- a/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala +++ b/examples/src/main/scala/org/apache/spark/examples/mllib/TallSkinnySVD.scala @@ -44,7 +44,7 @@ object TallSkinnySVD { .setMaster(args(0)) .setAppName("TallSkinnySVD") .setSparkHome(System.getenv("SPARK_HOME")) - .setJars(SparkContext.jarOfClass(this.getClass)) + .setJars(SparkContext.jarOfClass(this.getClass).toSeq) val sc = new SparkContext(conf) // Load and parse the data file. diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala index eb44768b9c101..c845dd8904c90 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ActorWordCount.scala @@ -150,7 +150,7 @@ object ActorWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ActorWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) /* * Following is the use of actorStream to plug in custom actor as receiver diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala index 11c3aaad3c8a8..26b6024534124 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/FlumeEventCount.scala @@ -51,7 +51,7 @@ object FlumeEventCount { val batchInterval = Milliseconds(2000) // Create the context and set the batch size val ssc = new StreamingContext(master, "FlumeEventCount", batchInterval, - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a flume stream val stream = FlumeUtils.createStream(ssc, host,port,StorageLevel.MEMORY_ONLY_SER_2) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala index 1c0ce3111e290..7f86fc792aacf 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/HdfsWordCount.scala @@ -41,7 +41,7 @@ object HdfsWordCount { // Create the context val ssc = new StreamingContext(args(0), "HdfsWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create the FileInputDStream on the directory and use the // stream to count words in new files created diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index cca0be2cbb9c9..3a18bd02b89ac 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -52,7 +52,7 @@ object KafkaWordCount { val Array(master, zkQuorum, group, topics, numThreads) = args val ssc = new StreamingContext(master, "KafkaWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) ssc.checkpoint("checkpoint") val topicpMap = topics.split(",").map((_,numThreads.toInt)).toMap diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index 656222e0c1b31..3d3c27ed78204 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -97,7 +97,7 @@ object MQTTWordCount { val Seq(master, brokerUrl, topic) = args.toSeq val ssc = new StreamingContext(master, "MqttWordCount", Seconds(2), System.getenv("SPARK_HOME"), - StreamingContext.jarOfClass(this.getClass)) + StreamingContext.jarOfClass(this.getClass).toSeq) val lines = MQTTUtils.createStream(ssc, brokerUrl, topic, StorageLevel.MEMORY_ONLY_SER_2) val words = lines.flatMap(x => x.toString.split(" ")) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala index d4c4d86b3466c..272ab11212451 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/NetworkWordCount.scala @@ -47,7 +47,7 @@ object NetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala index 612ecf7b7821a..ff2a205ec1c15 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/QueueStream.scala @@ -35,7 +35,7 @@ object QueueStream { // Create the context val ssc = new StreamingContext(args(0), "QueueStream", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create the queue through which RDDs can be pushed to // a QueueInputDStream diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala index cdd7547d0d3b4..d915c0c39b334 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RawNetworkGrep.scala @@ -50,7 +50,7 @@ object RawNetworkGrep { // Create the context val ssc = new StreamingContext(master, "RawNetworkGrep", Milliseconds(batchMillis), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Warm up the JVMs on master and slave for JIT compilation to kick in RawTextHelper.warmUp(ssc.sparkContext) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala index 56d10a964b71b..4aacbb1991418 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/RecoverableNetworkWordCount.scala @@ -77,7 +77,7 @@ object RecoverableNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(master, "RecoverableNetworkWordCount", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a NetworkInputDStream on target ip:port and count the // words in input stream of \n delimited text (eg. generated by 'nc') diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala index 14f65a2f8d46c..ef94c9298dd93 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/StatefulNetworkWordCount.scala @@ -54,7 +54,7 @@ object StatefulNetworkWordCount { // Create the context with a 1 second batch size val ssc = new StreamingContext(args(0), "NetworkWordCumulativeCountUpdateStateByKey", - Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + Seconds(1), System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) ssc.checkpoint(".") // Create a NetworkInputDStream on target ip:port and count the diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala index 8a654f8fada21..c38905e8f3663 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdCMS.scala @@ -68,7 +68,7 @@ object TwitterAlgebirdCMS { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdCMS", Seconds(10), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER_2) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala index 45771d7050eeb..c067046f9020e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterAlgebirdHLL.scala @@ -55,7 +55,7 @@ object TwitterAlgebirdHLL { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterAlgebirdHLL", Seconds(5), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) val stream = TwitterUtils.createStream(ssc, None, filters, StorageLevel.MEMORY_ONLY_SER) val users = stream.map(status => status.getUser.getId) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala index 896d010c68f18..2597c8178862a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/TwitterPopularTags.scala @@ -41,7 +41,7 @@ object TwitterPopularTags { val (master, filters) = (args.head, args.tail) val ssc = new StreamingContext(master, "TwitterPopularTags", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) val stream = TwitterUtils.createStream(ssc, None, filters) val hashTags = stream.flatMap(status => status.getText.split(" ").filter(_.startsWith("#"))) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala index 6f88db1abf19d..109ff855b5228 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/ZeroMQWordCount.scala @@ -86,7 +86,7 @@ object ZeroMQWordCount { // Create the context and set the batch size val ssc = new StreamingContext(master, "ZeroMQWordCount", Seconds(2), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) def bytesToStringIterator(x: Seq[ByteString]) = (x.map(_.utf8String)).iterator diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index 2b130fb30eaaa..edc769c59734a 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -45,7 +45,7 @@ object PageViewStream { // Create the context val ssc = new StreamingContext("local[2]", "PageViewStream", Seconds(1), - System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass)) + System.getenv("SPARK_HOME"), StreamingContext.jarOfClass(this.getClass).toSeq) // Create a NetworkInputDStream on target host:port and convert each line to a PageView val pageViews = ssc.socketTextStream(host, port) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index daa5c69bbadbf..6d9dc87a707ac 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -543,7 +543,7 @@ object StreamingContext extends Logging { * Find the JAR from which a given class was loaded, to make it easy for users to pass * their JARs to StreamingContext. */ - def jarOfClass(cls: Class[_]): Seq[String] = SparkContext.jarOfClass(cls) + def jarOfClass(cls: Class[_]): Option[String] = SparkContext.jarOfClass(cls) private[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { // Set the default cleaner delay to an hour if not already set. From 87de29084eed2a287ee114d492f45b211c500c6f Mon Sep 17 00:00:00 2001 From: CodingCat Date: Tue, 22 Apr 2014 09:43:13 -0700 Subject: [PATCH 332/397] [HOTFIX] SPARK-1399: remove outdated comments as the original PR was merged before this mistake is found....fix here, Sorry about that @pwendell, @andrewor14, I will be more careful next time Author: CodingCat Closes #474 from CodingCat/hotfix_1399 and squashes the following commits: f3a8ba9 [CodingCat] move outdated comments --- core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala index 2eb8c7a3a3073..153434a2032be 100644 --- a/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala +++ b/core/src/main/scala/org/apache/spark/ui/jobs/StageTable.scala @@ -37,7 +37,6 @@ private[ui] class StageTableBase( protected def isFairScheduler = parent.isFairScheduler protected def columns: Seq[Node] = { - // create dummy element to wrap the columns Stage Id ++ {if (isFairScheduler) {Pool Name} else Seq.empty} ++ Description From 0f87e6ad4366a8c453a7415bc89399030003c264 Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Tue, 22 Apr 2014 09:44:41 -0700 Subject: [PATCH 333/397] [SPARK-1560]: Updated Pyrolite Dependency to be Java 6 compatible Changed the Pyrolite dependency to a build which targets Java 6. Author: Ahir Reddy Closes #479 from ahirreddy/java6-pyrolite and squashes the following commits: 8ea25d3 [Ahir Reddy] Updated maven build to use java 6 compatible pyrolite dabc703 [Ahir Reddy] Updated Pyrolite dependency to be Java 6 compatible --- core/pom.xml | 2 +- project/SparkBuild.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/pom.xml b/core/pom.xml index d87e2bca030e3..058b7acba73ca 100644 --- a/core/pom.xml +++ b/core/pom.xml @@ -269,7 +269,7 @@ org.spark-project pyrolite - 2.0 + 2.0.1 diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index f115f0d3a041c..1ca4e0dc8add1 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -352,7 +352,7 @@ object SparkBuild extends Build { "com.twitter" % "chill-java" % chillVersion excludeAll(excludeAsm), "org.tachyonproject" % "tachyon" % "0.4.1-thrift" excludeAll(excludeHadoop, excludeCurator, excludeEclipseJetty, excludePowermock), "com.clearspring.analytics" % "stream" % "2.5.1" excludeAll(excludeFastutil), - "org.spark-project" % "pyrolite" % "2.0" + "org.spark-project" % "pyrolite" % "2.0.1" ), libraryDependencies ++= maybeAvro ) From c919798f0912dc03c8365b9a384d9ee6d5b25c51 Mon Sep 17 00:00:00 2001 From: Xusen Yin Date: Tue, 22 Apr 2014 11:06:18 -0700 Subject: [PATCH 334/397] fix bugs of dot in python If there are no `transpose()` in `self.theta`, a *ValueError: matrices are not aligned* is occurring. The former test case just ignore this situation. Author: Xusen Yin Closes #463 from yinxusen/python-naive-bayes and squashes the following commits: fcbe3bc [Xusen Yin] fix bugs of dot in python --- python/pyspark/mllib/classification.py | 2 +- python/pyspark/mllib/tests.py | 8 ++++---- 2 files changed, 5 insertions(+), 5 deletions(-) diff --git a/python/pyspark/mllib/classification.py b/python/pyspark/mllib/classification.py index 3a23e0801fe7b..c5844597c95f2 100644 --- a/python/pyspark/mllib/classification.py +++ b/python/pyspark/mllib/classification.py @@ -154,7 +154,7 @@ def __init__(self, labels, pi, theta): def predict(self, x): """Return the most likely class for a data vector x""" - return self.labels[numpy.argmax(self.pi + _dot(x, self.theta))] + return self.labels[numpy.argmax(self.pi + _dot(x, self.theta.transpose()))] class NaiveBayes(object): @classmethod diff --git a/python/pyspark/mllib/tests.py b/python/pyspark/mllib/tests.py index d4771d779f9f4..1ee96bb4af37b 100644 --- a/python/pyspark/mllib/tests.py +++ b/python/pyspark/mllib/tests.py @@ -104,10 +104,10 @@ def test_clustering(self): def test_classification(self): from pyspark.mllib.classification import LogisticRegressionWithSGD, SVMWithSGD, NaiveBayes data = [ - LabeledPoint(0.0, [1, 0]), - LabeledPoint(1.0, [0, 1]), - LabeledPoint(0.0, [2, 0]), - LabeledPoint(1.0, [0, 2]) + LabeledPoint(0.0, [1, 0, 0]), + LabeledPoint(1.0, [0, 1, 1]), + LabeledPoint(0.0, [2, 0, 0]), + LabeledPoint(1.0, [0, 2, 1]) ] rdd = self.sc.parallelize(data) features = [p.features.tolist() for p in data] From bf9d49b6d1f668b49795c2d380ab7d64ec0029da Mon Sep 17 00:00:00 2001 From: Tor Myklebust Date: Tue, 22 Apr 2014 11:07:30 -0700 Subject: [PATCH 335/397] [SPARK-1281] Improve partitioning in ALS ALS was using HashPartitioner and explicit uses of `%` together. Further, the naked use of `%` meant that, if the number of partitions corresponded with the stride of arithmetic progressions appearing in user and product ids, users and products could be mapped into buckets in an unfair or unwise way. This pull request: 1) Makes the Partitioner an instance variable of ALS. 2) Replaces the direct uses of `%` with calls to a Partitioner. 3) Defines an anonymous Partitioner that scrambles the bits of the object's hashCode before reducing to the number of present buckets. This pull request does not make the partitioner user-configurable. I'm not all that happy about the way I did (1). It introduces an icky lifetime issue and dances around it by nulling something. However, I don't know a better way to make the partitioner visible everywhere it needs to be visible. Author: Tor Myklebust Closes #407 from tmyklebu/master and squashes the following commits: dcf583a [Tor Myklebust] Remove the partitioner member variable; instead, thread that needle everywhere it needs to go. 23d6f91 [Tor Myklebust] Stop making the partitioner configurable. 495784f [Tor Myklebust] Merge branch 'master' of https://github.com/apache/spark 674933a [Tor Myklebust] Fix style. 40edc23 [Tor Myklebust] Fix missing space. f841345 [Tor Myklebust] Fix daft bug creating 'pairs', also for -> foreach. 5ec9e6c [Tor Myklebust] Clean a couple of things up using 'map'. 36a0f43 [Tor Myklebust] Make the partitioner private. d872b09 [Tor Myklebust] Add negative id ALS test. df27697 [Tor Myklebust] Support custom partitioners. Currently we use the same partitioner for users and products. c90b6d8 [Tor Myklebust] Scramble user and product ids before bucketing. c774d7d [Tor Myklebust] Make the partitioner a member variable and use it instead of modding directly. --- .../spark/mllib/recommendation/ALS.scala | 47 +++++++++++-------- .../spark/mllib/recommendation/ALSSuite.scala | 30 ++++++++++-- 2 files changed, 54 insertions(+), 23 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala index 1f5c746a3457c..60fb73f2b5be5 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/recommendation/ALS.scala @@ -21,6 +21,7 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import scala.math.{abs, sqrt} import scala.util.Random import scala.util.Sorting +import scala.util.hashing.byteswap32 import com.esotericsoftware.kryo.Kryo import org.jblas.{DoubleMatrix, SimpleBlas, Solve} @@ -32,6 +33,7 @@ import org.apache.spark.storage.StorageLevel import org.apache.spark.rdd.RDD import org.apache.spark.serializer.KryoRegistrator import org.apache.spark.SparkContext._ +import org.apache.spark.util.Utils /** * Out-link information for a user or product block. This includes the original user/product IDs @@ -169,34 +171,39 @@ class ALS private ( this.numBlocks } - val partitioner = new HashPartitioner(numBlocks) + val partitioner = new Partitioner { + val numPartitions = numBlocks - val ratingsByUserBlock = ratings.map{ rating => (rating.user % numBlocks, rating) } + def getPartition(x: Any): Int = { + Utils.nonNegativeMod(byteswap32(x.asInstanceOf[Int]), numPartitions) + } + } + + val ratingsByUserBlock = ratings.map{ rating => + (partitioner.getPartition(rating.user), rating) + } val ratingsByProductBlock = ratings.map{ rating => - (rating.product % numBlocks, Rating(rating.product, rating.user, rating.rating)) + (partitioner.getPartition(rating.product), + Rating(rating.product, rating.user, rating.rating)) } - val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock) - val (productInLinks, productOutLinks) = makeLinkRDDs(numBlocks, ratingsByProductBlock) + val (userInLinks, userOutLinks) = makeLinkRDDs(numBlocks, ratingsByUserBlock, partitioner) + val (productInLinks, productOutLinks) = + makeLinkRDDs(numBlocks, ratingsByProductBlock, partitioner) // Initialize user and product factors randomly, but use a deterministic seed for each // partition so that fault recovery works val seedGen = new Random(seed) val seed1 = seedGen.nextInt() val seed2 = seedGen.nextInt() - // Hash an integer to propagate random bits at all positions, similar to java.util.HashTable - def hash(x: Int): Int = { - val r = x ^ (x >>> 20) ^ (x >>> 12) - r ^ (r >>> 7) ^ (r >>> 4) - } var users = userOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(hash(seed1 ^ index)) + val rand = new Random(byteswap32(seed1 ^ index)) itr.map { case (x, y) => (x, y.elementIds.map(_ => randomFactor(rank, rand))) } } var products = productOutLinks.mapPartitionsWithIndex { (index, itr) => - val rand = new Random(hash(seed2 ^ index)) + val rand = new Random(byteswap32(seed2 ^ index)) itr.map { case (x, y) => (x, y.elementIds.map(_ => randomFactor(rank, rand))) } @@ -327,13 +334,14 @@ class ALS private ( * Make the out-links table for a block of the users (or products) dataset given the list of * (user, product, rating) values for the users in that block (or the opposite for products). */ - private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating]): OutLinkBlock = { + private def makeOutLinkBlock(numBlocks: Int, ratings: Array[Rating], + partitioner: Partitioner): OutLinkBlock = { val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap val shouldSend = Array.fill(numUsers)(new BitSet(numBlocks)) for (r <- ratings) { - shouldSend(userIdToPos(r.user))(r.product % numBlocks) = true + shouldSend(userIdToPos(r.user))(partitioner.getPartition(r.product)) = true } OutLinkBlock(userIds, shouldSend) } @@ -342,14 +350,15 @@ class ALS private ( * Make the in-links table for a block of the users (or products) dataset given a list of * (user, product, rating) values for the users in that block (or the opposite for products). */ - private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating]): InLinkBlock = { + private def makeInLinkBlock(numBlocks: Int, ratings: Array[Rating], + partitioner: Partitioner): InLinkBlock = { val userIds = ratings.map(_.user).distinct.sorted val numUsers = userIds.length val userIdToPos = userIds.zipWithIndex.toMap // Split out our ratings by product block val blockRatings = Array.fill(numBlocks)(new ArrayBuffer[Rating]) for (r <- ratings) { - blockRatings(r.product % numBlocks) += r + blockRatings(partitioner.getPartition(r.product)) += r } val ratingsForBlock = new Array[Array[(Array[Int], Array[Double])]](numBlocks) for (productBlock <- 0 until numBlocks) { @@ -374,14 +383,14 @@ class ALS private ( * the users (or (blockId, (p, u, r)) for the products). We create these simultaneously to avoid * having to shuffle the (blockId, (u, p, r)) RDD twice, or to cache it. */ - private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)]) + private def makeLinkRDDs(numBlocks: Int, ratings: RDD[(Int, Rating)], partitioner: Partitioner) : (RDD[(Int, InLinkBlock)], RDD[(Int, OutLinkBlock)]) = { val grouped = ratings.partitionBy(new HashPartitioner(numBlocks)) val links = grouped.mapPartitionsWithIndex((blockId, elements) => { val ratings = elements.map{_._2}.toArray - val inLinkBlock = makeInLinkBlock(numBlocks, ratings) - val outLinkBlock = makeOutLinkBlock(numBlocks, ratings) + val inLinkBlock = makeInLinkBlock(numBlocks, ratings, partitioner) + val outLinkBlock = makeOutLinkBlock(numBlocks, ratings, partitioner) Iterator.single((blockId, (inLinkBlock, outLinkBlock))) }, true) val inLinks = links.mapValues(_._1) diff --git a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala index 5aab9aba8f9c0..4dfcd4b52ec66 100644 --- a/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/mllib/recommendation/ALSSuite.scala @@ -27,6 +27,7 @@ import org.jblas.DoubleMatrix import org.apache.spark.mllib.util.LocalSparkContext import org.apache.spark.SparkContext._ +import org.apache.spark.Partitioner object ALSSuite { @@ -74,7 +75,6 @@ object ALSSuite { (sampledRatings, trueRatings, truePrefs) } - } @@ -128,6 +128,25 @@ class ALSSuite extends FunSuite with LocalSparkContext { assert(u11 != u2) } + test("negative ids") { + val data = ALSSuite.generateRatings(50, 50, 2, 0.7, false, false) + val ratings = sc.parallelize(data._1.map { case Rating(u, p, r) => + Rating(u - 25, p - 25, r) + }) + val correct = data._2 + val model = ALS.train(ratings, 5, 15) + + val pairs = Array.tabulate(50, 50)((u, p) => (u - 25, p - 25)).flatten + val ans = model.predict(sc.parallelize(pairs)).collect() + ans.foreach { r => + val u = r.user + 25 + val p = r.product + 25 + val v = r.rating + val error = v - correct.get(u, p) + assert(math.abs(error) < 0.4) + } + } + /** * Test if we can correctly factorize R = U * P where U and P are of known rank. * @@ -140,16 +159,19 @@ class ALSSuite extends FunSuite with LocalSparkContext { * @param implicitPrefs flag to test implicit feedback * @param bulkPredict flag to test bulk prediciton * @param negativeWeights whether the generated data can contain negative values + * @param numBlocks number of blocks to partition users and products into */ def testALS(users: Int, products: Int, features: Int, iterations: Int, samplingRate: Double, matchThreshold: Double, implicitPrefs: Boolean = false, - bulkPredict: Boolean = false, negativeWeights: Boolean = false) + bulkPredict: Boolean = false, negativeWeights: Boolean = false, numBlocks: Int = -1) { val (sampledRatings, trueRatings, truePrefs) = ALSSuite.generateRatings(users, products, features, samplingRate, implicitPrefs, negativeWeights) val model = implicitPrefs match { - case false => ALS.train(sc.parallelize(sampledRatings), features, iterations) - case true => ALS.trainImplicit(sc.parallelize(sampledRatings), features, iterations) + case false => ALS.train(sc.parallelize(sampledRatings), features, iterations, 0.01, + numBlocks, 0L) + case true => ALS.trainImplicit(sc.parallelize(sampledRatings), features, iterations, 0.01, + numBlocks, 1.0, 0L) } val predictedU = new DoubleMatrix(users, features) From 26d35f3fd942761b0adecd1a720e1fa834db4de9 Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 22 Apr 2014 11:20:47 -0700 Subject: [PATCH 336/397] [SPARK-1506][MLLIB] Documentation improvements for MLlib 1.0 Preview: http://54.82.240.23:4000/mllib-guide.html Table of contents: * Basics * Data types * Summary statistics * Classification and regression * linear support vector machine (SVM) * logistic regression * linear linear squares, Lasso, and ridge regression * decision tree * naive Bayes * Collaborative Filtering * alternating least squares (ALS) * Clustering * k-means * Dimensionality reduction * singular value decomposition (SVD) * principal component analysis (PCA) * Optimization * stochastic gradient descent * limited-memory BFGS (L-BFGS) Author: Xiangrui Meng Closes #422 from mengxr/mllib-doc and squashes the following commits: 944e3a9 [Xiangrui Meng] merge master f9fda28 [Xiangrui Meng] minor 9474065 [Xiangrui Meng] add alpha to ALS examples 928e630 [Xiangrui Meng] initialization_mode -> initializationMode 5bbff49 [Xiangrui Meng] add imports to labeled point examples c17440d [Xiangrui Meng] fix python nb example 28f40dc [Xiangrui Meng] remove localhost:4000 369a4d3 [Xiangrui Meng] Merge branch 'master' into mllib-doc 7dc95cc [Xiangrui Meng] update linear methods 053ad8a [Xiangrui Meng] add links to go back to the main page abbbf7e [Xiangrui Meng] update ALS argument names 648283e [Xiangrui Meng] level down statistics 14e2287 [Xiangrui Meng] add sample libsvm data and use it in guide 8cd2441 [Xiangrui Meng] minor updates 186ab07 [Xiangrui Meng] update section names 6568d65 [Xiangrui Meng] update toc, level up lr and svm 162ee12 [Xiangrui Meng] rename section names 5c1e1b1 [Xiangrui Meng] minor 8aeaba1 [Xiangrui Meng] wrap long lines 6ce6a6f [Xiangrui Meng] add summary statistics to toc 5760045 [Xiangrui Meng] claim beta cc604bf [Xiangrui Meng] remove classification and regression 92747b3 [Xiangrui Meng] make section titles consistent e605dd6 [Xiangrui Meng] add LIBSVM loader f639674 [Xiangrui Meng] add python section to migration guide c82ffb4 [Xiangrui Meng] clean optimization 31660eb [Xiangrui Meng] update linear algebra and stat 0a40837 [Xiangrui Meng] first pass over linear methods 1fc8271 [Xiangrui Meng] update toc 906ed0a [Xiangrui Meng] add a python example to naive bayes 5f0a700 [Xiangrui Meng] update collaborative filtering 656d416 [Xiangrui Meng] update mllib-clustering 86e143a [Xiangrui Meng] remove data types section from main page 8d1a128 [Xiangrui Meng] move part of linear algebra to data types and add Java/Python examples d1b5cbf [Xiangrui Meng] merge master 72e4804 [Xiangrui Meng] one pass over tree guide 64f8995 [Xiangrui Meng] move decision tree guide to a separate file 9fca001 [Xiangrui Meng] add first version of linear algebra guide 53c9552 [Xiangrui Meng] update dependencies f316ec2 [Xiangrui Meng] add migration guide f399f6c [Xiangrui Meng] move linear-algebra to dimensionality-reduction 182460f [Xiangrui Meng] add guide for naive Bayes 137fd1d [Xiangrui Meng] re-organize toc a61e434 [Xiangrui Meng] update mllib's toc --- docs/mllib-basics.md | 476 ++++++++++++++++++++ docs/mllib-classification-regression.md | 568 ------------------------ docs/mllib-clustering.md | 44 +- docs/mllib-collaborative-filtering.md | 78 ++-- docs/mllib-decision-tree.md | 185 ++++++++ docs/mllib-dimensionality-reduction.md | 86 ++++ docs/mllib-guide.md | 172 ++++--- docs/mllib-linear-algebra.md | 74 --- docs/mllib-linear-methods.md | 389 ++++++++++++++++ docs/mllib-naive-bayes.md | 115 +++++ docs/mllib-optimization.md | 25 +- mllib/data/sample_libsvm_data.txt | 100 +++++ 12 files changed, 1543 insertions(+), 769 deletions(-) create mode 100644 docs/mllib-basics.md delete mode 100644 docs/mllib-classification-regression.md create mode 100644 docs/mllib-decision-tree.md create mode 100644 docs/mllib-dimensionality-reduction.md delete mode 100644 docs/mllib-linear-algebra.md create mode 100644 docs/mllib-linear-methods.md create mode 100644 docs/mllib-naive-bayes.md create mode 100644 mllib/data/sample_libsvm_data.txt diff --git a/docs/mllib-basics.md b/docs/mllib-basics.md new file mode 100644 index 0000000000000..710ce1721fe25 --- /dev/null +++ b/docs/mllib-basics.md @@ -0,0 +1,476 @@ +--- +layout: global +title: MLlib - Basics +--- + +* Table of contents +{:toc} + +MLlib supports local vectors and matrices stored on a single machine, +as well as distributed matrices backed by one or more RDDs. +In the current implementation, local vectors and matrices are simple data models +to serve public interfaces. The underly linear algebra operations are provided by +[Breeze](http://www.scalanlp.org/) and [jblas](http://jblas.org/). +A training example used in supervised learning is called "labeled point" in MLlib. + +## Local vector + +A local vector has integer-typed and 0-based indices and double-typed values, stored on a single +machine. MLlib supports two types of local vectors: dense and sparse. A dense vector is backed by +a double array representing its entry values, while a sparse vector is backed by two parallel +arrays: indices and values. For example, a vector $(1.0, 0.0, 3.0)$ can be represented in dense +format as `[1.0, 0.0, 3.0]` or in sparse format as `(3, [0, 2], [1.0, 3.0])`, where `3` is the size +of the vector. + +
      +
      + +The base class of local vectors is +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector), and we provide two +implementations: [`DenseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseVector) and +[`SparseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.SparseVector). We recommend +using the factory methods implemented in +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) to create local vectors. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +// Create a dense vector (1.0, 0.0, 3.0). +val dv: Vector = Vectors.dense(1.0, 0.0, 3.0) +// Create a sparse vector (1.0, 0.0, 3.0) by specifying its indices and values corresponding to nonzero entries. +val sv1: Vector = Vectors.sparse(3, Array(0, 2), Array(1.0, 3.0)) +// Create a sparse vector (1.0, 0.0, 3.0) by specifying its nonzero entries. +val sv2: Vector = Vectors.sparse(3, Seq((0, 1.0), (2, 3.0))) +{% endhighlight %} + +***Note*** + +Scala imports `scala.collection.immutable.Vector` by default, so you have to import +`org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. + +
      + +
      + +The base class of local vectors is +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector), and we provide two +implementations: [`DenseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseVector) and +[`SparseVector`](api/mllib/index.html#org.apache.spark.mllib.linalg.SparseVector). We recommend +using the factory methods implemented in +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) to create local vectors. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; + +// Create a dense vector (1.0, 0.0, 3.0). +Vector dv = Vectors.dense(1.0, 0.0, 3.0); +// Create a sparse vector (1.0, 0.0, 3.0) by specifying its indices and values corresponding to nonzero entries. +Vector sv = Vectors.sparse(3, new int[] {0, 2}, new double[] {1.0, 3.0}); +{% endhighlight %} +
      + +
      +MLlib recognizes the following types as dense vectors: + +* NumPy's [`array`](http://docs.scipy.org/doc/numpy/reference/generated/numpy.array.html) +* Python's list, e.g., `[1, 2, 3]` + +and the following as sparse vectors: + +* MLlib's [`SparseVector`](api/pyspark/pyspark.mllib.linalg.SparseVector-class.html). +* SciPy's + [`csc_matrix`](http://docs.scipy.org/doc/scipy/reference/generated/scipy.sparse.csc_matrix.html#scipy.sparse.csc_matrix) + with a single column + +We recommend using NumPy arrays over lists for efficiency, and using the factory methods implemented +in [`Vectors`](api/pyspark/pyspark.mllib.linalg.Vectors-class.html) to create sparse vectors. + +{% highlight python %} +import numpy as np +import scipy.sparse as sps +from pyspark.mllib.linalg import Vectors + +# Use a NumPy array as a dense vector. +dv1 = np.array([1.0, 0.0, 3.0]) +# Use a Python list as a dense vector. +dv2 = [1.0, 0.0, 3.0] +# Create a SparseVector. +sv1 = Vectors.sparse(3, [0, 2], [1.0, 3.0]) +# Use a single-column SciPy csc_matrix as a sparse vector. +sv2 = sps.csc_matrix((np.array([1.0, 3.0]), np.array([0, 2]), np.array([0, 2])), shape = (3, 1)) +{% endhighlight %} + +
      +
      + +## Labeled point + +A labeled point is a local vector, either dense or sparse, associated with a label/response. +In MLlib, labeled points are used in supervised learning algorithms. +We use a double to store a label, so we can use labeled points in both regression and classification. +For binary classification, label should be either $0$ (negative) or $1$ (positive). +For multiclass classification, labels should be class indices staring from zero: $0, 1, 2, \ldots$. + +
      + +
      + +A labeled point is represented by the case class +[`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint). + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.regression.LabeledPoint + +// Create a labeled point with a positive label and a dense feature vector. +val pos = LabeledPoint(1.0, Vectors.dense(1.0, 0.0, 3.0)) + +// Create a labeled point with a negative label and a sparse feature vector. +val neg = LabeledPoint(0.0, Vectors.sparse(3, Array(0, 2), Array(1.0, 3.0))) +{% endhighlight %} +
      + +
      + +A labeled point is represented by +[`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint). + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vectors; +import org.apache.spark.mllib.regression.LabeledPoint; + +// Create a labeled point with a positive label and a dense feature vector. +LabeledPoint pos = new LabeledPoint(1.0, Vectors.dense(1.0, 0.0, 3.0)); + +// Create a labeled point with a negative label and a sparse feature vector. +LabeledPoint neg = new LabeledPoint(1.0, Vectors.sparse(3, new int[] {0, 2}, new double[] {1.0, 3.0})); +{% endhighlight %} +
      + +
      + +A labeled point is represented by +[`LabeledPoint`](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html). + +{% highlight python %} +from pyspark.mllib.linalg import SparseVector +from pyspark.mllib.regression import LabeledPoint + +# Create a labeled point with a positive label and a dense feature vector. +pos = LabeledPoint(1.0, [1.0, 0.0, 3.0]) + +# Create a labeled point with a negative label and a sparse feature vector. +neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0])) +{% endhighlight %} +
      +
      + +***Sparse data*** + +It is very common in practice to have sparse training data. MLlib supports reading training +examples stored in `LIBSVM` format, which is the default format used by +[`LIBSVM`](http://www.csie.ntu.edu.tw/~cjlin/libsvm/) and +[`LIBLINEAR`](http://www.csie.ntu.edu.tw/~cjlin/liblinear/). It is a text format. Each line +represents a labeled sparse feature vector using the following format: + +~~~ +label index1:value1 index2:value2 ... +~~~ + +where the indices are one-based and in ascending order. +After loading, the feature indices are converted to zero-based. + +
      +
      + +[`MLUtils.loadLibSVMData`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +examples stored in LIBSVM format. + +{% highlight scala %} +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.util.MLUtils +import org.apache.spark.rdd.RDD + +val training: RDD[LabeledPoint] = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +{% endhighlight %} +
      + +
      +[`MLUtils.loadLibSVMData`](api/mllib/index.html#org.apache.spark.mllib.util.MLUtils$) reads training +examples stored in LIBSVM format. + +{% highlight java %} +import org.apache.spark.mllib.regression.LabeledPoint; +import org.apache.spark.mllib.util.MLUtils; +import org.apache.spark.rdd.RDDimport; + +RDD[LabeledPoint] training = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") +{% endhighlight %} +
      +
      + +## Local matrix + +A local matrix has integer-typed row and column indices and double-typed values, stored on a single +machine. MLlib supports dense matrix, whose entry values are stored in a single double array in +column major. For example, the following matrix `\[ \begin{pmatrix} +1.0 & 2.0 \\ +3.0 & 4.0 \\ +5.0 & 6.0 +\end{pmatrix} +\]` +is stored in a one-dimensional array `[1.0, 3.0, 5.0, 2.0, 4.0, 6.0]` with the matrix size `(3, 2)`. +We are going to add sparse matrix in the next release. + +
      +
      + +The base class of local matrices is +[`Matrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one +implementation: [`DenseMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseMatrix). +Sparse matrix will be added in the next release. We recommend using the factory methods implemented +in [`Matrices`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrices) to create local +matrices. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.{Matrix, Matrices} + +// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +val dm: Matrix = Matrices.dense(3, 2, Array(1.0, 3.0, 5.0, 2.0, 4.0, 6.0)) +{% endhighlight %} +
      + +
      + +The base class of local matrices is +[`Matrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrix), and we provide one +implementation: [`DenseMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.DenseMatrix). +Sparse matrix will be added in the next release. We recommend using the factory methods implemented +in [`Matrices`](api/mllib/index.html#org.apache.spark.mllib.linalg.Matrices) to create local +matrices. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Matrix; +import org.apache.spark.mllib.linalg.Matrices; + +// Create a dense matrix ((1.0, 2.0), (3.0, 4.0), (5.0, 6.0)) +Matrix dm = Matrices.dense(3, 2, new double[] {1.0, 3.0, 5.0, 2.0, 4.0, 6.0}); +{% endhighlight %} +
      + +
      + +## Distributed matrix + +A distributed matrix has long-typed row and column indices and double-typed values, stored +distributively in one or more RDDs. It is very important to choose the right format to store large +and distributed matrices. Converting a distributed matrix to a different format may require a +global shuffle, which is quite expensive. We implemented three types of distributed matrices in +this release and will add more types in the future. + +***Note*** + +The underlying RDDs of a distributed matrix must be deterministic, because we cache the matrix size. +It is always error-prone to have non-deterministic RDDs. + +### RowMatrix + +A `RowMatrix` is a row-oriented distributed matrix without meaningful row indices, backed by an RDD +of its rows, where each row is a local vector. This is similar to `data matrix` in the context of +multivariate statistics. Since each row is represented by a local vector, the number of columns is +limited by the integer range but it should be much smaller in practice. + +
      +
      + +A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be +created from an `RDD[Vector]` instance. Then we can compute its column summary statistics. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Vector +import org.apache.spark.mllib.linalg.distributed.RowMatrix + +val rows: RDD[Vector] = ... // an RDD of local vectors +// Create a RowMatrix from an RDD[Vector]. +val mat: RowMatrix = new RowMatrix(rows) + +// Get its size. +val m = mat.numRows() +val n = mat.numCols() +{% endhighlight %} +
      + +
      + +A [`RowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.RowMatrix) can be +created from a `JavaRDD` instance. Then we can compute its column summary statistics. + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; + +JavaRDD rows = ... // a JavaRDD of local vectors +// Create a RowMatrix from an JavaRDD. +RowMatrix mat = new RowMatrix(rows.rdd()); + +// Get its size. +long m = mat.numRows(); +long n = mat.numCols(); +{% endhighlight %} +
      +
      + +#### Multivariate summary statistics + +We provide column summary statistics for `RowMatrix`. +If the number of columns is not large, say, smaller than 3000, you can also compute +the covariance matrix as a local matrix, which requires $\mathcal{O}(n^2)$ storage where $n$ is the +number of columns. The total CPU time is $\mathcal{O}(m n^2)$, where $m$ is the number of rows, +which could be faster if the rows are sparse. + +
      +
      + +`RowMatrix#computeColumnSummaryStatistics` returns an instance of +[`MultivariateStatisticalSummary`](api/mllib/index.html#org.apache.spark.mllib.stat.MultivariateStatisticalSummary), +which contains the column-wise max, min, mean, variance, and number of nonzeros, as well as the +total count. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.Matrix +import org.apache.spark.mllib.linalg.distributed.RowMatrix +import org.apache.spark.mllib.stat.MultivariateStatisticalSummary + +val mat: RowMatrix = ... // a RowMatrix + +// Compute column summary statistics. +val summary: MultivariateStatisticalSummary = mat.computeColumnSummaryStatistics() +println(summary.mean) // a dense vector containing the mean value for each column +println(summary.variance) // column-wise variance +println(summary.numNonzers) // number of nonzeros in each column + +// Compute the covariance matrix. +val Cov: Matrix = mat.computeCovariance() +{% endhighlight %} +
      +
      + +### IndexedRowMatrix + +An `IndexedRowMatrix` is similar to a `RowMatrix` but with meaningful row indices. It is backed by +an RDD of indexed rows, which each row is represented by its index (long-typed) and a local vector. + +
      +
      + +An +[`IndexedRowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix) +can be created from an `RDD[IndexedRow]` instance, where +[`IndexedRow`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRow) is a +wrapper over `(Long, Vector)`. An `IndexedRowMatrix` can be converted to a `RowMatrix` by dropping +its row indices. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.distributed.{IndexedRow, IndexedRowMatrix, RowMatrix} + +val rows: RDD[IndexedRow] = ... // an RDD of indexed rows +// Create an IndexedRowMatrix from an RDD[IndexedRow]. +val mat: IndexedRowMatrix = new IndexedRowMatrix(rows) + +// Get its size. +val m = mat.numRows() +val n = mat.numCols() + +// Drop its row indices. +val rowMat: RowMatrix = mat.toRowMatrix() +{% endhighlight %} +
      + +
      + +An +[`IndexedRowMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix) +can be created from an `JavaRDD` instance, where +[`IndexedRow`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.IndexedRow) is a +wrapper over `(long, Vector)`. An `IndexedRowMatrix` can be converted to a `RowMatrix` by dropping +its row indices. + +{% highlight java %} +import org.apache.spark.mllib.linalg.distributed.IndexedRow; +import org.apache.spark.mllib.linalg.distributed.IndexedRowMatrix; +import org.apache.spark.mllib.linalg.distributed.RowMatrix; + +JavaRDD[IndexedRow] rows = ... // a JavaRDD of indexed rows +// Create an IndexedRowMatrix from a JavaRDD. +IndexedRowMatrix mat = new IndexedRowMatrix(rows.rdd()); + +// Get its size. +long m = mat.numRows(); +long n = mat.numCols(); + +// Drop its row indices. +RowMatrix rowMat = mat.toRowMatrix(); +{% endhighlight %} +
      + +### CoordinateMatrix + +A `CoordinateMatrix` is a distributed matrix backed by an RDD of its entries. Each entry is a tuple +of `(i: Long, j: Long, value: Double)`, where `i` is the row index, `j` is the column index, and +`value` is the entry value. A `CoordinateMatrix` should be used only in the case when both +dimensions of the matrix are huge and the matrix is very sparse. + +
      +
      + +A +[`CoordinateMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) +can be created from an `RDD[MatrixEntry]` instance, where +[`MatrixEntry`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a +wrapper over `(Long, Long, Double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. In this release, we do not provide other +computation for `CoordinateMatrix`. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.distributed.{CoordinateMatrix, MatrixEntry} + +val entries: RDD[MatrixEntry] = ... // an RDD of matrix entries +// Create a CoordinateMatrix from an RDD[MatrixEntry]. +val mat: CoordinateMatrix = new CoordinateMatrix(entries) + +// Get its size. +val m = mat.numRows() +val n = mat.numCols() + +// Convert it to an IndexRowMatrix whose rows are sparse vectors. +val indexedRowMatrix = mat.toIndexedRowMatrix() +{% endhighlight %} +
      + +
      + +A +[`CoordinateMatrix`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.CoordinateMatrix) +can be created from a `JavaRDD` instance, where +[`MatrixEntry`](api/mllib/index.html#org.apache.spark.mllib.linalg.distributed.MatrixEntry) is a +wrapper over `(long, long, double)`. A `CoordinateMatrix` can be converted to a `IndexedRowMatrix` +with sparse rows by calling `toIndexedRowMatrix`. + +{% highlight scala %} +import org.apache.spark.mllib.linalg.distributed.CoordinateMatrix; +import org.apache.spark.mllib.linalg.distributed.MatrixEntry; + +JavaRDD entries = ... // a JavaRDD of matrix entries +// Create a CoordinateMatrix from a JavaRDD. +CoordinateMatrix mat = new CoordinateMatrix(entries); + +// Get its size. +long m = mat.numRows(); +long n = mat.numCols(); + +// Convert it to an IndexRowMatrix whose rows are sparse vectors. +IndexedRowMatrix indexedRowMatrix = mat.toIndexedRowMatrix(); +{% endhighlight %} +
      +
      diff --git a/docs/mllib-classification-regression.md b/docs/mllib-classification-regression.md deleted file mode 100644 index 2e0fa093dccaa..0000000000000 --- a/docs/mllib-classification-regression.md +++ /dev/null @@ -1,568 +0,0 @@ ---- -layout: global -title: MLlib - Classification and Regression ---- - -* Table of contents -{:toc} - - -`\[ -\newcommand{\R}{\mathbb{R}} -\newcommand{\E}{\mathbb{E}} -\newcommand{\x}{\mathbf{x}} -\newcommand{\y}{\mathbf{y}} -\newcommand{\wv}{\mathbf{w}} -\newcommand{\av}{\mathbf{\alpha}} -\newcommand{\bv}{\mathbf{b}} -\newcommand{\N}{\mathbb{N}} -\newcommand{\id}{\mathbf{I}} -\newcommand{\ind}{\mathbf{1}} -\newcommand{\0}{\mathbf{0}} -\newcommand{\unit}{\mathbf{e}} -\newcommand{\one}{\mathbf{1}} -\newcommand{\zero}{\mathbf{0}} -\]` - - -# Supervised Machine Learning -Supervised machine learning is the setting where we are given a set of training data examples -`$\{\x_i\}$`, each example `$\x_i$` coming with a corresponding label `$y_i$`. -Given the training data `$\{(\x_i,y_i)\}$`, we want to learn a function to predict these labels. -The two most well known classes of methods are -[classification](http://en.wikipedia.org/wiki/Statistical_classification), and -[regression](http://en.wikipedia.org/wiki/Regression_analysis). -In classification, the label is a category (e.g. whether or not emails are spam), whereas in -regression, the label is real value, and we want our prediction to be as close to the true value -as possible. - -Supervised Learning involves executing a learning *Algorithm* on a set of *labeled* training -examples. The algorithm returns a trained *Model* (such as for example a linear function) that -can predict the label for new data examples for which the label is unknown. - -## Discriminative Training using Linear Methods - -### Mathematical Formulation -Many standard *machine learning* methods can be formulated as a convex optimization problem, i.e. -the task of finding a minimizer of a convex function `$f$` that depends on a variable vector -`$\wv$` (called `weights` in the code), which has `$d$` entries. -Formally, we can write this as the optimization problem `$\min_{\wv \in\R^d} \; f(\wv)$`, where -the objective function is of the form -`\begin{equation} - f(\wv) := - \lambda\, R(\wv) + - \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) - \label{eq:regPrimal} - \ . -\end{equation}` -Here the vectors `$\x_i\in\R^d$` are the training data examples, for `$1\le i\le n$`, and -`$y_i\in\R$` are their corresponding labels, which we want to predict. - -The objective function `$f$` has two parts: -The *loss-function* measures the error of the model on the training data. The loss-function -`$L(\wv;.)$` must be a convex function in `$\wv$`. -The purpose of the [regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to -encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid -over-fitting. -Usually, the regularizer `$R(.)$` is chosen as either the standard (Euclidean) L2-norm, `$R(\wv) -:= \frac{1}{2}\|\wv\|^2$`, or the L1-norm, `$R(\wv) := \|\wv\|_1$`, see -[below](#using-different-regularizers) for more details. - -The fixed regularization parameter `$\lambda\ge0$` (`regParam` in the code) defines the trade-off -between the two goals of small loss and small model complexity. - - -### Binary Classification - -**Input:** Datapoints `$\x_i\in\R^{d}$`, labels `$y_i\in\{+1,-1\}$`, for `$1\le i\le n$`. - -**Distributed Datasets.** -For all currently implemented optimization methods for classification, the data must be -distributed between processes on the worker machines *by examples*. Machines hold consecutive -blocks of the `$n$` example/label pairs `$(\x_i,y_i)$`. -In other words, the input distributed dataset -([RDD](scala-programming-guide.html#resilient-distributed-datasets-rdds)) must be the set of -vectors `$\x_i\in\R^d$`. - -#### Support Vector Machine -The linear [Support Vector Machine (SVM)](http://en.wikipedia.org/wiki/Support_vector_machine) -has become a standard choice for classification tasks. -Here the loss function in formulation `$\eqref{eq:regPrimal}$` is given by the hinge-loss -`\[ -L(\wv;\x_i,y_i) := \max \{0, 1-y_i \wv^T \x_i \} \ . -\]` - -By default, SVMs are trained with an L2 regularization, which gives rise to the large-margin -interpretation if these classifiers. We also support alternative L1 regularization. In this case, -the primal optimization problem becomes an [LP](http://en.wikipedia.org/wiki/Linear_programming). - -#### Logistic Regression -Despite its name, [Logistic Regression](http://en.wikipedia.org/wiki/Logistic_regression) is a -binary classification method, again when the labels are given by binary values -`$y_i\in\{+1,-1\}$`. The logistic loss function in formulation `$\eqref{eq:regPrimal}$` is -defined as -`\[ -L(\wv;\x_i,y_i) := \log(1+\exp( -y_i \wv^T \x_i)) \ . -\]` - - -### Linear Regression (Least Squares, Lasso and Ridge Regression) - -**Input:** Data matrix `$A\in\R^{n\times d}$`, right hand side vector `$\y\in\R^n$`. - -**Distributed Datasets.** -For all currently implemented optimization methods for regression, the data matrix -`$A\in\R^{n\times d}$` must be distributed between the worker machines *by rows* of `$A$`. In -other words, the input distributed dataset -([RDD](scala-programming-guide.html#resilient-distributed-datasets-rdds)) must be the set of the -`$n$` rows `$A_{i:}$` of `$A$`. - -Least Squares Regression refers to the setting where we try to fit a vector `$\y\in\R^n$` by -linear combination of our observed data `$A\in\R^{n\times d}$`, which is given as a matrix. - -It comes in 3 flavors: - -#### Least Squares -Plain old [least squares](http://en.wikipedia.org/wiki/Least_squares) linear regression is the -problem of minimizing - `\[ f_{\text{LS}}(\wv) := \frac1n \|A\wv-\y\|_2^2 \ . \]` - -#### Lasso -The popular [Lasso](http://en.wikipedia.org/wiki/Lasso_(statistics)#Lasso_method) (alternatively -also known as `$L_1$`-regularized least squares regression) is given by - `\[ f_{\text{Lasso}}(\wv) := \frac1n \|A\wv-\y\|_2^2 + \lambda \|\wv\|_1 \ . \]` - -#### Ridge Regression -[Ridge regression](http://en.wikipedia.org/wiki/Ridge_regression) uses the same loss function but -with a L2 regularizer term: - `\[ f_{\text{Ridge}}(\wv) := \frac1n \|A\wv-\y\|_2^2 + \frac{\lambda}{2}\|\wv\|^2 \ . \]` - -**Loss Function.** -For all 3, the loss function (i.e. the measure of model fit) is given by the squared deviations -from the right hand side `$\y$`. -`\[ -\frac1n \|A\wv-\y\|_2^2 -= \frac1n \sum_{i=1}^n (A_{i:} \wv - y_i )^2 -= \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) -\]` -This is also known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). -In our generic problem formulation `$\eqref{eq:regPrimal}$`, this means the loss function is -`$L(\wv;\x_i,y_i) := (A_{i:} \wv - y_i )^2$`, each depending only on a single row `$A_{i:}$` of -the data matrix `$A$`. - - -### Using Different Regularizers - -As we have mentioned above, the purpose of *regularizer* in `$\eqref{eq:regPrimal}$` is to -encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid -over-fitting. -All machine learning methods for classification and regression that we have mentioned above are -of interest for different types of regularization, the 3 most common ones being - -* **L2-Regularization.** -`$R(\wv) := \frac{1}{2}\|\wv\|^2$`. -This regularizer is most commonly used for SVMs, logistic regression and ridge regression. - -* **L1-Regularization.** -`$R(\wv) := \|\wv\|_1$`. The L1 norm `$\|\wv\|_1$` is the sum of the absolut values of the -entries of a vector `$\wv$`. -This regularizer is most commonly used for sparse methods, and feature selection, such as the -Lasso. - -* **Non-Regularized.** -`$R(\wv):=0$`. -Of course we can also train the models without any regularization, or equivalently by setting the -regularization parameter `$\lambda:=0$`. - -The optimization problems of the form `$\eqref{eq:regPrimal}$` with convex regularizers such as -the 3 mentioned here can be conveniently optimized with gradient descent type methods (such as -SGD) which is implemented in `MLlib` currently, and explained in the next section. - - -### Optimization Methods Working on the Primal Formulation - -**Stochastic subGradient Descent (SGD).** -For optimization objectives `$f$` written as a sum, *stochastic subgradient descent (SGD)* can be -an efficient choice of optimization method, as we describe in the optimization section in more detail. -Because all methods considered here fit into the optimization formulation -`$\eqref{eq:regPrimal}$`, this is especially natural, because the loss is written as an average -of the individual losses coming from each datapoint. - -Picking one datapoint `$i\in[1..n]$` uniformly at random, we obtain a stochastic subgradient of -`$\eqref{eq:regPrimal}$`, with respect to `$\wv$` as follows: -`\[ -f'_{\wv,i} := L'_{\wv,i} + \lambda\, R'_\wv \ , -\]` -where `$L'_{\wv,i} \in \R^d$` is a subgradient of the part of the loss function determined by the -`$i$`-th datapoint, that is `$L'_{\wv,i} \in \frac{\partial}{\partial \wv} L(\wv;\x_i,y_i)$`. -Furthermore, `$R'_\wv$` is a subgradient of the regularizer `$R(\wv)$`, i.e. `$R'_\wv \in -\frac{\partial}{\partial \wv} R(\wv)$`. The term `$R'_\wv$` does not depend on which random -datapoint is picked. - - - -**Gradients.** -The following table summarizes the gradients (or subgradients) of all loss functions and -regularizers that we currently support: - - - - - - - - - - - - - - - - - - - - - - - - - - - - -
      FunctionStochastic (Sub)Gradient
      SVM Hinge Loss$L(\wv;\x_i,y_i) := \max \{0, 1-y_i \wv^T \x_i \}$$L'_{\wv,i} = \begin{cases}-y_i \x_i & \text{if $y_i \wv^T \x_i <1$}, \\ 0 & -\text{otherwise}.\end{cases}$
      Logistic Loss$L(\wv;\x_i,y_i) := \log(1+\exp( -y_i \wv^T \x_i))$$L'_{\wv,i} = -y_i \x_i \left(1-\frac1{1+\exp(-y_i \wv^T \x_i)} \right)$
      Least Squares Loss$L(\wv;\x_i,y_i) := (A_{i:} \wv - y_i)^2$$L'_{\wv,i} = 2 A_{i:}^T (A_{i:} \wv - y_i)$
      Non-Regularized$R(\wv) := 0$$R'_\wv = \0$
      L2 Regularizer$R(\wv) := \frac{1}{2}\|\wv\|^2$$R'_\wv = \wv$
      L1 Regularizer$R(\wv) := \|\wv\|_1$$R'_\wv = \mathop{sign}(\wv)$
      - -Here `$\mathop{sign}(\wv)$` is the vector consisting of the signs (`$\pm1$`) of all the entries -of `$\wv$`. -Also, note that `$A_{i:} \in \R^d$` is a row-vector, but the gradient is a column vector. - -## Decision Tree Classification and Regression - -Decision trees and their ensembles are popular methods for the machine learning tasks of classification and regression. Decision trees are widely used since they are easy to interpret, handle categorical variables, extend to the multi-class classification setting, do not require feature scaling and are able to capture non-linearities and feature interactions. Tree ensemble algorithms such as decision forest and boosting are among the top performers for classification and regression tasks. - -### Basic Algorithm - -The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature space by choosing a single element from the *best split set* where each element of the set maximimizes the information gain at a tree node. In other words, the split chosen at each tree node is chosen from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` is the information gain when a split `$s$` is applied to a dataset `$D$`. - -#### Node Impurity and Information Gain - -The *node impurity* is a measure of the homogeneity of the labels at the node. The current implementation provides two impurity measures for classification (Gini index and entropy) and one impurity measure for regression (variance). - - - - - - - - - - - - - - - - -
      ImpurityTaskFormulaDescription
      Gini indexClassification$\sum_{i=1}^{M} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
      EntropyClassification$\sum_{i=1}^{M} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
      VarianceClassification$\frac{1}{n} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^n x_i$.
      - -The *information gain* is the difference in the parent node impurity and the weighted sum of the two child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` into two datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, respectively: - -`$IG(D,s) = Impurity(D) - \frac{N_{left}}{N} Impurity(D_{left}) - \frac{N_{right}}{N} Impurity(D_{right})$` - -#### Split Candidates - -**Continuous Features** - -For small datasets in single machine implementations, the split candidates for each continuous feature are typically the unique values for the feature. Some implementations sort the feature values and then use the ordered unique values as split candidates for faster tree calculations. - -Finding ordered unique feature values is computationally intensive for large distributed datasets. One can get an approximate set of split candidates by performing a quantile calculation over a sampled fraction of the data. The ordered splits create "bins" and the maximum number of such bins can be specified using the `maxBins` parameters. - -Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of bins if the condition is not satisfied. - -**Categorical Features** - -For `$M$` categorical features, one could come up with `$2^M-1$` split candidates. However, for binary classification, the number of split candidates can be reduced to `$M-1$` by ordering the categorical feature values by the proportion of labels falling in one of the two classes (see Section 9.2.4 in [Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for details). For example, for a binary classification problem with one categorical feature with three categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical features are orded as A followed by C followed B or A, B, C. The two split candidates are A \| C, B and A , B \| C where \| denotes the split. - -#### Stopping Rule - -The recursive tree construction is stopped at a node when one of the two conditions is met: - -1. The node depth is equal to the `maxDepth` training paramemter -2. No split candidate leads to an information gain at the node. - -### Practical Limitations - -The tree implementation stores an Array[Double] of size *O(#features \* #splits \* 2^maxDepth)* in memory for aggregating histograms over partitions. The current implementation might not scale to very deep trees since the memory requirement grows exponentially with tree depth. - -Please drop us a line if you encounter any issues. We are planning to solve this problem in the near future and real-world examples will be great. - - -## Implementation in MLlib - -#### Linear Methods - -For both classification and regression algorithms with convex loss functions, `MLlib` implements a simple distributed version of -stochastic subgradient descent (SGD), building on the underlying gradient descent primitive (as -described in the -optimization section). -All provided algorithms take as input a regularization parameter (`regParam`) along with various -parameters associated with stochastic gradient -descent (`stepSize`, `numIterations`, `miniBatchFraction`). -For each of them, we support all 3 possible regularizations (none, L1 or L2). - -Available algorithms for binary classification: - -* [SVMWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.SVMWithSGD) -* [LogisticRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) - -Available algorithms for linear regression: - -* [LinearRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) -* [RidgeRegressionWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) -* [LassoWithSGD](api/scala/index.html#org.apache.spark.mllib.regression.LassoWithSGD) - -Behind the scenes, all above methods use the SGD implementation from the -gradient descent primitive in MLlib, see the -optimization part: - -* [GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) - -#### Tree-based Methods - -The decision tree algorithm supports binary classification and regression: - -* [DecisionTee](api/scala/index.html#org.apache.spark.mllib.tree.DecisionTree) - - -# Usage in Scala - -Following code snippets can be executed in `spark-shell`. - -## Linear Methods - - -#### Binary Classification - -The following code snippet illustrates how to load a sample dataset, execute a -training algorithm on this training data using a static method in the algorithm -object, and make predictions with the resulting model to compute the training -error. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.classification.SVMWithSGD -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_svm_data.txt") -val parsedData = data.map { line => - val parts = line.split(' ').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} - -// Run training algorithm to build the model -val numIterations = 100 -val model = SVMWithSGD.train(parsedData, numIterations) - -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count -println("Training Error = " + trainErr) -{% endhighlight %} - - -The `SVMWithSGD.train()` method by default performs L2 regularization with the -regularization parameter set to 1.0. If we want to configure this algorithm, we -can customize `SVMWithSGD` further by creating a new object directly and -calling setter methods. All other MLlib algorithms support customization in -this way as well. For example, the following code produces an L1 regularized -variant of SVMs with regularization parameter set to 0.1, and runs the training -algorithm for 200 iterations. - -{% highlight scala %} -import org.apache.spark.mllib.optimization.L1Updater - -val svmAlg = new SVMWithSGD() -svmAlg.optimizer.setNumIterations(200) - .setRegParam(0.1) - .setUpdater(new L1Updater) -val modelL1 = svmAlg.run(parsedData) -{% endhighlight %} - -#### Linear Regression - -The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. -The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -{% highlight scala %} -import org.apache.spark.mllib.regression.LinearRegressionWithSGD -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors - -// Load and parse the data -val data = sc.textFile("mllib/data/ridge-data/lpsa.data") -val parsedData = data.map { line => - val parts = line.split(',') - LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) -} - -// Building the model -val numIterations = 100 -val model = LinearRegressionWithSGD.train(parsedData, numIterations) - -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.reduce(_ + _) / valuesAndPreds.count -println("training Mean Squared Error = " + MSE) -{% endhighlight %} - - -Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training -[Mean Squared Errors](http://en.wikipedia.org/wiki/Mean_squared_error). - -## Decision Tree - -#### Classification - -The example below demonstrates how to load a CSV file, parse it as an RDD of LabeledPoint and then perform classification using a decision tree using Gini index as an impurity measure and a maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy. - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Gini - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} - -// Run training algorithm to build the model -val maxDepth = 5 -val model = DecisionTree.train(parsedData, Classification, Gini, maxDepth) - -// Evaluate model on training examples and compute training error -val labelAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count -println("Training Error = " + trainErr) -{% endhighlight %} - -#### Regression - -The example below demonstrates how to load a CSV file, parse it as an RDD of LabeledPoint and then perform regression using a decision tree using variance as an impurity measure and a maximum tree depth of 5. The Mean Squared Error is computed at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -{% highlight scala %} -import org.apache.spark.SparkContext -import org.apache.spark.mllib.tree.DecisionTree -import org.apache.spark.mllib.regression.LabeledPoint -import org.apache.spark.mllib.linalg.Vectors -import org.apache.spark.mllib.tree.configuration.Algo._ -import org.apache.spark.mllib.tree.impurity.Variance - -// Load and parse the data file -val data = sc.textFile("mllib/data/sample_tree_data.csv") -val parsedData = data.map { line => - val parts = line.split(',').map(_.toDouble) - LabeledPoint(parts(0), Vectors.dense(parts.tail)) -} - -// Run training algorithm to build the model -val maxDepth = 5 -val model = DecisionTree.train(parsedData, Regression, Variance, maxDepth) - -// Evaluate model on training examples and compute training error -val valuesAndPreds = parsedData.map { point => - val prediction = model.predict(point.features) - (point.label, prediction) -} -val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count -println("training Mean Squared Error = " + MSE) -{% endhighlight %} - - -# Usage in Java - -All of MLlib's methods use Java-friendly types, so you can import and call them there the same -way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the -Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by -calling `.rdd()` on your `JavaRDD` object. - -# Usage in Python - -Following examples can be tested in the PySpark shell. - -## Linear Methods - -### Binary Classification -The following example shows how to load a sample dataset, build Logistic Regression model, -and make predictions with the resulting model to compute the training error. - -{% highlight python %} -from pyspark.mllib.classification import LogisticRegressionWithSGD -from pyspark.mllib.regression import LabeledPoint -from numpy import array - -# Load and parse the data -def parsePoint(line): - values = [float(x) for x in line.split(' ')] - return LabeledPoint(values[0], values[1:]) - -data = sc.textFile("mllib/data/sample_svm_data.txt") -parsedData = data.map(parsePoint) - -# Build the model -model = LogisticRegressionWithSGD.train(parsedData) - -# Evaluating the model on training data -labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) -trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) -print("Training Error = " + str(trainErr)) -{% endhighlight %} - -### Linear Regression -The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. -The example then uses LinearRegressionWithSGD to build a simple linear model to predict label -values. We compute the Mean Squared Error at the end to evaluate -[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). - -{% highlight python %} -from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD -from numpy import array - -# Load and parse the data -def parsePoint(line): - values = [float(x) for x in line.replace(',', ' ').split(' ')] - return LabeledPoint(values[0], values[1:]) - -data = sc.textFile("mllib/data/ridge-data/lpsa.data") -parsedData = data.map(parsePoint) - -# Build the model -model = LinearRegressionWithSGD.train(parsedData) - -# Evaluate the model on training data -valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) -MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y) / valuesAndPreds.count() -print("Mean Squared Error = " + str(MSE)) -{% endhighlight %} diff --git a/docs/mllib-clustering.md b/docs/mllib-clustering.md index 0359c67157168..b3293afe408d0 100644 --- a/docs/mllib-clustering.md +++ b/docs/mllib-clustering.md @@ -1,19 +1,21 @@ --- layout: global -title: MLlib - Clustering +title: MLlib - Clustering --- * Table of contents {:toc} -# Clustering +## Clustering Clustering is an unsupervised learning problem whereby we aim to group subsets of entities with one another based on some notion of similarity. Clustering is often used for exploratory analysis and/or as a component of a hierarchical supervised learning pipeline (in which distinct classifiers or regression -models are trained for each cluster). MLlib supports +models are trained for each cluster). + +MLlib supports [k-means](http://en.wikipedia.org/wiki/K-means_clustering) clustering, one of the most commonly used clustering algorithms that clusters the data points into predfined number of clusters. The MLlib implementation includes a parallelized @@ -31,17 +33,14 @@ a given dataset, the algorithm returns the best clustering result). * *initializiationSteps* determines the number of steps in the k-means\|\| algorithm. * *epsilon* determines the distance threshold within which we consider k-means to have converged. -Available algorithms for clustering: - -* [KMeans](api/scala/index.html#org.apache.spark.mllib.clustering.KMeans) - - - -# Usage in Scala +## Examples +
      +
      Following code snippets can be executed in `spark-shell`. -In the following example after loading and parsing data, we use the KMeans object to cluster the data +In the following example after loading and parsing data, we use the +[`KMeans`](api/mllib/index.html#org.apache.spark.mllib.clustering.KMeans) object to cluster the data into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the optimal *k* is usually one where there is an "elbow" in the WSSSE graph. @@ -63,22 +62,22 @@ val clusters = KMeans.train(parsedData, numClusters, numIterations) val WSSSE = clusters.computeCost(parsedData) println("Within Set Sum of Squared Errors = " + WSSSE) {% endhighlight %} +
      - -# Usage in Java - +
      All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. +
      -# Usage in Python +
      Following examples can be tested in the PySpark shell. -In the following example after loading and parsing data, we use the KMeans object to cluster the data -into two clusters. The number of desired clusters is passed to the algorithm. We then compute Within -Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In fact the -optimal *k* is usually one where there is an "elbow" in the WSSSE graph. +In the following example after loading and parsing data, we use the KMeans object to cluster the +data into two clusters. The number of desired clusters is passed to the algorithm. We then compute +Within Set Sum of Squared Error (WSSSE). You can reduce this error measure by increasing *k*. In +fact the optimal *k* is usually one where there is an "elbow" in the WSSSE graph. {% highlight python %} from pyspark.mllib.clustering import KMeans @@ -91,7 +90,7 @@ parsedData = data.map(lambda line: array([float(x) for x in line.split(' ')])) # Build the model (cluster the data) clusters = KMeans.train(parsedData, 2, maxIterations=10, - runs=10, initialization_mode="random") + runs=10, initializationMode="random") # Evaluate clustering by computing Within Set Sum of Squared Errors def error(point): @@ -101,7 +100,6 @@ def error(point): WSSSE = parsedData.map(lambda point: error(point)).reduce(lambda x, y: x + y) print("Within Set Sum of Squared Error = " + str(WSSSE)) {% endhighlight %} +
      -Similarly you can use RidgeRegressionWithSGD and LassoWithSGD and compare training Mean Squared -Errors. - +
      diff --git a/docs/mllib-collaborative-filtering.md b/docs/mllib-collaborative-filtering.md index 2f1f5f3856efe..79f5e3a7ca4fb 100644 --- a/docs/mllib-collaborative-filtering.md +++ b/docs/mllib-collaborative-filtering.md @@ -1,12 +1,12 @@ --- layout: global -title: MLlib - Collaborative Filtering +title: MLlib - Collaborative Filtering --- * Table of contents {:toc} -# Collaborative Filtering +## Collaborative filtering [Collaborative filtering](http://en.wikipedia.org/wiki/Recommender_system#Collaborative_filtering) is commonly used for recommender systems. These techniques aim to fill in the @@ -14,44 +14,43 @@ missing entries of a user-item association matrix. MLlib currently supports model-based collaborative filtering, in which users and products are described by a small set of latent factors that can be used to predict missing entries. In particular, we implement the [alternating least squares -(ALS)](http://www2.research.att.com/~volinsky/papers/ieeecomputer.pdf) +(ALS)](http://dl.acm.org/citation.cfm?id=1608614) algorithm to learn these latent factors. The implementation in MLlib has the following parameters: -* *numBlocks* is the number of blacks used to parallelize computation (set to -1 to auto-configure). +* *numBlocks* is the number of blocks used to parallelize computation (set to -1 to auto-configure). * *rank* is the number of latent factors in our model. * *iterations* is the number of iterations to run. * *lambda* specifies the regularization parameter in ALS. -* *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for *implicit feedback* data -* *alpha* is a parameter applicable to the implicit feedback variant of ALS that governs the *baseline* confidence in preference observations +* *implicitPrefs* specifies whether to use the *explicit feedback* ALS variant or one adapted for + *implicit feedback* data. +* *alpha* is a parameter applicable to the implicit feedback variant of ALS that governs the + *baseline* confidence in preference observations. -## Explicit vs Implicit Feedback +### Explicit vs. implicit feedback The standard approach to matrix factorization based collaborative filtering treats the entries in the user-item matrix as *explicit* preferences given by the user to the item. -It is common in many real-world use cases to only have access to *implicit feedback* -(e.g. views, clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with -such data is taken from -[Collaborative Filtering for Implicit Feedback Datasets](http://www2.research.att.com/~yifanhu/PUB/cf.pdf). -Essentially instead of trying to model the matrix of ratings directly, this approach treats the data as -a combination of binary preferences and *confidence values*. The ratings are then related -to the level of confidence in observed user preferences, rather than explicit ratings given to items. -The model then tries to find latent factors that can be used to predict the expected preference of a user -for an item. +It is common in many real-world use cases to only have access to *implicit feedback* (e.g. views, +clicks, purchases, likes, shares etc.). The approach used in MLlib to deal with such data is taken +from +[Collaborative Filtering for Implicit Feedback Datasets](http://dx.doi.org/10.1109/ICDM.2008.22). +Essentially instead of trying to model the matrix of ratings directly, this approach treats the data +as a combination of binary preferences and *confidence values*. The ratings are then related to the +level of confidence in observed user preferences, rather than explicit ratings given to items. The +model then tries to find latent factors that can be used to predict the expected preference of a +user for an item. -Available algorithms for collaborative filtering: +## Examples -* [ALS](api/scala/index.html#org.apache.spark.mllib.recommendation.ALS) - - -# Usage in Scala - -Following code snippets can be executed in `spark-shell`. +
      +
      In the following example we load rating data. Each row consists of a user, a product and a rating. -We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation -model by measuring the Mean Squared Error of rating prediction. +We use the default [ALS.train()](api/mllib/index.html#org.apache.spark.mllib.recommendation.ALS$) +method which assumes ratings are explicit. We evaluate the +recommendation model by measuring the Mean Squared Error of rating prediction. {% highlight scala %} import org.apache.spark.mllib.recommendation.ALS @@ -64,8 +63,9 @@ val ratings = data.map(_.split(',') match { }) // Build the recommendation model using ALS +val rank = 10 val numIterations = 20 -val model = ALS.train(ratings, 1, 20, 0.01) +val model = ALS.train(ratings, rank, numIterations, 0.01) // Evaluate the model on rating data val usersProducts = ratings.map{ case Rating(user, product, rate) => (user, product)} @@ -85,19 +85,19 @@ If the rating matrix is derived from other source of information (i.e., it is in other signals), you can use the trainImplicit method to get better results. {% highlight scala %} -val model = ALS.trainImplicit(ratings, 1, 20, 0.01) +val alpha = 0.01 +val model = ALS.trainImplicit(ratings, rank, numIterations, alpha) {% endhighlight %} +
      -# Usage in Java - +
      All of MLlib's methods use Java-friendly types, so you can import and call them there the same way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by calling `.rdd()` on your `JavaRDD` object. +
      -# Usage in Python -Following examples can be tested in the PySpark shell. - +
      In the following example we load rating data. Each row consists of a user, a product and a rating. We use the default ALS.train() method which assumes ratings are explicit. We evaluate the recommendation by measuring the Mean Squared Error of rating prediction. @@ -111,7 +111,9 @@ data = sc.textFile("mllib/data/als/test.data") ratings = data.map(lambda line: array([float(x) for x in line.split(',')])) # Build the recommendation model using Alternating Least Squares -model = ALS.train(ratings, 1, 20) +rank = 10 +numIterations = 20 +model = ALS.train(ratings, rank, numIterations) # Evaluate the model on training data testdata = ratings.map(lambda p: (int(p[0]), int(p[1]))) @@ -126,5 +128,13 @@ signals), you can use the trainImplicit method to get better results. {% highlight python %} # Build the recommendation model using Alternating Least Squares based on implicit ratings -model = ALS.trainImplicit(ratings, 1, 20) +model = ALS.trainImplicit(ratings, rank, numIterations, alpha = 0.01) {% endhighlight %} +
      + +
      + +## Tutorial + +[AMP Camp](http://ampcamp.berkeley.edu/) provides a hands-on tutorial for +[personalized movie recommendation with MLlib](http://ampcamp.berkeley.edu/big-data-mini-course/movie-recommendation-with-mllib.html). diff --git a/docs/mllib-decision-tree.md b/docs/mllib-decision-tree.md new file mode 100644 index 0000000000000..0693766990732 --- /dev/null +++ b/docs/mllib-decision-tree.md @@ -0,0 +1,185 @@ +--- +layout: global +title: MLlib - Decision Tree +--- + +* Table of contents +{:toc} + +Decision trees and their ensembles are popular methods for the machine learning tasks of +classification and regression. Decision trees are widely used since they are easy to interpret, +handle categorical variables, extend to the multiclass classification setting, do not require +feature scaling and are able to capture nonlinearities and feature interactions. Tree ensemble +algorithms such as decision forest and boosting are among the top performers for classification and +regression tasks. + +## Basic algorithm + +The decision tree is a greedy algorithm that performs a recursive binary partitioning of the feature +space by choosing a single element from the *best split set* where each element of the set maximizes +the information gain at a tree node. In other words, the split chosen at each tree node is chosen +from the set `$\underset{s}{\operatorname{argmax}} IG(D,s)$` where `$IG(D,s)$` is the information +gain when a split `$s$` is applied to a dataset `$D$`. + +### Node impurity and information gain + +The *node impurity* is a measure of the homogeneity of the labels at the node. The current +implementation provides two impurity measures for classification (Gini impurity and entropy) and one +impurity measure for regression (variance). + + + + + + + + + + + + + + + + + + + + + + +
      ImpurityTaskFormulaDescription
      Gini impurityClassification$\sum_{i=1}^{M} f_i(1-f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
      EntropyClassification$\sum_{i=1}^{M} -f_ilog(f_i)$$f_i$ is the frequency of label $i$ at a node and $M$ is the number of unique labels.
      VarianceRegression$\frac{1}{n} \sum_{i=1}^{N} (x_i - \mu)^2$$y_i$ is label for an instance, + $N$ is the number of instances and $\mu$ is the mean given by $\frac{1}{N} \sum_{i=1}^n x_i$.
      + +The *information gain* is the difference in the parent node impurity and the weighted sum of the two +child node impurities. Assuming that a split $s$ partitions the dataset `$D$` of size `$N$` into two +datasets `$D_{left}$` and `$D_{right}$` of sizes `$N_{left}$` and `$N_{right}$`, respectively: + +`$IG(D,s) = Impurity(D) - \frac{N_{left}}{N} Impurity(D_{left}) - \frac{N_{right}}{N} Impurity(D_{right})$` + +### Split candidates + +**Continuous features** + +For small datasets in single machine implementations, the split candidates for each continuous +feature are typically the unique values for the feature. Some implementations sort the feature +values and then use the ordered unique values as split candidates for faster tree calculations. + +Finding ordered unique feature values is computationally intensive for large distributed +datasets. One can get an approximate set of split candidates by performing a quantile calculation +over a sampled fraction of the data. The ordered splits create "bins" and the maximum number of such +bins can be specified using the `maxBins` parameters. + +Note that the number of bins cannot be greater than the number of instances `$N$` (a rare scenario +since the default `maxBins` value is 100). The tree algorithm automatically reduces the number of +bins if the condition is not satisfied. + +**Categorical features** + +For `$M$` categorical features, one could come up with `$2^M-1$` split candidates. However, for +binary classification, the number of split candidates can be reduced to `$M-1$` by ordering the +categorical feature values by the proportion of labels falling in one of the two classes (see +Section 9.2.4 in +[Elements of Statistical Machine Learning](http://statweb.stanford.edu/~tibs/ElemStatLearn/) for +details). For example, for a binary classification problem with one categorical feature with three +categories A, B and C with corresponding proportion of label 1 as 0.2, 0.6 and 0.4, the categorical +features are orded as A followed by C followed B or A, B, C. The two split candidates are A \| C, B +and A , B \| C where \| denotes the split. + +### Stopping rule + +The recursive tree construction is stopped at a node when one of the two conditions is met: + +1. The node depth is equal to the `maxDepth` training parammeter +2. No split candidate leads to an information gain at the node. + +### Practical limitations + +1. The tree implementation stores an Array[Double] of size *O(#features \* #splits \* 2^maxDepth)* + in memory for aggregating histograms over partitions. The current implementation might not scale + to very deep trees since the memory requirement grows exponentially with tree depth. +2. The implemented algorithm reads both sparse and dense data. However, it is not optimized for + sparse input. +3. Python is not supported in this release. + +We are planning to solve these problems in the near future. Please drop us a line if you encounter +any issues. + +## Examples + +### Classification + +The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then +perform classification using a decision tree using Gini impurity as an impurity measure and a +maximum tree depth of 5. The training error is calculated to measure the algorithm accuracy. + +
      +
      +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.impurity.Gini + +// Load and parse the data file +val data = sc.textFile("mllib/data/sample_tree_data.csv") +val parsedData = data.map { line => + val parts = line.split(',').map(_.toDouble) + LabeledPoint(parts(0), Vectors.dense(parts.tail)) +} + +// Run training algorithm to build the model +val maxDepth = 5 +val model = DecisionTree.train(parsedData, Classification, Gini, maxDepth) + +// Evaluate model on training examples and compute training error +val labelAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val trainErr = labelAndPreds.filter(r => r._1 != r._2).count.toDouble / parsedData.count +println("Training Error = " + trainErr) +{% endhighlight %} +
      +
      + +### Regression + +The example below demonstrates how to load a CSV file, parse it as an RDD of `LabeledPoint` and then +perform regression using a decision tree using variance as an impurity measure and a maximum tree +depth of 5. The Mean Squared Error (MSE) is computed at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +
      +
      +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.tree.DecisionTree +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.tree.configuration.Algo._ +import org.apache.spark.mllib.tree.impurity.Variance + +// Load and parse the data file +val data = sc.textFile("mllib/data/sample_tree_data.csv") +val parsedData = data.map { line => + val parts = line.split(',').map(_.toDouble) + LabeledPoint(parts(0), Vectors.dense(parts.tail)) +} + +// Run training algorithm to build the model +val maxDepth = 5 +val model = DecisionTree.train(parsedData, Regression, Variance, maxDepth) + +// Evaluate model on training examples and compute training error +val valuesAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val MSE = valuesAndPreds.map{ case(v, p) => math.pow((v - p), 2)}.reduce(_ + _)/valuesAndPreds.count +println("training Mean Squared Error = " + MSE) +{% endhighlight %} +
      +
      diff --git a/docs/mllib-dimensionality-reduction.md b/docs/mllib-dimensionality-reduction.md new file mode 100644 index 0000000000000..4e9ecf7c006fa --- /dev/null +++ b/docs/mllib-dimensionality-reduction.md @@ -0,0 +1,86 @@ +--- +layout: global +title: MLlib - Dimensionality Reduction +--- + +* Table of contents +{:toc} + +[Dimensionality reduction](http://en.wikipedia.org/wiki/Dimensionality_reduction) is the process +of reducing the number of variables under consideration. +It is used to extract latent features from raw and noisy features, +or compress data while maintaining the structure. +In this release, we provide preliminary support for dimensionality reduction on tall-and-skinny matrices. + +## Singular value decomposition (SVD) + +[Singular value decomposition (SVD)](http://en.wikipedia.org/wiki/Singular_value_decomposition) +factorizes a matrix into three matrices: $U$, $\Sigma$, and $V$ such that + +`\[ +A = U \Sigma V^T, +\]` + +where + +* $U$ is an orthonormal matrix, whose columns are called left singular vectors, +* $\Sigma$ is a diagonal matrix with non-negative diagonals in descending order, + whose diagonals are called singular values, +* $V$ is an orthonormal matrix, whose columns are called right singular vectors. + +For large matrices, usually we don't need the complete factorization but only the top singular +values and its associated singular vectors. This can save storage, and more importantly, de-noise +and recover the low-rank structure of the matrix. + +If we keep the top $k$ singular values, then the dimensions of the return will be: + +* `$U$`: `$m \times k$`, +* `$\Sigma$`: `$k \times k$`, +* `$V$`: `$n \times k$`. + +In this release, we provide SVD computation to row-oriented matrices that have only a few columns, +say, less than $1000$, but many rows, which we call *tall-and-skinny*. + +
      +
      +{% highlight scala %} +val mat: RowMatrix = ... + +// Compute the top 20 singular values and corresponding singular vectors. +val svd: SingularValueDecomposition[RowMatrix, Matrix] = mat.computeSVD(20, computeU = true) +val U: RowMatrix = svd.U // The U factor is a RowMatrix. +val s: Vector = svd.s // The singular values are stored in a local dense vector. +val V: Matrix = svd.V // The V factor is a local dense matrix. +{% endhighlight %} +
      +Same code applies to `IndexedRowMatrix`. +The only difference that the `U` matrix becomes an `IndexedRowMatrix`. +
      + +## Principal component analysis (PCA) + +[Principal component analysis (PCA)](http://en.wikipedia.org/wiki/Principal_component_analysis) is a +statistical method to find a rotation such that the first coordinate has the largest variance +possible, and each succeeding coordinate in turn has the largest variance possible. The columns of +the rotation matrix are called principal components. PCA is used widely in dimensionality reduction. + +In this release, we implement PCA for tall-and-skinny matrices stored in row-oriented format. + +
      +
      + +The following code demonstrates how to compute principal components on a tall-and-skinny `RowMatrix` +and use them to project the vectors into a low-dimensional space. +The number of columns should be small, e.g, less than 1000. + +{% highlight scala %} +val mat: RowMatrix = ... + +// Compute the top 10 principal components. +val pc: Matrix = mat.computePrincipalComponents(10) // Principal components are stored in a local dense matrix. + +// Project the rows to the linear space spanned by the top 10 principal components. +val projected: RowMatrix = mat.multiply(pc) +{% endhighlight %} +
      +
      diff --git a/docs/mllib-guide.md b/docs/mllib-guide.md index 0963a99881c9d..c49f857d07557 100644 --- a/docs/mllib-guide.md +++ b/docs/mllib-guide.md @@ -3,63 +3,121 @@ layout: global title: Machine Learning Library (MLlib) --- +MLlib is a Spark implementation of some common machine learning algorithms and utilities, +including classification, regression, clustering, collaborative +filtering, dimensionality reduction, as well as underlying optimization primitives: -MLlib is a Spark implementation of some common machine learning (ML) -functionality, as well associated tests and data generators. MLlib -currently supports four common types of machine learning problem settings, -namely classification, regression, clustering and collaborative filtering, -as well as an underlying gradient descent optimization primitive and several -linear algebra methods. - -# Available Methods -The following links provide a detailed explanation of the methods and usage examples for each of them: - -* Classification and Regression - * Binary Classification - * SVM (L1 and L2 regularized) - * Logistic Regression (L1 and L2 regularized) - * Linear Regression - * Least Squares - * Lasso - * Ridge Regression - * Decision Tree (for classification and regression) -* Clustering - * k-Means -* Collaborative Filtering - * Matrix Factorization using Alternating Least Squares -* Optimization - * Gradient Descent and Stochastic Gradient Descent -* Linear Algebra - * Singular Value Decomposition - * Principal Component Analysis - -# Data Types - -Most MLlib algorithms operate on RDDs containing vectors. In Java and Scala, the -[Vector](api/scala/index.html#org.apache.spark.mllib.linalg.Vector) class is used to -represent vectors. You can create either dense or sparse vectors using the -[Vectors](api/scala/index.html#org.apache.spark.mllib.linalg.Vectors$) factory. - -In Python, MLlib can take the following vector types: - -* [NumPy](http://www.numpy.org) arrays -* Standard Python lists (e.g. `[1, 2, 3]`) -* The MLlib [SparseVector](api/python/pyspark.mllib.linalg.SparseVector-class.html) class -* [SciPy sparse matrices](http://docs.scipy.org/doc/scipy/reference/sparse.html) - -For efficiency, we recommend using NumPy arrays over lists, and using the -[CSC format](http://docs.scipy.org/doc/scipy/reference/generated/scipy.sparse.csc_matrix.html#scipy.sparse.csc_matrix) -for SciPy matrices, or MLlib's own SparseVector class. - -Several other simple data types are used throughout the library, e.g. the LabeledPoint -class ([Java/Scala](api/scala/index.html#org.apache.spark.mllib.regression.LabeledPoint), -[Python](api/python/pyspark.mllib.regression.LabeledPoint-class.html)) for labeled data. - -# Dependencies -MLlib uses the [jblas](https://github.com/mikiobraun/jblas) linear algebra library, which itself -depends on native Fortran routines. You may need to install the -[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) -if it is not already present on your nodes. MLlib will throw a linking error if it cannot -detect these libraries automatically. +* [Basics](mllib-basics.html) + * data types + * summary statistics +* Classification and regression + * [linear support vector machine (SVM)](mllib-linear-methods.html#linear-support-vector-machine-svm) + * [logistic regression](mllib-linear-methods.html#logistic-regression) + * [linear least squares, Lasso, and ridge regression](mllib-linear-methods.html#linear-least-squares-lasso-and-ridge-regression) + * [decision tree](mllib-decision-tree.html) + * [naive Bayes](mllib-naive-bayes.html) +* [Collaborative filtering](mllib-collaborative-filtering.html) + * alternating least squares (ALS) +* [Clustering](mllib-clustering.html) + * k-means +* [Dimensionality reduction](mllib-dimensionality-reduction.html) + * singular value decomposition (SVD) + * principal component analysis (PCA) +* [Optimization](mllib-optimization.html) + * stochastic gradient descent + * limited-memory BFGS (L-BFGS) + +MLlib is currently a *beta* component under active development. +The APIs may change in the future releases, and we will provide migration guide between releases. + +## Dependencies + +MLlib uses linear algebra packages [Breeze](http://www.scalanlp.org/), which depends on +[netlib-java](https://github.com/fommil/netlib-java), and +[jblas](https://github.com/mikiobraun/jblas). +`netlib-java` and `jblas` depend on native Fortran routines. +You need to install the +[gfortran runtime library](https://github.com/mikiobraun/jblas/wiki/Missing-Libraries) if it is not +already present on your nodes. MLlib will throw a linking error if it cannot detect these libraries +automatically. Due to license issues, we do not include `netlib-java`'s native libraries in MLlib's +dependency set. If no native library is available at runtime, you will see a warning message. To +use native libraries from `netlib-java`, please include artifact +`com.github.fommil.netlib:all:1.1.2` as a dependency of your project or build your own (see +[instructions](https://github.com/fommil/netlib-java/blob/master/README.md#machine-optimised-system-libraries)). To use MLlib in Python, you will need [NumPy](http://www.numpy.org) version 1.4 or newer. + +--- + +## Migration guide + +### From 0.9 to 1.0 + +In MLlib v1.0, we support both dense and sparse input in a unified way, which introduces a few +breaking changes. If your data is sparse, please store it in a sparse format instead of dense to +take advantage of sparsity in both storage and computation. + +
      +
      + +We used to represent a feature vector by `Array[Double]`, which is replaced by +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used +to accept `RDD[Array[Double]]` now take +`RDD[Vector]`. [`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) +is now a wrapper of `(Double, Vector)` instead of `(Double, Array[Double])`. Converting +`Array[Double]` to `Vector` is straightforward: + +{% highlight scala %} +import org.apache.spark.mllib.linalg.{Vector, Vectors} + +val array: Array[Double] = ... // a double array +val vector: Vector = Vectors.dense(array) // a dense vector +{% endhighlight %} + +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to create sparse vectors. + +*Note*. Scala imports `scala.collection.immutable.Vector` by default, so you have to import `org.apache.spark.mllib.linalg.Vector` explicitly to use MLlib's `Vector`. + +
      + +
      + +We used to represent a feature vector by `double[]`, which is replaced by +[`Vector`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vector) in v1.0. Algorithms that used +to accept `RDD` now take +`RDD`. [`LabeledPoint`](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) +is now a wrapper of `(double, Vector)` instead of `(double, double[])`. Converting `double[]` to +`Vector` is straightforward: + +{% highlight java %} +import org.apache.spark.mllib.linalg.Vector; +import org.apache.spark.mllib.linalg.Vectors; + +double[] array = ... // a double array +Vector vector = Vectors.dense(array) // a dense vector +{% endhighlight %} + +[`Vectors`](api/mllib/index.html#org.apache.spark.mllib.linalg.Vectors$) provides factory methods to +create sparse vectors. + +
      + +
      + +We used to represent a labeled feature vector in a NumPy array, where the first entry corresponds to +the label and the rest are features. This representation is replaced by class +[`LabeledPoint`](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html), which takes both +dense and sparse feature vectors. + +{% highlight python %} +from pyspark.mllib.linalg import SparseVector +from pyspark.mllib.regression import LabeledPoint + +# Create a labeled point with a positive label and a dense feature vector. +pos = LabeledPoint(1.0, [1.0, 0.0, 3.0]) + +# Create a labeled point with a negative label and a sparse feature vector. +neg = LabeledPoint(0.0, SparseVector(3, [0, 2], [1.0, 3.0])) +{% endhighlight %} +
      +
      diff --git a/docs/mllib-linear-algebra.md b/docs/mllib-linear-algebra.md deleted file mode 100644 index 09598be7903ac..0000000000000 --- a/docs/mllib-linear-algebra.md +++ /dev/null @@ -1,74 +0,0 @@ ---- -layout: global -title: MLlib - Linear Algebra ---- - -* Table of contents -{:toc} - - -# Singular Value Decomposition -Singular Value `Decomposition` for Tall and Skinny matrices. -Given an `$m \times n$` matrix `$A$`, we can compute matrices `$U,S,V$` such that - -`\[ - A = U \cdot S \cdot V^T - \]` - -There is no restriction on m, but we require n^2 doubles to -fit in memory locally on one machine. -Further, n should be less than m. - -The decomposition is computed by first computing `$A^TA = V S^2 V^T$`, -computing SVD locally on that (since `$n \times n$` is small), -from which we recover `$S$` and `$V$`. -Then we compute U via easy matrix multiplication -as `$U = A \cdot V \cdot S^{-1}$`. - -Only singular vectors associated with largest k singular values -are recovered. If there are k -such values, then the dimensions of the return will be: - -* `$S$` is `$k \times k$` and diagonal, holding the singular values on diagonal. -* `$U$` is `$m \times k$` and satisfies `$U^T U = \mathop{eye}(k)$`. -* `$V$` is `$n \times k$` and satisfies `$V^T V = \mathop{eye}(k)$`. - -All input and output is expected in sparse matrix format, 0-indexed -as tuples of the form ((i,j),value) all in -SparseMatrix RDDs. Below is example usage. - -{% highlight scala %} - -import org.apache.spark.SparkContext -import org.apache.spark.mllib.linalg.SVD -import org.apache.spark.mllib.linalg.SparseMatrix -import org.apache.spark.mllib.linalg.MatrixEntry - -// Load and parse the data file -val data = sc.textFile("mllib/data/als/test.data").map { line => - val parts = line.split(',') - MatrixEntry(parts(0).toInt, parts(1).toInt, parts(2).toDouble) -} -val m = 4 -val n = 4 -val k = 1 - -// recover largest singular vector -val decomposed = SVD.sparseSVD(SparseMatrix(data, m, n), k) -val = decomposed.S.data - -println("singular values = " + s.toArray.mkString) -{% endhighlight %} - - -# Principal Component Analysis - -Computes the top k principal component coefficients for the m-by-n data matrix X. -Rows of X correspond to observations and columns correspond to variables. -The coefficient matrix is n-by-k. Each column of the return matrix contains coefficients -for one principal component, and the columns are in descending -order of component variance. This function centers the data and uses the -singular value decomposition (SVD) algorithm. - -All input and output is expected in DenseMatrix matrix format. See the examples directory -under "SparkPCA.scala" for example usage. diff --git a/docs/mllib-linear-methods.md b/docs/mllib-linear-methods.md new file mode 100644 index 0000000000000..8108440698495 --- /dev/null +++ b/docs/mllib-linear-methods.md @@ -0,0 +1,389 @@ +--- +layout: global +title: MLlib - Linear Methods +--- + +* Table of contents +{:toc} + + +`\[ +\newcommand{\R}{\mathbb{R}} +\newcommand{\E}{\mathbb{E}} +\newcommand{\x}{\mathbf{x}} +\newcommand{\y}{\mathbf{y}} +\newcommand{\wv}{\mathbf{w}} +\newcommand{\av}{\mathbf{\alpha}} +\newcommand{\bv}{\mathbf{b}} +\newcommand{\N}{\mathbb{N}} +\newcommand{\id}{\mathbf{I}} +\newcommand{\ind}{\mathbf{1}} +\newcommand{\0}{\mathbf{0}} +\newcommand{\unit}{\mathbf{e}} +\newcommand{\one}{\mathbf{1}} +\newcommand{\zero}{\mathbf{0}} +\]` + +## Mathematical formulation + +Many standard *machine learning* methods can be formulated as a convex optimization problem, i.e. +the task of finding a minimizer of a convex function `$f$` that depends on a variable vector +`$\wv$` (called `weights` in the code), which has `$d$` entries. +Formally, we can write this as the optimization problem `$\min_{\wv \in\R^d} \; f(\wv)$`, where +the objective function is of the form +`\begin{equation} + f(\wv) := + \frac1n \sum_{i=1}^n L(\wv;\x_i,y_i) + + \lambda\, R(\wv_i) + \label{eq:regPrimal} + \ . +\end{equation}` +Here the vectors `$\x_i\in\R^d$` are the training data examples, for `$1\le i\le n$`, and +`$y_i\in\R$` are their corresponding labels, which we want to predict. +We call the method *linear* if $L(\wv; \x, y)$ can be expressed as a function of $\wv^T x$ and $y$. +Several MLlib's classification and regression algorithms fall into this category, +and are discussed here. + +The objective function `$f$` has two parts: +the loss that measures the error of the model on the training data, +and the regularizer that measures the complexity of the model. +The loss function `$L(\wv;.)$` must be a convex function in `$\wv$`. +The fixed regularization parameter `$\lambda \ge 0$` (`regParam` in the code) defines the trade-off +between the two goals of small loss and small model complexity. + +### Loss functions + +The following table summarizes the loss functions and their gradients or sub-gradients for the +methods MLlib supports: + + + + + + + + + + + + + + + + + + + +
      loss function $L(\wv; \x, y)$gradient or sub-gradient
      hinge loss$\max \{0, 1-y \wv^T \x \}, \quad y \in \{-1, +1\}$$\begin{cases}-y \cdot \x & \text{if $y \wv^T \x <1$}, \\ 0 & +\text{otherwise}.\end{cases}$
      logistic loss$\log(1+\exp( -y \wv^T \x)), \quad y \in \{-1, +1\}$$-y \left(1-\frac1{1+\exp(-y \wv^T \x)} \right) \cdot \x$
      squared loss$\frac{1}{2} (\wv^T \x - y)^2, \quad y \in \R$$(\wv^T \x - y) \cdot \x$
      + +### Regularizers + +The purpose of the [regularizer](http://en.wikipedia.org/wiki/Regularization_(mathematics)) is to +encourage simple models, by punishing the complexity of the model `$\wv$`, in order to e.g. avoid +over-fitting. +We support the following regularizers in MLlib: + + + + + + + + + + + + + + + + +
      regularizer $R(\wv)$gradient or sub-gradient
      zero (unregularized)0$\0$
      L2$\frac{1}{2}\|\wv\|_2^2$$\wv$
      L1$\|\wv\|_1$$\mathrm{sign}(\wv)$
      + +Here `$\mathrm{sign}(\wv)$` is the vector consisting of the signs (`$\pm1$`) of all the entries +of `$\wv$`. + +L2-regularized problems are generally easier to solve than L1-regularized due to smoothness. +However, L1 regularization can help promote sparsity in weights, leading to simpler models, which is +also used for feature selection. It is not recommended to train models without any regularization, +especially when the number of training examples is small. + +## Binary classification + +[Binary classification](http://en.wikipedia.org/wiki/Binary_classification) is to divide items into +two categories: positive and negative. MLlib supports two linear methods for binary classification: +linear support vector machine (SVM) and logistic regression. The training data set is represented +by an RDD of [LabeledPoint](mllib-data-types.html) in MLlib. Note that, in the mathematical +formulation, a training label $y$ is either $+1$ (positive) or $-1$ (negative), which is convenient +for the formulation. *However*, the negative label is represented by $0$ in MLlib instead of $-1$, +to be consistent with multiclass labeling. + +### Linear support vector machine (SVM) + +The [linear SVM](http://en.wikipedia.org/wiki/Support_vector_machine#Linear_SVM) +has become a standard choice for large-scale classification tasks. +The name "linear SVM" is actually ambiguous. +By "linear SVM", we mean specifically the linear method with the loss function in formulation +`$\eqref{eq:regPrimal}$` given by the hinge loss +`\[ +L(\wv;\x,y) := \max \{0, 1-y \wv^T \x \}. +\]` +By default, linear SVMs are trained with an L2 regularization. +We also support alternative L1 regularization. In this case, +the problem becomes a [linear program](http://en.wikipedia.org/wiki/Linear_programming). + +Linear SVM algorithm outputs a SVM model, which makes predictions based on the value of $\wv^T \x$. +By the default, if $\wv^T \x \geq 0$, the outcome is positive, or negative otherwise. +However, quite often in practice, the default threshold $0$ is not a good choice. +The threshold should be determined via model evaluation. + +### Logistic regression + +[Logistic regression](http://en.wikipedia.org/wiki/Logistic_regression) is widely used to predict a +binary response. It is a linear method with the loss function in formulation +`$\eqref{eq:regPrimal}$` given by the logistic loss +`\[ +L(\wv;\x,y) := \log(1+\exp( -y \wv^T \x)). +\]` + +Logistic regression algorithm outputs a logistic regression model, which makes predictions by +applying the logistic function +`\[ +\mathrm{logit}(z) = \frac{1}{1 + e^{-z}} +\]` +$\wv^T \x$. +By default, if $\mathrm{logit}(\wv^T x) > 0.5$, the outcome is positive, or negative otherwise. +For the same reason mentioned above, quite often in practice, this default threshold is not a good choice. +The threshold should be determined via model evaluation. + +### Evaluation metrics + +MLlib supports common evaluation metrics for binary classification (not available in Python). This +includes precision, recall, [F-measure](http://en.wikipedia.org/wiki/F1_score), +[receiver operating characteristic (ROC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic), +precision-recall curve, and +[area under the curves (AUC)](http://en.wikipedia.org/wiki/Receiver_operating_characteristic#Area_under_the_curve). +Among the metrics, area under ROC is commonly used to compare models and precision/recall/F-measure +can help determine the threshold to use. + +### Examples + +
      + +
      +The following code snippet illustrates how to load a sample dataset, execute a +training algorithm on this training data using a static method in the algorithm +object, and make predictions with the resulting model to compute the training +error. + +{% highlight scala %} +import org.apache.spark.SparkContext +import org.apache.spark.mllib.classification.SVMWithSGD +import org.apache.spark.mllib.evaluation.binary.BinaryClassificationMetrics +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors +import org.apache.spark.mllib.util.MLUtils + +// Load training data in LIBSVM format. +val data = MLUtils.loadLibSVMData(sc, "mllib/data/sample_libsvm_data.txt") + +// Split data into training (60%) and test (40%). +val splits = data.randomSplit(Array(0.6, 0.4), seed = 11L) +val training = splits(0).cache() +val test = splits(1) + +// Run training algorithm to build the model +val numIterations = 100 +val model = SVMWithSGD.train(training, numIterations) + +// Clear the default threshold. +model.clearThreshold() + +// Compute raw scores on the test set. +val scoreAndLabels = test.map { point => + val score = model.predict(point.features) + (score, point.label) +} + +// Get evaluation metrics. +val metrics = new BinaryClassificationMetrics(scoreAndLabels) +val auROC = metrics.areaUnderROC() + +println("Area under ROC = " + auROC) +{% endhighlight %} + +The `SVMWithSGD.train()` method by default performs L2 regularization with the +regularization parameter set to 1.0. If we want to configure this algorithm, we +can customize `SVMWithSGD` further by creating a new object directly and +calling setter methods. All other MLlib algorithms support customization in +this way as well. For example, the following code produces an L1 regularized +variant of SVMs with regularization parameter set to 0.1, and runs the training +algorithm for 200 iterations. + +{% highlight scala %} +import org.apache.spark.mllib.optimization.L1Updater + +val svmAlg = new SVMWithSGD() +svmAlg.optimizer.setNumIterations(200) + .setRegParam(0.1) + .setUpdater(new L1Updater) +val modelL1 = svmAlg.run(parsedData) +{% endhighlight %} + +Similarly, you can use replace `SVMWithSGD` by +[`LogisticRegressionWithSGD`](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD). + +
      + +
      +All of MLlib's methods use Java-friendly types, so you can import and call them there the same +way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the +Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by +calling `.rdd()` on your `JavaRDD` object. +
      + +
      +The following example shows how to load a sample dataset, build Logistic Regression model, +and make predictions with the resulting model to compute the training error. + +{% highlight python %} +from pyspark.mllib.classification import LogisticRegressionWithSGD +from pyspark.mllib.regression import LabeledPoint +from numpy import array + +# Load and parse the data +def parsePoint(line): + values = [float(x) for x in line.split(' ')] + return LabeledPoint(values[0], values[1:]) + +data = sc.textFile("mllib/data/sample_svm_data.txt") +parsedData = data.map(parsePoint) + +# Build the model +model = LogisticRegressionWithSGD.train(parsedData) + +# Evaluating the model on training data +labelsAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) +trainErr = labelsAndPreds.filter(lambda (v, p): v != p).count() / float(parsedData.count()) +print("Training Error = " + str(trainErr)) +{% endhighlight %} +
      +
      + +## Linear least squares, Lasso, and ridge regression + +Linear least squares is a family of linear methods with the loss function in formulation +`$\eqref{eq:regPrimal}$` given by the squared loss + +`\[ +L(\wv;\x,y) := \frac{1}{2} (\wv^T \x - y)^2. +\]` + +Depending on the regularization type, we call the method +[*ordinary least squares*](http://en.wikipedia.org/wiki/Ordinary_least_squares) or simply +[*linear least squares*](http://en.wikipedia.org/wiki/Linear_least_squares_(mathematics)) if there +is no regularization, [*ridge regression*](http://en.wikipedia.org/wiki/Ridge_regression) if L2 +regularization is used, and [*Lasso*](http://en.wikipedia.org/wiki/Lasso_(statistics)) if L1 +regularization is used. This average loss $\frac{1}{n} \sum_{i=1}^n (\wv^T x_i - y_i)^2$ is also +known as the [mean squared error](http://en.wikipedia.org/wiki/Mean_squared_error). + +Note that the squared loss is sensitive to outliers. +Regularization or a robust alternative (e.g., $\ell_1$ regression) is usually necessary in practice. + +### Examples + +
      + +
      +The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. +The example then uses LinearRegressionWithSGD to build a simple linear model to predict label +values. We compute the Mean Squared Error at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +{% highlight scala %} +import org.apache.spark.mllib.regression.LinearRegressionWithSGD +import org.apache.spark.mllib.regression.LabeledPoint +import org.apache.spark.mllib.linalg.Vectors + +// Load and parse the data +val data = sc.textFile("mllib/data/ridge-data/lpsa.data") +val parsedData = data.map { line => + val parts = line.split(',') + LabeledPoint(parts(0).toDouble, Vectors.dense(parts(1).split(' ').map(_.toDouble))) +} + +// Building the model +val numIterations = 100 +val model = LinearRegressionWithSGD.train(parsedData, numIterations) + +// Evaluate model on training examples and compute training error +val valuesAndPreds = parsedData.map { point => + val prediction = model.predict(point.features) + (point.label, prediction) +} +val MSE = valuesAndPreds.map{case(v, p) => math.pow((v - p), 2)}.reduce(_ + _) / valuesAndPreds.count +println("training Mean Squared Error = " + MSE) +{% endhighlight %} + +Similarly you can use +[`RidgeRegressionWithSGD`](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +and [`LassoWithSGD`](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD). + +
      + +
      +All of MLlib's methods use Java-friendly types, so you can import and call them there the same +way you do in Scala. The only caveat is that the methods take Scala RDD objects, while the +Spark Java API uses a separate `JavaRDD` class. You can convert a Java RDD to a Scala one by +calling `.rdd()` on your `JavaRDD` object. +
      + +
      +The following example demonstrate how to load training data, parse it as an RDD of LabeledPoint. +The example then uses LinearRegressionWithSGD to build a simple linear model to predict label +values. We compute the Mean Squared Error at the end to evaluate +[goodness of fit](http://en.wikipedia.org/wiki/Goodness_of_fit). + +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint, LinearRegressionWithSGD +from numpy import array + +# Load and parse the data +def parsePoint(line): + values = [float(x) for x in line.replace(',', ' ').split(' ')] + return LabeledPoint(values[0], values[1:]) + +data = sc.textFile("mllib/data/ridge-data/lpsa.data") +parsedData = data.map(parsePoint) + +# Build the model +model = LinearRegressionWithSGD.train(parsedData) + +# Evaluate the model on training data +valuesAndPreds = parsedData.map(lambda p: (p.label, model.predict(p.features))) +MSE = valuesAndPreds.map(lambda (v, p): (v - p)**2).reduce(lambda x, y: x + y) / valuesAndPreds.count() +print("Mean Squared Error = " + str(MSE)) +{% endhighlight %} +
      +
      + +## Implementation (developer) + +Behind the scene, MLlib implements a simple distributed version of stochastic gradient descent +(SGD), building on the underlying gradient descent primitive (as described in the optimization section). All provided algorithms take as input a +regularization parameter (`regParam`) along with various parameters associated with stochastic +gradient descent (`stepSize`, `numIterations`, `miniBatchFraction`). For each of them, we support +all three possible regularizations (none, L1 or L2). + +Algorithms are all implemented in Scala: + +* [SVMWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.SVMWithSGD) +* [LogisticRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.classification.LogisticRegressionWithSGD) +* [LinearRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LinearRegressionWithSGD) +* [RidgeRegressionWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.RidgeRegressionWithSGD) +* [LassoWithSGD](api/mllib/index.html#org.apache.spark.mllib.regression.LassoWithSGD) + +Python calls the Scala implementation via +[PythonMLLibAPI](api/mllib/index.html#org.apache.spark.mllib.api.python.PythonMLLibAPI). diff --git a/docs/mllib-naive-bayes.md b/docs/mllib-naive-bayes.md new file mode 100644 index 0000000000000..6160fe5b2fe8c --- /dev/null +++ b/docs/mllib-naive-bayes.md @@ -0,0 +1,115 @@ +--- +layout: global +title: MLlib - Naive Bayes +--- + +Naive Bayes is a simple multiclass classification algorithm with the assumption of independence +between every pair of features. Naive Bayes can be trained very efficiently. Within a single pass to +the training data, it computes the conditional probability distribution of each feature given label, +and then it applies Bayes' theorem to compute the conditional probability distribution of label +given an observation and use it for prediction. For more details, please visit the wikipedia page +[Naive Bayes classifier](http://en.wikipedia.org/wiki/Naive_Bayes_classifier). + +In MLlib, we implemented multinomial naive Bayes, which is typically used for document +classification. Within that context, each observation is a document, each feature represents a term, +whose value is the frequency of the term. For its formulation, please visit the wikipedia page +[Multinomial naive Bayes](http://en.wikipedia.org/wiki/Naive_Bayes_classifier#Multinomial_naive_Bayes) +or the section +[Naive Bayes text classification](http://nlp.stanford.edu/IR-book/html/htmledition/naive-bayes-text-classification-1.html) +from the book Introduction to Information +Retrieval. [Additive smoothing](http://en.wikipedia.org/wiki/Lidstone_smoothing) can be used by +setting the parameter $\lambda$ (default to $1.0$). For document classification, the input feature +vectors are usually sparse. Please supply sparse vectors as input to take advantage of +sparsity. Since the training data is only used once, it is not necessary to cache it. + +## Examples + +
      +
      + +[NaiveBayes](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements +multinomial naive Bayes. It takes an RDD of +[LabeledPoint](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an optional +smoothing parameter `lambda` as input, and output a +[NaiveBayesModel](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which +can be used for evaluation and prediction. + +{% highlight scala %} +import org.apache.spark.mllib.classification.NaiveBayes + +val training: RDD[LabeledPoint] = ... // training set +val test: RDD[LabeledPoint] = ... // test set + +val model = NaiveBayes.train(training, lambda = 1.0) +val prediction = model.predict(test.map(_.features)) + +val predictionAndLabel = prediction.zip(test.map(_.label)) +val accuracy = 1.0 * predictionAndLabel.filter(x => x._1 == x._2).count() / test.count() +{% endhighlight %} +
      + +
      + +[NaiveBayes](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayes$) implements +multinomial naive Bayes. It takes a Scala RDD of +[LabeledPoint](api/mllib/index.html#org.apache.spark.mllib.regression.LabeledPoint) and an +optionally smoothing parameter `lambda` as input, and output a +[NaiveBayesModel](api/mllib/index.html#org.apache.spark.mllib.classification.NaiveBayesModel), which +can be used for evaluation and prediction. + +{% highlight java %} +import org.apache.spark.mllib.classification.NaiveBayes; + +JavaRDD training = ... // training set +JavaRDD test = ... // test set + +NaiveBayesModel model = NaiveBayes.train(training.rdd(), 1.0); + +JavaRDD prediction = model.predict(test.map(new Function() { + public Vector call(LabeledPoint p) { + return p.features(); + } + }) +JavaPairRDD predictionAndLabel = + prediction.zip(test.map(new Function() { + public Double call(LabeledPoint p) { + return p.label(); + } + }) +double accuracy = 1.0 * predictionAndLabel.filter(new Function, Boolean>() { + public Boolean call(Tuple2 pl) { + return pl._1() == pl._2(); + } + }).count() / test.count() +{% endhighlight %} +
      + +
      + +[NaiveBayes](api/pyspark/pyspark.mllib.classification.NaiveBayes-class.html) implements multinomial +naive Bayes. It takes an RDD of +[LabeledPoint](api/pyspark/pyspark.mllib.regression.LabeledPoint-class.html) and an optionally +smoothing parameter `lambda` as input, and output a +[NaiveBayesModel](api/pyspark/pyspark.mllib.classification.NaiveBayesModel-class.html), which can be +used for evaluation and prediction. + + +{% highlight python %} +from pyspark.mllib.regression import LabeledPoint +from pyspark.mllib.classification import NaiveBayes + +# an RDD of LabeledPoint +data = sc.parallelize([ + LabeledPoint(0.0, [0.0, 0.0]) + ... # more labeled points +]) + +# Train a naive Bayes model. +model = NaiveBayes.train(data, 1.0) + +# Make prediction. +prediction = model.predict([0.0, 0.0]) +{% endhighlight %} + +
      +
      diff --git a/docs/mllib-optimization.md b/docs/mllib-optimization.md index c79cc3d944f32..bec3912b55dc7 100644 --- a/docs/mllib-optimization.md +++ b/docs/mllib-optimization.md @@ -1,6 +1,6 @@ --- layout: global -title: MLlib - Optimization +title: MLlib - Optimization --- * Table of contents @@ -25,9 +25,10 @@ title: MLlib - Optimization -# Mathematical Description +## Mathematical description + +### Gradient descent -## (Sub)Gradient Descent The simplest method to solve optimization problems of the form `$\min_{\wv \in\R^d} \; f(\wv)$` is [gradient descent](http://en.wikipedia.org/wiki/Gradient_descent). Such first-order optimization methods (including gradient descent and stochastic variants @@ -38,14 +39,14 @@ the direction of steepest descent, which is the negative of the derivative (call [gradient](http://en.wikipedia.org/wiki/Gradient)) of the function at the current point, i.e., at the current parameter value. If the objective function `$f$` is not differentiable at all arguments, but still convex, then a -*subgradient* +*sub-gradient* is the natural generalization of the gradient, and assumes the role of the step direction. -In any case, computing a gradient or subgradient of `$f$` is expensive --- it requires a full +In any case, computing a gradient or sub-gradient of `$f$` is expensive --- it requires a full pass through the complete dataset, in order to compute the contributions from all loss terms. -## Stochastic (Sub)Gradient Descent (SGD) +### Stochastic gradient descent (SGD) Optimization problems whose objective function `$f$` is written as a sum are particularly -suitable to be solved using *stochastic subgradient descent (SGD)*. +suitable to be solved using *stochastic gradient descent (SGD)*. In our case, for the optimization formulations commonly used in supervised machine learning, `\begin{equation} @@ -98,7 +99,7 @@ For the L1-regularizer, the proximal operator is given by soft thresholding, as [L1Updater](api/scala/index.html#org.apache.spark.mllib.optimization.L1Updater). -## Update Schemes for Distributed SGD +### Update schemes for distributed SGD The SGD implementation in [GradientDescent](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) uses a simple (distributed) sampling of the data examples. @@ -129,12 +130,12 @@ point. -# Implementation in MLlib +## Implementation in MLlib Gradient descent methods including stochastic subgradient descent (SGD) as included as a low-level primitive in `MLlib`, upon which various ML algorithms are developed, see the -classification and regression +linear methods section for example. The SGD method @@ -161,6 +162,4 @@ each iteration, to compute the gradient direction. Available algorithms for gradient descent: -* [GradientDescent.runMiniBatchSGD](api/scala/index.html#org.apache.spark.mllib.optimization.GradientDescent) - - +* [GradientDescent.runMiniBatchSGD](api/mllib/index.html#org.apache.spark.mllib.optimization.GradientDescent) diff --git a/mllib/data/sample_libsvm_data.txt b/mllib/data/sample_libsvm_data.txt new file mode 100644 index 0000000000000..861c70cde7fd2 --- /dev/null +++ b/mllib/data/sample_libsvm_data.txt @@ -0,0 +1,100 @@ +0 128:51 129:159 130:253 131:159 132:50 155:48 156:238 157:252 158:252 159:252 160:237 182:54 183:227 184:253 185:252 186:239 187:233 188:252 189:57 190:6 208:10 209:60 210:224 211:252 212:253 213:252 214:202 215:84 216:252 217:253 218:122 236:163 237:252 238:252 239:252 240:253 241:252 242:252 243:96 244:189 245:253 246:167 263:51 264:238 265:253 266:253 267:190 268:114 269:253 270:228 271:47 272:79 273:255 274:168 290:48 291:238 292:252 293:252 294:179 295:12 296:75 297:121 298:21 301:253 302:243 303:50 317:38 318:165 319:253 320:233 321:208 322:84 329:253 330:252 331:165 344:7 345:178 346:252 347:240 348:71 349:19 350:28 357:253 358:252 359:195 372:57 373:252 374:252 375:63 385:253 386:252 387:195 400:198 401:253 402:190 413:255 414:253 415:196 427:76 428:246 429:252 430:112 441:253 442:252 443:148 455:85 456:252 457:230 458:25 467:7 468:135 469:253 470:186 471:12 483:85 484:252 485:223 494:7 495:131 496:252 497:225 498:71 511:85 512:252 513:145 521:48 522:165 523:252 524:173 539:86 540:253 541:225 548:114 549:238 550:253 551:162 567:85 568:252 569:249 570:146 571:48 572:29 573:85 574:178 575:225 576:253 577:223 578:167 579:56 595:85 596:252 597:252 598:252 599:229 600:215 601:252 602:252 603:252 604:196 605:130 623:28 624:199 625:252 626:252 627:253 628:252 629:252 630:233 631:145 652:25 653:128 654:252 655:253 656:252 657:141 658:37 +1 159:124 160:253 161:255 162:63 186:96 187:244 188:251 189:253 190:62 214:127 215:251 216:251 217:253 218:62 241:68 242:236 243:251 244:211 245:31 246:8 268:60 269:228 270:251 271:251 272:94 296:155 297:253 298:253 299:189 323:20 324:253 325:251 326:235 327:66 350:32 351:205 352:253 353:251 354:126 378:104 379:251 380:253 381:184 382:15 405:80 406:240 407:251 408:193 409:23 432:32 433:253 434:253 435:253 436:159 460:151 461:251 462:251 463:251 464:39 487:48 488:221 489:251 490:251 491:172 515:234 516:251 517:251 518:196 519:12 543:253 544:251 545:251 546:89 570:159 571:255 572:253 573:253 574:31 597:48 598:228 599:253 600:247 601:140 602:8 625:64 626:251 627:253 628:220 653:64 654:251 655:253 656:220 681:24 682:193 683:253 684:220 +1 125:145 126:255 127:211 128:31 152:32 153:237 154:253 155:252 156:71 180:11 181:175 182:253 183:252 184:71 209:144 210:253 211:252 212:71 236:16 237:191 238:253 239:252 240:71 264:26 265:221 266:253 267:252 268:124 269:31 293:125 294:253 295:252 296:252 297:108 322:253 323:252 324:252 325:108 350:255 351:253 352:253 353:108 378:253 379:252 380:252 381:108 406:253 407:252 408:252 409:108 434:253 435:252 436:252 437:108 462:255 463:253 464:253 465:170 490:253 491:252 492:252 493:252 494:42 518:149 519:252 520:252 521:252 522:144 546:109 547:252 548:252 549:252 550:144 575:218 576:253 577:253 578:255 579:35 603:175 604:252 605:252 606:253 607:35 631:73 632:252 633:252 634:253 635:35 659:31 660:211 661:252 662:253 663:35 +1 153:5 154:63 155:197 181:20 182:254 183:230 184:24 209:20 210:254 211:254 212:48 237:20 238:254 239:255 240:48 265:20 266:254 267:254 268:57 293:20 294:254 295:254 296:108 321:16 322:239 323:254 324:143 350:178 351:254 352:143 378:178 379:254 380:143 406:178 407:254 408:162 434:178 435:254 436:240 462:113 463:254 464:240 490:83 491:254 492:245 493:31 518:79 519:254 520:246 521:38 547:214 548:254 549:150 575:144 576:241 577:8 603:144 604:240 605:2 631:144 632:254 633:82 659:230 660:247 661:40 687:168 688:209 689:31 +1 152:1 153:168 154:242 155:28 180:10 181:228 182:254 183:100 209:190 210:254 211:122 237:83 238:254 239:162 265:29 266:254 267:248 268:25 293:29 294:255 295:254 296:103 321:29 322:254 323:254 324:109 349:29 350:254 351:254 352:109 377:29 378:254 379:254 380:109 405:29 406:255 407:254 408:109 433:29 434:254 435:254 436:109 461:29 462:254 463:254 464:63 489:29 490:254 491:254 492:28 517:29 518:254 519:254 520:28 545:29 546:254 547:254 548:35 573:29 574:254 575:254 576:109 601:6 602:212 603:254 604:109 630:203 631:254 632:178 658:155 659:254 660:190 686:32 687:199 688:104 +0 130:64 131:253 132:255 133:63 157:96 158:205 159:251 160:253 161:205 162:111 163:4 184:96 185:189 186:251 187:251 188:253 189:251 190:251 191:31 209:16 210:64 211:223 212:244 213:251 214:251 215:211 216:213 217:251 218:251 219:31 236:80 237:181 238:251 239:253 240:251 241:251 242:251 243:94 244:96 245:251 246:251 247:31 263:92 264:253 265:253 266:253 267:255 268:253 269:253 270:253 271:95 272:96 273:253 274:253 275:31 290:92 291:236 292:251 293:243 294:220 295:233 296:251 297:251 298:243 299:82 300:96 301:251 302:251 303:31 317:80 318:253 319:251 320:251 321:188 323:96 324:251 325:251 326:109 328:96 329:251 330:251 331:31 344:96 345:240 346:253 347:243 348:188 349:42 351:96 352:204 353:109 354:4 356:12 357:197 358:251 359:31 372:221 373:251 374:253 375:121 379:36 380:23 385:190 386:251 387:31 399:48 400:234 401:253 413:191 414:253 415:31 426:44 427:221 428:251 429:251 440:12 441:197 442:251 443:31 454:190 455:251 456:251 457:251 468:96 469:251 470:251 471:31 482:190 483:251 484:251 485:113 495:40 496:234 497:251 498:219 499:23 510:190 511:251 512:251 513:94 522:40 523:217 524:253 525:231 526:47 538:191 539:253 540:253 541:253 548:12 549:174 550:253 551:253 552:219 553:39 566:67 567:236 568:251 569:251 570:191 571:190 572:111 573:72 574:190 575:191 576:197 577:251 578:243 579:121 580:39 595:63 596:236 597:251 598:253 599:251 600:251 601:251 602:251 603:253 604:251 605:188 606:94 624:27 625:129 626:253 627:251 628:251 629:251 630:251 631:229 632:168 633:15 654:95 655:212 656:251 657:211 658:94 659:59 +1 159:121 160:254 161:136 186:13 187:230 188:253 189:248 190:99 213:4 214:118 215:253 216:253 217:225 218:42 241:61 242:253 243:253 244:253 245:74 268:32 269:206 270:253 271:253 272:186 273:9 296:211 297:253 298:253 299:239 300:69 324:254 325:253 326:253 327:133 351:142 352:255 353:253 354:186 355:8 378:149 379:229 380:254 381:207 382:21 405:54 406:229 407:253 408:254 409:105 433:152 434:254 435:254 436:213 437:26 460:112 461:251 462:253 463:253 464:26 487:29 488:212 489:253 490:250 491:149 514:36 515:214 516:253 517:253 518:137 542:75 543:253 544:253 545:253 546:59 570:93 571:253 572:253 573:189 574:17 598:224 599:253 600:253 601:84 625:43 626:235 627:253 628:126 629:1 653:99 654:248 655:253 656:119 682:225 683:235 684:49 +1 100:166 101:222 102:55 128:197 129:254 130:218 131:5 155:29 156:249 157:254 158:254 159:9 183:45 184:254 185:254 186:174 187:2 210:4 211:164 212:254 213:254 214:85 238:146 239:254 240:254 241:254 242:85 265:101 266:245 267:254 268:254 269:254 270:85 292:97 293:248 294:254 295:204 296:254 297:254 298:85 315:12 316:59 317:98 318:151 319:237 320:254 321:254 322:109 323:35 324:254 325:254 326:85 343:41 344:216 345:254 346:254 347:239 348:153 349:37 350:4 351:32 352:254 353:254 354:85 372:7 373:44 374:44 375:30 379:32 380:254 381:254 382:96 407:19 408:230 409:254 410:174 436:197 437:254 438:110 464:197 465:254 466:85 492:197 493:253 494:63 515:37 516:54 517:54 518:45 519:26 520:84 521:221 522:84 523:21 524:31 525:162 526:78 540:6 541:41 542:141 543:244 544:254 545:254 546:248 547:236 548:254 549:254 550:254 551:233 552:239 553:254 554:138 567:23 568:167 569:254 570:254 571:254 572:254 573:229 574:228 575:185 576:138 577:138 578:138 579:138 580:138 581:138 582:44 595:113 596:254 597:254 598:254 599:179 600:64 601:5 623:32 624:209 625:183 626:97 +0 155:53 156:255 157:253 158:253 159:253 160:124 183:180 184:253 185:251 186:251 187:251 188:251 189:145 190:62 209:32 210:217 211:241 212:253 213:251 214:251 215:251 216:251 217:253 218:107 237:37 238:251 239:251 240:253 241:251 242:251 243:251 244:251 245:253 246:107 265:166 266:251 267:251 268:253 269:251 270:96 271:148 272:251 273:253 274:107 291:73 292:253 293:253 294:253 295:253 296:130 299:110 300:253 301:255 302:108 319:73 320:251 321:251 322:251 323:251 327:109 328:251 329:253 330:107 347:202 348:251 349:251 350:251 351:225 354:6 355:129 356:251 357:253 358:107 375:150 376:251 377:251 378:251 379:71 382:115 383:251 384:251 385:253 386:107 403:253 404:251 405:251 406:173 407:20 410:217 411:251 412:251 413:253 414:107 430:182 431:255 432:253 433:216 438:218 439:253 440:253 441:182 457:63 458:221 459:253 460:251 461:215 465:84 466:236 467:251 468:251 469:77 485:109 486:251 487:253 488:251 489:215 492:11 493:160 494:251 495:251 496:96 513:109 514:251 515:253 516:251 517:137 520:150 521:251 522:251 523:251 524:71 541:109 542:251 543:253 544:251 545:35 547:130 548:253 549:251 550:251 551:173 552:20 569:110 570:253 571:255 572:253 573:98 574:150 575:253 576:255 577:253 578:164 597:109 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:35 625:93 626:241 627:253 628:251 629:251 630:251 631:251 632:216 633:112 634:5 654:103 655:253 656:251 657:251 658:251 659:251 683:124 684:251 685:225 686:71 687:71 +0 128:73 129:253 130:227 131:73 132:21 156:73 157:251 158:251 159:251 160:174 182:16 183:166 184:228 185:251 186:251 187:251 188:122 210:62 211:220 212:253 213:251 214:251 215:251 216:251 217:79 238:79 239:231 240:253 241:251 242:251 243:251 244:251 245:232 246:77 264:145 265:253 266:253 267:253 268:255 269:253 270:253 271:253 272:253 273:255 274:108 292:144 293:251 294:251 295:251 296:253 297:168 298:107 299:169 300:251 301:253 302:189 303:20 318:27 319:89 320:236 321:251 322:235 323:215 324:164 325:15 326:6 327:129 328:251 329:253 330:251 331:35 345:47 346:211 347:253 348:251 349:251 350:142 354:37 355:251 356:251 357:253 358:251 359:35 373:109 374:251 375:253 376:251 377:251 378:142 382:11 383:148 384:251 385:253 386:251 387:164 400:11 401:150 402:253 403:255 404:211 405:25 410:11 411:150 412:253 413:255 414:211 415:25 428:140 429:251 430:251 431:253 432:107 438:37 439:251 440:251 441:211 442:46 456:190 457:251 458:251 459:253 460:128 461:5 466:37 467:251 468:251 469:51 484:115 485:251 486:251 487:253 488:188 489:20 492:32 493:109 494:129 495:251 496:173 497:103 512:217 513:251 514:251 515:201 516:30 520:73 521:251 522:251 523:251 524:71 540:166 541:253 542:253 543:255 544:149 545:73 546:150 547:253 548:255 549:253 550:253 551:143 568:140 569:251 570:251 571:253 572:251 573:251 574:251 575:251 576:253 577:251 578:230 579:61 596:190 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:242 605:215 606:55 624:21 625:189 626:251 627:253 628:251 629:251 630:251 631:173 632:103 653:31 654:200 655:253 656:251 657:96 658:71 659:20 +1 155:178 156:255 157:105 182:6 183:188 184:253 185:216 186:14 210:14 211:202 212:253 213:253 214:23 238:12 239:199 240:253 241:128 242:6 266:42 267:253 268:253 269:158 294:42 295:253 296:253 297:158 322:155 323:253 324:253 325:158 350:160 351:253 352:253 353:147 378:160 379:253 380:253 381:41 405:17 406:225 407:253 408:235 409:31 433:24 434:253 435:253 436:176 461:24 462:253 463:253 464:176 489:24 490:253 491:253 492:176 517:24 518:253 519:253 520:176 545:24 546:253 547:253 548:162 573:46 574:253 575:253 576:59 601:142 602:253 603:253 604:59 629:142 630:253 631:253 632:59 657:142 658:253 659:202 660:8 685:87 686:253 687:139 +0 154:46 155:105 156:254 157:254 158:254 159:254 160:255 161:239 162:41 180:37 181:118 182:222 183:254 184:253 185:253 186:253 187:253 188:253 189:253 190:211 191:54 207:14 208:200 209:253 210:253 211:254 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:116 233:16 234:160 235:236 236:253 237:253 238:253 239:254 240:253 241:253 242:246 243:229 244:253 245:253 246:253 247:116 261:99 262:253 263:253 264:253 265:253 266:253 267:254 268:253 269:253 270:213 271:99 272:253 273:253 274:253 275:116 288:25 289:194 290:253 291:253 292:253 293:253 294:131 295:97 296:169 297:253 298:93 299:99 300:253 301:253 302:253 303:116 316:206 317:253 318:253 319:251 320:233 321:127 322:9 324:18 325:38 326:3 327:15 328:171 329:253 330:253 331:116 343:55 344:240 345:253 346:253 347:233 355:31 356:186 357:253 358:253 359:116 371:176 372:253 373:253 374:253 375:127 383:99 384:253 385:253 386:253 387:116 399:176 400:253 401:253 402:131 403:9 411:99 412:253 413:253 414:253 415:116 426:119 427:254 428:254 429:232 430:75 440:158 441:254 442:254 443:117 454:118 455:253 456:253 457:154 468:156 469:253 470:253 471:116 482:118 483:253 484:253 485:154 496:156 497:253 498:253 499:116 509:46 510:222 511:253 512:253 513:154 522:7 523:116 524:246 525:253 526:180 527:9 538:118 539:253 540:253 541:154 550:116 551:253 552:253 553:253 554:174 566:118 567:253 568:253 569:154 577:110 578:246 579:253 580:253 581:240 582:67 594:118 595:253 596:253 597:238 598:215 599:49 600:20 601:20 602:20 603:66 604:215 605:241 606:253 607:245 608:233 609:64 622:82 623:229 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:254 632:253 633:253 634:240 635:107 651:176 652:253 653:253 654:253 655:253 656:253 657:253 658:253 659:254 660:253 661:253 662:108 679:40 680:239 681:253 682:253 683:253 684:253 685:253 686:253 687:254 688:161 689:57 690:4 +0 152:56 153:105 154:220 155:254 156:63 178:18 179:166 180:233 181:253 182:253 183:253 184:236 185:209 186:209 187:209 188:77 189:18 206:84 207:253 208:253 209:253 210:253 211:253 212:254 213:253 214:253 215:253 216:253 217:172 218:8 233:57 234:238 235:253 236:253 237:253 238:253 239:253 240:254 241:253 242:253 243:253 244:253 245:253 246:119 260:14 261:238 262:253 263:253 264:253 265:253 266:253 267:253 268:179 269:196 270:253 271:253 272:253 273:253 274:238 275:12 288:33 289:253 290:253 291:253 292:253 293:253 294:248 295:134 297:18 298:83 299:237 300:253 301:253 302:253 303:14 316:164 317:253 318:253 319:253 320:253 321:253 322:128 327:57 328:119 329:214 330:253 331:94 343:57 344:248 345:253 346:253 347:253 348:126 349:14 350:4 357:179 358:253 359:248 360:56 371:175 372:253 373:253 374:240 375:190 376:28 385:179 386:253 387:253 388:173 399:209 400:253 401:253 402:178 413:92 414:253 415:253 416:208 427:211 428:254 429:254 430:179 442:135 443:255 444:209 455:209 456:253 457:253 458:90 470:134 471:253 472:208 483:209 484:253 485:253 486:178 497:2 498:142 499:253 500:208 511:209 512:253 513:253 514:214 515:35 525:30 526:253 527:253 528:208 539:165 540:253 541:253 542:253 543:215 544:36 553:163 554:253 555:253 556:164 567:18 568:172 569:253 570:253 571:253 572:214 573:127 574:7 580:72 581:232 582:253 583:171 584:17 596:8 597:182 598:253 599:253 600:253 601:253 602:162 603:56 607:64 608:240 609:253 610:253 611:14 625:7 626:173 627:253 628:253 629:253 630:253 631:245 632:241 633:239 634:239 635:246 636:253 637:225 638:14 639:1 654:18 655:59 656:138 657:224 658:253 659:253 660:254 661:253 662:253 663:253 664:240 665:96 685:37 686:104 687:192 688:255 689:253 690:253 691:182 692:73 +1 130:7 131:176 132:254 133:224 158:51 159:253 160:253 161:223 185:4 186:170 187:253 188:253 189:214 213:131 214:253 215:253 216:217 217:39 241:209 242:253 243:253 244:134 268:75 269:240 270:253 271:239 272:26 296:184 297:253 298:245 299:63 323:142 324:255 325:253 326:185 350:62 351:229 352:254 353:242 354:73 377:54 378:229 379:253 380:254 381:105 405:152 406:254 407:254 408:213 409:26 432:32 433:243 434:253 435:253 436:115 459:2 460:142 461:253 462:253 463:155 487:30 488:253 489:253 490:232 491:55 515:75 516:253 517:253 518:164 542:72 543:232 544:253 545:189 546:17 570:224 571:253 572:253 573:163 597:43 598:235 599:253 600:253 601:195 602:21 625:28 626:231 627:253 628:253 629:184 630:14 654:225 655:253 656:253 657:75 +0 155:21 156:176 157:253 158:253 159:124 182:105 183:176 184:251 185:251 186:251 187:251 188:105 208:58 209:217 210:241 211:253 212:251 213:251 214:251 215:251 216:243 217:113 218:5 235:63 236:231 237:251 238:251 239:253 240:251 241:251 242:251 243:251 244:253 245:251 246:113 263:144 264:251 265:251 266:251 267:253 268:251 269:251 270:251 271:251 272:253 273:251 274:215 290:125 291:253 292:253 293:253 294:253 295:255 296:253 297:253 298:253 299:253 300:255 301:253 302:227 303:42 318:253 319:251 320:251 321:251 322:251 323:253 324:251 325:251 326:251 327:251 328:253 329:251 330:251 331:142 345:27 346:253 347:251 348:251 349:235 350:241 351:253 352:251 353:246 354:137 355:35 356:98 357:251 358:251 359:236 360:61 372:47 373:211 374:253 375:251 376:235 377:82 378:103 379:253 380:251 381:137 384:73 385:251 386:251 387:251 388:71 399:27 400:211 401:251 402:253 403:251 404:86 407:72 408:71 409:10 412:73 413:251 414:251 415:173 416:20 427:89 428:253 429:253 430:255 431:253 432:35 440:73 441:253 442:253 443:253 444:72 454:84 455:236 456:251 457:251 458:253 459:251 460:138 468:73 469:251 470:251 471:251 472:71 481:63 482:236 483:251 484:251 485:251 486:227 487:251 488:246 489:138 490:11 494:16 495:37 496:228 497:251 498:246 499:137 500:10 509:73 510:251 511:251 512:251 513:173 514:42 515:142 516:142 517:142 518:41 522:109 523:251 524:253 525:251 526:137 537:73 538:251 539:251 540:173 541:20 549:27 550:211 551:251 552:253 553:147 554:10 565:73 566:253 567:253 568:143 575:21 576:176 577:253 578:253 579:253 593:73 594:251 595:251 596:205 597:144 603:176 604:251 605:251 606:188 607:107 621:62 622:236 623:251 624:251 625:251 626:218 627:217 628:217 629:217 630:217 631:253 632:230 633:189 634:20 650:83 651:158 652:251 653:251 654:253 655:251 656:251 657:251 658:251 659:253 660:107 679:37 680:251 681:251 682:253 683:251 684:251 685:251 686:122 687:72 688:30 +1 151:68 152:45 153:131 154:131 155:131 156:101 157:68 158:92 159:44 187:19 188:170 211:29 212:112 213:89 215:40 216:222 239:120 240:254 241:251 242:127 243:40 244:222 267:197 268:254 269:254 270:91 271:40 272:222 294:64 295:247 296:254 297:236 298:50 299:40 300:107 322:184 323:254 324:254 325:91 327:6 328:14 350:203 351:254 352:254 353:71 377:23 378:218 379:254 380:254 381:71 405:113 406:254 407:255 408:239 409:53 433:210 434:254 435:254 436:195 460:62 461:242 462:254 463:241 464:88 468:28 488:86 489:254 490:254 491:189 495:28 496:104 516:106 517:254 518:254 519:168 523:40 524:91 544:216 545:254 546:245 547:51 551:35 552:80 572:216 573:254 574:102 599:55 600:239 601:254 602:52 627:166 628:254 629:210 630:23 655:223 656:252 657:104 683:223 684:169 +0 125:29 126:170 127:255 128:255 129:141 151:29 152:198 153:255 154:255 155:255 156:226 157:255 158:86 178:141 179:255 180:255 181:170 182:29 184:86 185:255 186:255 187:141 204:29 205:226 206:255 207:198 208:57 213:226 214:255 215:255 216:226 217:114 231:29 232:255 233:255 234:114 241:141 242:170 243:114 244:255 245:255 246:141 259:226 260:255 261:170 269:29 270:57 273:141 274:255 275:226 286:57 287:255 288:170 302:114 303:255 304:198 314:226 315:255 331:170 332:255 333:57 342:255 343:226 360:255 361:170 370:255 371:170 388:114 389:198 398:255 399:226 416:86 417:255 426:198 427:255 444:86 445:255 454:114 455:255 456:57 472:86 473:255 482:29 483:255 484:226 500:141 501:255 511:170 512:255 513:170 528:226 529:198 539:29 540:226 541:255 542:170 555:29 556:255 557:114 568:29 569:226 570:255 571:141 582:57 583:226 584:226 598:141 599:255 600:255 601:170 602:86 607:29 608:86 609:226 610:255 611:226 612:29 627:86 628:198 629:255 630:255 631:255 632:255 633:255 634:255 635:255 636:255 637:255 638:141 639:29 657:29 658:114 659:170 660:170 661:170 662:170 663:170 664:86 +0 153:203 154:254 155:252 156:252 157:252 158:214 159:51 160:20 180:62 181:221 182:252 183:250 184:250 185:250 186:252 187:250 188:160 189:20 207:62 208:211 209:250 210:252 211:250 212:250 213:250 214:252 215:250 216:250 217:49 234:41 235:221 236:250 237:250 238:252 239:250 240:250 241:250 242:252 243:250 244:128 245:10 262:254 263:252 264:252 265:252 266:254 267:252 268:252 269:252 270:254 271:252 272:252 273:90 290:150 291:190 292:250 293:250 294:252 295:250 296:250 297:169 298:171 299:250 300:250 301:250 302:82 318:31 319:191 320:250 321:250 322:252 323:189 324:100 325:20 326:172 327:250 328:250 329:250 330:80 346:213 347:250 348:250 349:250 350:212 351:29 354:252 355:250 356:250 357:250 374:92 375:252 376:252 377:252 382:51 383:252 384:252 385:252 386:203 401:82 402:252 403:250 404:250 405:169 410:132 411:250 412:250 413:250 414:121 428:92 429:231 430:252 431:250 432:159 433:20 438:252 439:250 440:250 441:250 456:30 457:211 458:252 459:250 460:221 461:40 466:90 467:250 468:250 469:250 470:163 484:31 485:213 486:254 487:232 488:80 494:92 495:252 496:252 497:212 498:163 512:151 513:250 514:252 515:149 522:252 523:250 524:250 525:49 540:60 541:221 542:252 543:210 544:60 550:252 551:250 552:250 553:49 569:202 570:252 571:250 572:221 573:40 576:123 577:202 578:252 579:250 580:250 581:49 596:123 597:243 598:255 599:252 600:252 601:252 602:254 603:252 604:252 605:252 606:254 607:252 608:100 625:121 626:171 627:250 628:250 629:250 630:252 631:250 632:250 633:250 634:252 635:250 636:100 654:20 655:160 656:250 657:250 658:252 659:250 660:250 661:250 662:252 663:189 664:40 683:20 684:170 685:250 686:252 687:250 688:128 689:49 690:49 691:29 +1 98:64 99:191 100:70 125:68 126:243 127:253 128:249 129:63 152:30 153:223 154:253 155:253 156:247 157:41 179:73 180:238 181:253 182:253 183:253 184:242 206:73 207:236 208:253 209:253 210:253 211:253 212:242 234:182 235:253 236:253 237:191 238:247 239:253 240:149 262:141 263:253 264:143 265:86 266:249 267:253 268:122 290:9 291:36 292:7 293:14 294:233 295:253 296:122 322:230 323:253 324:122 350:230 351:253 352:122 378:231 379:255 380:123 406:230 407:253 408:52 433:61 434:245 435:253 461:98 462:253 463:253 468:35 469:12 489:98 490:253 491:253 494:9 495:142 496:233 497:146 517:190 518:253 519:253 520:128 521:7 522:99 523:253 524:253 525:180 544:29 545:230 546:253 547:253 548:252 549:210 550:253 551:253 552:253 553:140 571:28 572:207 573:253 574:253 575:253 576:254 577:253 578:253 579:235 580:70 581:9 599:126 600:253 601:253 602:253 603:253 604:254 605:253 606:168 607:19 627:79 628:253 629:253 630:201 631:190 632:132 633:63 634:5 +1 125:26 126:240 127:72 153:25 154:238 155:208 182:209 183:226 184:14 210:209 211:254 212:43 238:175 239:254 240:128 266:63 267:254 268:204 294:107 295:254 296:204 322:88 323:254 324:204 350:55 351:254 352:204 378:126 379:254 380:204 406:126 407:254 408:189 434:169 435:254 436:121 462:209 463:254 464:193 490:209 491:254 492:111 517:22 518:235 519:254 520:37 545:137 546:254 547:227 548:16 573:205 574:255 575:185 601:205 602:254 603:125 629:205 630:254 631:125 657:111 658:212 659:43 +0 155:62 156:91 157:213 158:255 159:228 160:91 161:12 182:70 183:230 184:253 185:253 186:253 187:253 188:253 189:152 190:7 210:246 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:106 237:21 238:247 239:253 240:253 241:253 242:253 243:253 244:253 245:208 246:24 265:156 266:253 267:253 268:253 269:253 270:253 271:253 272:253 273:195 292:88 293:238 294:253 295:253 296:253 297:221 298:253 299:253 300:253 301:195 320:230 321:253 322:253 323:253 324:198 325:40 326:177 327:253 328:253 329:195 346:56 347:156 348:251 349:253 350:189 351:182 352:15 354:86 355:240 356:253 357:210 358:28 374:213 375:253 376:253 377:156 378:3 383:205 384:253 385:253 386:106 401:121 402:252 403:253 404:135 405:3 411:46 412:253 413:253 414:106 428:28 429:212 430:253 431:248 432:23 439:42 440:253 441:253 442:106 456:197 457:253 458:234 459:70 467:42 468:253 469:253 470:106 483:11 484:202 485:253 486:187 495:58 496:253 497:210 498:27 511:107 512:253 513:253 514:40 522:53 523:227 524:253 525:195 539:107 540:253 541:253 542:40 549:47 550:227 551:253 552:231 553:58 567:107 568:253 569:253 570:40 575:5 576:131 577:222 578:253 579:231 580:59 595:14 596:204 597:253 598:226 599:222 600:73 601:58 602:58 603:170 604:253 605:253 606:227 607:58 624:197 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:238 634:58 652:33 653:179 654:241 655:253 656:253 657:253 658:253 659:250 660:116 661:14 682:75 683:179 684:253 685:151 686:89 687:86 +1 157:42 158:228 159:253 160:253 185:144 186:251 187:251 188:251 212:89 213:236 214:251 215:235 216:215 239:79 240:253 241:251 242:251 243:142 267:180 268:253 269:251 270:251 271:142 294:32 295:202 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:112 349:6 350:129 351:251 352:253 353:127 354:5 377:37 378:251 379:251 380:253 381:107 405:166 406:251 407:251 408:201 409:30 432:42 433:228 434:253 435:253 460:144 461:251 462:251 463:147 487:63 488:236 489:251 490:251 491:71 515:150 516:251 517:251 518:204 519:41 543:253 544:251 545:251 546:142 571:255 572:253 573:164 598:105 599:253 600:251 601:35 626:180 627:253 628:251 629:35 654:180 655:253 656:251 657:35 682:180 683:253 684:251 685:35 +1 128:62 129:254 130:213 156:102 157:253 158:252 159:102 160:20 184:102 185:254 186:253 187:254 188:50 212:102 213:253 214:252 215:253 216:50 240:102 241:254 242:253 243:254 244:50 268:142 269:253 270:252 271:253 272:50 295:51 296:253 297:254 298:253 299:224 300:20 323:132 324:252 325:253 326:252 327:162 351:173 352:253 353:254 354:253 355:102 378:82 379:253 380:252 381:253 382:252 383:61 406:203 407:254 408:253 409:254 410:233 433:41 434:243 435:253 436:252 437:253 438:111 461:132 462:253 463:254 464:253 465:203 488:41 489:253 490:252 491:253 492:252 493:40 515:11 516:213 517:254 518:253 519:254 520:151 543:92 544:252 545:253 546:252 547:192 548:50 570:21 571:214 572:253 573:255 574:253 575:41 598:142 599:253 600:252 601:253 602:171 625:113 626:253 627:255 628:253 629:203 630:40 653:30 654:131 655:233 656:111 +0 154:28 155:195 156:254 157:254 158:254 159:254 160:254 161:255 162:61 181:6 182:191 183:253 184:253 185:253 186:253 187:253 188:253 189:253 190:60 208:26 209:190 210:253 211:253 212:253 213:253 214:240 215:191 216:242 217:253 218:60 235:15 236:187 237:253 238:253 239:253 240:253 241:253 242:200 244:211 245:253 246:60 262:22 263:66 264:253 265:253 266:253 267:253 268:241 269:209 270:44 271:23 272:218 273:253 274:60 290:124 291:253 292:253 293:253 294:253 295:253 296:182 299:131 300:253 301:253 302:60 318:38 319:217 320:253 321:253 322:244 323:111 324:37 327:131 328:253 329:253 330:60 346:124 347:253 348:253 349:253 350:165 354:22 355:182 356:253 357:253 358:60 374:124 375:253 376:253 377:240 378:45 382:53 383:253 384:253 385:249 386:58 401:16 402:168 403:253 404:216 405:45 410:53 411:253 412:253 413:138 429:159 430:253 431:253 432:147 438:53 439:253 440:253 441:138 456:136 457:252 458:253 459:227 460:5 466:53 467:253 468:243 469:101 484:140 485:253 486:253 487:124 494:156 495:253 496:218 511:13 512:164 513:253 514:142 515:5 521:32 522:233 523:253 524:218 539:62 540:253 541:253 542:130 548:37 549:203 550:253 551:253 552:127 567:62 568:253 569:253 570:147 571:36 572:36 573:36 574:36 575:151 576:222 577:253 578:245 579:127 580:8 595:34 596:202 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:200 624:140 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:248 633:235 634:65 652:87 653:173 654:253 655:253 656:253 657:253 658:253 659:253 660:182 681:14 682:78 683:96 684:253 685:253 686:253 687:137 688:56 +0 123:8 124:76 125:202 126:254 127:255 128:163 129:37 130:2 150:13 151:182 152:253 153:253 154:253 155:253 156:253 157:253 158:23 177:15 178:179 179:253 180:253 181:212 182:91 183:218 184:253 185:253 186:179 187:109 205:105 206:253 207:253 208:160 209:35 210:156 211:253 212:253 213:253 214:253 215:250 216:113 232:19 233:212 234:253 235:253 236:88 237:121 238:253 239:233 240:128 241:91 242:245 243:253 244:248 245:114 260:104 261:253 262:253 263:110 264:2 265:142 266:253 267:90 270:26 271:199 272:253 273:248 274:63 287:1 288:173 289:253 290:253 291:29 293:84 294:228 295:39 299:72 300:251 301:253 302:215 303:29 315:36 316:253 317:253 318:203 319:13 328:82 329:253 330:253 331:170 343:36 344:253 345:253 346:164 356:11 357:198 358:253 359:184 360:6 371:36 372:253 373:253 374:82 385:138 386:253 387:253 388:35 399:128 400:253 401:253 402:47 413:48 414:253 415:253 416:35 427:154 428:253 429:253 430:47 441:48 442:253 443:253 444:35 455:102 456:253 457:253 458:99 469:48 470:253 471:253 472:35 483:36 484:253 485:253 486:164 496:16 497:208 498:253 499:211 500:17 511:32 512:244 513:253 514:175 515:4 524:44 525:253 526:253 527:156 540:171 541:253 542:253 543:29 551:30 552:217 553:253 554:188 555:19 568:171 569:253 570:253 571:59 578:60 579:217 580:253 581:253 582:70 596:78 597:253 598:253 599:231 600:48 604:26 605:128 606:249 607:253 608:244 609:94 610:15 624:8 625:151 626:253 627:253 628:234 629:101 630:121 631:219 632:229 633:253 634:253 635:201 636:80 653:38 654:232 655:253 656:253 657:253 658:253 659:253 660:253 661:253 662:201 663:66 +0 127:68 128:254 129:255 130:254 131:107 153:11 154:176 155:230 156:253 157:253 158:253 159:212 180:28 181:197 182:253 183:253 184:253 185:253 186:253 187:229 188:107 189:14 208:194 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:253 217:53 235:69 236:241 237:253 238:253 239:253 240:253 241:241 242:186 243:253 244:253 245:195 262:10 263:161 264:253 265:253 266:253 267:246 268:40 269:57 270:231 271:253 272:253 273:195 290:140 291:253 292:253 293:253 294:253 295:154 297:25 298:253 299:253 300:253 301:195 318:213 319:253 320:253 321:253 322:135 323:8 325:3 326:128 327:253 328:253 329:195 345:77 346:238 347:253 348:253 349:253 350:7 354:116 355:253 356:253 357:195 372:11 373:165 374:253 375:253 376:231 377:70 378:1 382:78 383:237 384:253 385:195 400:33 401:253 402:253 403:253 404:182 411:200 412:253 413:195 428:98 429:253 430:253 431:253 432:24 439:42 440:253 441:195 456:197 457:253 458:253 459:253 460:24 467:163 468:253 469:195 484:197 485:253 486:253 487:189 488:13 494:53 495:227 496:253 497:121 512:197 513:253 514:253 515:114 521:21 522:227 523:253 524:231 525:27 540:197 541:253 542:253 543:114 547:5 548:131 549:143 550:253 551:231 552:59 568:197 569:253 570:253 571:236 572:73 573:58 574:217 575:223 576:253 577:253 578:253 579:174 596:197 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:48 624:149 625:253 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:182 634:15 635:3 652:12 653:168 654:253 655:253 656:253 657:253 658:253 659:248 660:89 661:23 +1 157:85 158:255 159:103 160:1 185:205 186:253 187:253 188:30 213:205 214:253 215:253 216:30 240:44 241:233 242:253 243:244 244:27 268:135 269:253 270:253 271:100 296:153 297:253 298:240 299:76 323:12 324:208 325:253 326:166 351:69 352:253 353:253 354:142 378:14 379:110 380:253 381:235 382:33 406:63 407:223 408:235 409:130 434:186 435:253 436:235 437:37 461:17 462:145 463:253 464:231 465:35 489:69 490:220 491:231 492:123 516:18 517:205 518:253 519:176 520:27 543:17 544:125 545:253 546:185 547:39 571:71 572:214 573:231 574:41 599:167 600:253 601:225 602:33 626:72 627:205 628:207 629:14 653:30 654:249 655:233 656:49 681:32 682:253 683:89 +1 126:94 127:132 154:250 155:250 156:4 182:250 183:254 184:95 210:250 211:254 212:95 238:250 239:254 240:95 266:250 267:254 268:95 294:250 295:254 296:95 322:250 323:254 324:95 350:250 351:254 352:95 378:250 379:254 380:95 405:77 406:254 407:250 408:19 433:96 434:254 435:249 461:53 462:253 463:252 464:43 490:250 491:251 492:32 517:85 518:254 519:249 545:96 546:254 547:249 573:83 574:254 575:250 576:14 602:250 603:254 604:95 630:250 631:255 632:95 658:132 659:254 660:95 +1 124:32 125:253 126:31 152:32 153:251 154:149 180:32 181:251 182:188 208:32 209:251 210:188 236:32 237:251 238:228 239:59 264:32 265:253 266:253 267:95 292:28 293:236 294:251 295:114 321:127 322:251 323:251 349:127 350:251 351:251 377:48 378:232 379:251 406:223 407:253 408:159 434:221 435:251 436:158 462:142 463:251 464:158 490:64 491:251 492:242 493:55 518:64 519:251 520:253 521:161 546:64 547:253 548:255 549:221 574:16 575:181 576:253 577:220 603:79 604:253 605:236 606:63 632:213 633:251 634:126 660:96 661:251 662:126 +1 129:39 130:254 131:255 132:254 133:140 157:136 158:253 159:253 160:228 161:67 184:6 185:227 186:253 187:253 188:58 211:29 212:188 213:253 214:253 215:253 216:17 239:95 240:253 241:253 242:253 243:157 244:8 266:3 267:107 268:253 269:253 270:245 271:77 294:29 295:253 296:253 297:240 298:100 322:141 323:253 324:253 325:215 349:129 350:248 351:253 352:253 353:215 377:151 378:253 379:253 380:253 381:144 405:151 406:253 407:253 408:253 409:27 431:3 432:102 433:242 434:253 435:253 436:110 437:3 459:97 460:253 461:253 462:253 463:214 464:55 487:207 488:253 489:253 490:253 491:158 515:67 516:253 517:253 518:253 519:158 543:207 544:253 545:253 546:240 547:88 571:207 572:253 573:253 574:224 598:32 599:217 600:253 601:253 602:224 626:141 627:253 628:253 629:253 630:133 654:36 655:219 656:253 657:140 658:10 +0 123:59 124:55 149:71 150:192 151:254 152:250 153:147 154:17 176:123 177:247 178:253 179:254 180:253 181:253 182:196 183:79 184:176 185:175 186:175 187:124 188:48 203:87 204:247 205:247 206:176 207:95 208:102 209:117 210:243 211:237 212:192 213:232 214:253 215:253 216:245 217:152 218:6 230:23 231:229 232:253 233:138 238:219 239:58 241:95 242:118 243:80 244:230 245:254 246:196 247:30 258:120 259:254 260:205 261:8 266:114 272:38 273:255 274:254 275:155 276:5 286:156 287:253 288:92 301:61 302:235 303:253 304:102 314:224 315:253 316:78 330:117 331:253 332:196 333:18 342:254 343:253 344:78 358:9 359:211 360:253 361:73 370:254 371:253 372:78 387:175 388:253 389:155 398:194 399:254 400:101 415:79 416:254 417:155 426:112 427:253 428:211 429:9 443:73 444:251 445:200 454:41 455:241 456:253 457:87 471:25 472:240 473:253 483:147 484:253 485:227 486:47 499:94 500:253 501:200 511:5 512:193 513:253 514:230 515:76 527:175 528:253 529:155 540:31 541:219 542:254 543:255 544:126 545:18 553:14 554:149 555:254 556:244 557:45 569:21 570:158 571:254 572:253 573:226 574:162 575:118 576:96 577:20 578:20 579:73 580:118 581:224 582:253 583:247 584:85 598:30 599:155 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:254 610:247 611:84 627:5 628:27 629:117 630:206 631:244 632:229 633:213 634:213 635:213 636:176 637:117 638:32 659:45 660:23 +1 128:58 129:139 156:247 157:247 158:25 183:121 184:253 185:156 186:3 211:133 212:253 213:145 238:11 239:227 240:253 241:145 266:7 267:189 268:253 269:145 294:35 295:252 296:253 297:145 322:146 323:252 324:253 325:131 350:146 351:252 352:253 353:13 378:146 379:252 380:253 381:13 406:147 407:253 408:255 409:13 434:146 435:252 436:253 437:13 462:146 463:252 464:253 465:13 490:146 491:252 492:253 493:13 517:22 518:230 519:252 520:221 521:9 545:22 546:230 547:252 548:133 574:146 575:252 576:133 602:146 603:252 604:120 630:146 631:252 658:146 659:252 +1 129:28 130:247 131:255 132:165 156:47 157:221 158:252 159:252 160:164 184:177 185:252 186:252 187:252 188:164 212:177 213:252 214:252 215:223 216:78 240:177 241:252 242:252 243:197 267:114 268:236 269:252 270:235 271:42 294:5 295:148 296:252 297:252 298:230 321:14 322:135 323:252 324:252 325:252 326:230 349:78 350:252 351:252 352:252 353:252 354:162 377:78 378:252 379:252 380:252 381:252 382:9 405:78 406:252 407:252 408:252 409:252 410:9 432:32 433:200 434:252 435:252 436:252 437:105 438:3 459:10 460:218 461:252 462:252 463:252 464:105 465:8 487:225 488:252 489:252 490:252 491:240 492:69 514:44 515:237 516:252 517:252 518:228 519:85 541:59 542:218 543:252 544:252 545:225 546:93 568:65 569:208 570:252 571:252 572:252 573:175 596:133 597:252 598:252 599:252 600:225 601:68 624:133 625:252 626:252 627:244 628:54 652:133 653:252 654:252 655:48 +0 156:13 157:6 181:10 182:77 183:145 184:253 185:190 186:67 207:11 208:77 209:193 210:252 211:252 212:253 213:252 214:238 215:157 216:71 217:26 233:10 234:78 235:193 236:252 237:252 238:252 239:252 240:253 241:252 242:252 243:252 244:252 245:228 246:128 247:49 248:5 259:6 260:78 261:194 262:252 263:252 264:252 265:252 266:252 267:252 268:253 269:217 270:192 271:232 272:252 273:252 274:252 275:252 276:135 277:3 286:4 287:147 288:252 289:252 290:252 291:252 292:252 293:252 294:252 295:252 296:175 297:26 299:40 300:145 301:235 302:252 303:252 304:252 305:104 314:208 315:252 316:252 317:252 318:252 319:252 320:252 321:133 322:48 323:48 329:71 330:236 331:252 332:252 333:230 342:253 343:185 344:170 345:252 346:252 347:252 348:173 349:22 358:102 359:252 360:252 361:252 370:24 371:141 372:243 373:252 374:252 375:186 376:5 386:8 387:220 388:252 389:252 398:70 399:247 400:252 401:252 402:165 403:37 414:81 415:251 416:252 417:194 426:255 427:253 428:253 429:251 430:69 441:39 442:231 443:253 444:253 445:127 454:253 455:252 456:249 457:127 468:6 469:147 470:252 471:252 472:190 473:5 482:253 483:252 484:216 495:7 496:145 497:252 498:252 499:252 500:69 510:253 511:252 512:223 513:16 522:25 523:185 524:252 525:252 526:252 527:107 528:8 538:167 539:252 540:252 541:181 542:18 549:105 550:191 551:252 552:252 553:235 554:151 555:10 566:37 567:221 568:252 569:252 570:210 571:193 572:96 573:73 574:130 575:188 576:194 577:227 578:252 579:252 580:235 581:128 595:97 596:220 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:253 605:252 606:252 607:236 608:70 624:40 625:174 626:252 627:252 628:252 629:252 630:252 631:252 632:253 633:197 634:138 635:29 653:5 654:23 655:116 656:143 657:143 658:143 659:143 660:24 661:10 +0 127:28 128:164 129:254 130:233 131:148 132:11 154:3 155:164 156:254 157:234 158:225 159:254 160:204 182:91 183:254 184:235 185:48 186:32 187:166 188:251 189:92 208:33 209:111 210:214 211:205 212:49 215:24 216:216 217:210 235:34 236:217 237:254 238:254 239:211 244:87 245:237 246:43 262:34 263:216 264:254 265:254 266:252 267:243 268:61 272:38 273:248 274:182 290:171 291:254 292:184 293:205 294:175 295:36 301:171 302:227 317:28 318:234 319:190 320:13 321:193 322:157 329:124 330:238 331:26 345:140 346:254 347:131 349:129 350:157 357:124 358:254 359:95 373:201 374:238 375:56 377:70 378:103 385:124 386:254 387:148 400:62 401:255 402:210 413:150 414:254 415:122 428:86 429:254 430:201 431:15 440:28 441:237 442:246 443:44 456:128 457:254 458:143 468:34 469:243 470:227 484:62 485:254 486:210 496:58 497:249 498:179 512:30 513:240 514:210 524:207 525:254 526:64 541:216 542:231 543:34 551:129 552:248 553:170 554:9 569:131 570:254 571:170 577:17 578:129 579:248 580:225 581:24 597:50 598:245 599:245 600:184 601:106 602:106 603:106 604:133 605:231 606:254 607:244 608:53 626:67 627:249 628:254 629:254 630:254 631:254 632:254 633:251 634:193 635:40 655:38 656:157 657:248 658:166 659:166 660:139 661:57 +0 129:105 130:255 131:219 132:67 133:67 134:52 156:20 157:181 158:253 159:253 160:253 161:253 162:226 163:69 182:4 183:129 184:206 185:253 186:253 187:253 188:253 189:253 190:253 191:130 209:9 210:141 211:253 212:253 213:253 214:253 215:253 216:253 217:253 218:253 219:166 220:20 237:134 238:253 239:253 240:253 241:253 242:253 243:253 244:253 245:253 246:253 247:253 248:65 262:2 263:83 264:207 265:246 266:253 267:253 268:253 269:253 270:253 271:249 272:234 273:247 274:253 275:253 276:65 290:83 291:253 292:253 293:253 294:253 295:253 296:189 297:253 298:253 299:205 301:179 302:253 303:253 304:65 317:85 318:234 319:253 320:253 321:253 322:253 323:157 324:26 325:164 326:151 327:83 329:179 330:253 331:253 332:65 344:65 345:237 346:253 347:253 348:253 349:67 350:36 351:14 353:15 354:12 357:179 358:253 359:253 360:65 371:4 372:141 373:253 374:253 375:221 376:158 377:23 385:179 386:253 387:253 388:65 399:129 400:253 401:253 402:241 403:62 412:72 413:226 414:253 415:175 416:24 426:119 427:247 428:253 429:253 430:206 439:8 440:134 441:253 442:253 443:130 454:132 455:253 456:253 457:194 458:27 467:125 468:253 469:253 470:253 471:130 481:45 482:213 483:253 484:253 485:112 493:70 494:170 495:247 496:253 497:253 498:89 499:43 509:67 510:253 511:253 512:196 513:55 514:9 520:8 521:131 522:253 523:253 524:253 525:86 526:1 537:67 538:253 539:253 540:253 541:253 542:129 546:43 547:114 548:134 549:253 550:253 551:231 552:139 553:41 565:20 566:167 567:253 568:253 569:253 570:247 571:179 572:179 573:179 574:206 575:253 576:253 577:253 578:253 579:72 594:103 595:240 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:244 605:119 606:8 607:1 623:107 624:253 625:253 626:253 627:253 628:253 629:253 630:253 631:175 632:111 651:3 652:121 653:253 654:253 655:253 656:253 657:253 658:182 659:24 +0 125:22 126:183 127:252 128:254 129:252 130:252 131:252 132:76 151:85 152:85 153:168 154:250 155:250 156:252 157:250 158:250 159:250 160:250 161:71 163:43 164:85 165:14 178:107 179:252 180:250 181:250 182:250 183:250 184:252 185:250 186:250 187:250 188:250 189:210 191:127 192:250 193:146 205:114 206:237 207:252 208:250 209:250 210:250 211:250 212:252 213:250 214:250 215:250 216:250 217:210 219:127 220:250 221:250 232:107 233:237 234:250 235:252 236:250 237:250 238:250 239:74 240:41 241:41 242:41 243:41 244:217 245:34 247:127 248:250 249:250 259:15 260:148 261:252 262:252 263:254 264:238 265:105 275:128 276:252 277:252 286:15 287:140 288:250 289:250 290:250 291:167 292:111 303:127 304:250 305:250 314:43 315:250 316:250 317:250 318:250 331:127 332:250 333:250 342:183 343:250 344:250 345:250 346:110 358:57 359:210 360:250 361:250 370:252 371:250 372:250 373:110 374:7 386:85 387:250 388:250 389:250 398:254 399:252 400:252 401:83 414:86 415:252 416:252 417:217 426:252 427:250 428:250 429:138 430:14 441:15 442:140 443:250 444:250 445:41 454:252 455:250 456:250 457:250 458:41 469:43 470:250 471:250 472:250 473:41 482:252 483:250 484:250 485:250 486:181 497:183 498:250 499:250 500:250 501:41 510:76 511:250 512:250 513:250 514:250 524:177 525:252 526:250 527:250 528:110 529:7 538:36 539:224 540:252 541:252 542:252 543:219 544:43 545:43 546:43 547:7 549:15 550:43 551:183 552:252 553:255 554:252 555:126 567:85 568:250 569:250 570:250 571:252 572:250 573:250 574:250 575:111 576:86 577:140 578:250 579:250 580:250 581:252 582:222 583:83 595:42 596:188 597:250 598:250 599:252 600:250 601:250 602:250 603:250 604:252 605:250 606:250 607:250 608:250 609:126 610:83 624:127 625:250 626:250 627:252 628:250 629:250 630:250 631:250 632:252 633:250 634:250 635:137 636:83 652:21 653:41 654:217 655:252 656:250 657:250 658:250 659:250 660:217 661:41 662:41 663:14 +1 155:114 156:206 157:25 183:238 184:252 185:55 211:222 212:252 213:55 239:113 240:252 241:55 267:113 268:252 269:55 295:255 296:253 297:56 323:253 324:176 325:6 350:32 351:253 352:233 353:43 378:140 379:253 380:195 381:19 406:140 407:253 408:167 433:29 434:253 435:141 461:29 462:252 463:140 489:29 490:252 491:140 517:29 518:252 519:140 545:29 546:252 547:140 573:169 574:253 575:79 601:169 602:252 628:76 629:234 630:141 656:197 657:233 658:37 684:197 685:223 +1 127:73 128:253 129:253 130:63 155:115 156:252 157:252 158:144 183:217 184:252 185:252 186:144 210:63 211:237 212:252 213:252 214:144 238:109 239:252 240:252 241:252 266:109 267:252 268:252 269:252 294:109 295:252 296:252 297:252 322:191 323:252 324:252 325:252 349:145 350:255 351:253 352:253 353:253 376:32 377:237 378:253 379:252 380:252 381:210 404:37 405:252 406:253 407:252 408:252 409:108 432:37 433:252 434:253 435:252 436:252 437:108 460:21 461:207 462:255 463:253 464:253 465:108 489:144 490:253 491:252 492:252 493:108 516:27 517:221 518:253 519:252 520:252 521:108 544:16 545:190 546:253 547:252 548:252 549:108 573:145 574:255 575:253 576:253 577:253 601:144 602:253 603:252 604:252 605:210 629:144 630:253 631:252 632:252 633:108 657:62 658:253 659:252 660:252 661:108 +1 120:85 121:253 122:132 123:9 147:82 148:241 149:251 150:251 151:128 175:175 176:251 177:251 178:251 179:245 180:121 203:13 204:204 205:251 206:251 207:251 208:245 209:107 232:39 233:251 234:251 235:251 236:251 237:167 238:22 260:15 261:155 262:251 263:251 264:251 265:251 266:177 289:15 290:157 291:248 292:251 293:251 294:251 295:165 319:214 320:251 321:251 322:251 323:212 324:78 325:24 347:109 348:251 349:251 350:251 351:253 352:251 353:170 354:10 375:5 376:57 377:162 378:251 379:253 380:251 381:251 382:18 405:106 406:239 407:255 408:253 409:253 410:213 434:105 435:253 436:251 437:251 438:230 439:72 463:253 464:251 465:251 466:251 467:221 468:67 491:72 492:251 493:251 494:251 495:251 496:96 519:36 520:199 521:251 522:251 523:251 524:155 525:15 548:45 549:204 550:251 551:251 552:251 553:157 577:161 578:249 579:251 580:251 581:248 582:147 606:233 607:251 608:251 609:251 610:173 634:233 635:251 636:251 637:251 638:173 662:53 663:131 664:251 665:251 666:173 +1 126:15 127:200 128:255 129:90 154:42 155:254 156:254 157:173 182:42 183:254 184:254 185:199 210:26 211:237 212:254 213:221 214:12 239:213 240:254 241:231 242:17 267:213 268:254 269:199 295:213 296:254 297:199 323:213 324:254 325:96 350:20 351:232 352:254 353:33 378:84 379:254 380:229 381:17 406:168 407:254 408:203 433:8 434:217 435:254 436:187 461:84 462:254 463:254 464:48 489:195 490:254 491:254 492:37 516:20 517:233 518:254 519:212 520:4 544:132 545:254 546:254 547:82 571:9 572:215 573:254 574:254 575:116 576:46 599:55 600:254 601:254 602:254 603:254 604:121 627:113 628:254 629:254 630:254 631:254 632:40 655:12 656:163 657:254 658:185 659:58 660:1 +0 182:32 183:57 184:57 185:57 186:57 187:57 188:57 189:57 208:67 209:185 210:229 211:252 212:252 213:252 214:253 215:252 216:252 217:252 218:185 219:66 234:13 235:188 236:246 237:252 238:253 239:252 240:252 241:252 242:241 243:139 244:177 245:252 246:253 247:246 248:187 249:13 261:26 262:255 263:253 264:244 265:175 266:101 274:126 275:244 276:253 277:153 288:82 289:243 290:253 291:214 292:81 303:169 304:252 305:252 315:19 316:215 317:252 318:206 319:56 331:169 332:252 333:252 343:157 344:252 345:252 346:13 359:169 360:252 361:151 370:41 371:253 372:253 373:128 386:92 387:253 388:206 389:13 398:166 399:252 400:196 401:9 414:216 415:252 416:142 426:253 427:252 428:168 441:89 442:253 443:208 444:13 454:253 455:252 456:68 468:38 469:225 470:253 471:96 482:254 483:253 484:56 495:45 496:229 497:253 498:151 510:253 511:252 512:81 522:70 523:225 524:252 525:227 538:216 539:252 540:168 548:29 549:134 550:253 551:252 552:186 553:31 566:91 567:252 568:243 569:125 573:51 574:114 575:113 576:210 577:252 578:253 579:151 580:19 595:157 596:253 597:253 598:254 599:253 600:253 601:253 602:254 603:253 604:244 605:175 606:51 623:19 624:122 625:196 626:197 627:221 628:196 629:196 630:197 631:121 632:56 655:25 +0 127:42 128:235 129:255 130:84 153:15 154:132 155:208 156:253 157:253 158:171 159:108 180:6 181:177 182:253 183:253 184:253 185:253 186:253 187:242 188:110 208:151 209:253 210:253 211:253 212:253 213:253 214:253 215:253 216:139 235:48 236:208 237:253 238:253 239:253 240:253 241:253 242:253 243:253 244:139 263:85 264:253 265:253 266:253 267:253 268:236 269:156 270:184 271:253 272:148 273:6 290:7 291:141 292:253 293:253 294:253 295:253 296:27 298:170 299:253 300:253 301:74 318:19 319:253 320:253 321:253 322:253 323:253 324:27 326:170 327:253 328:253 329:74 345:16 346:186 347:253 348:253 349:253 350:242 351:105 352:4 354:170 355:253 356:253 357:94 358:1 373:141 374:253 375:253 376:253 377:242 378:100 382:170 383:253 384:253 385:253 386:8 401:141 402:253 403:253 404:253 405:224 410:170 411:253 412:253 413:253 414:8 428:12 429:158 430:253 431:253 432:230 433:51 438:18 439:237 440:253 441:253 442:8 456:76 457:253 458:253 459:218 460:61 467:236 468:253 469:253 470:8 484:76 485:253 486:253 487:168 495:110 496:253 497:132 498:3 512:76 513:253 514:253 515:168 521:20 522:174 523:239 524:147 525:5 539:5 540:155 541:253 542:253 543:168 548:102 549:170 550:253 551:253 552:139 567:3 568:128 569:253 570:253 571:228 572:179 573:179 574:179 575:179 576:245 577:253 578:253 579:219 580:41 596:76 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:253 605:253 606:253 607:163 624:39 625:199 626:253 627:253 628:253 629:253 630:253 631:253 632:253 633:253 634:170 635:9 653:36 654:219 655:253 656:253 657:253 658:253 659:253 660:224 661:65 662:22 +1 156:202 157:253 158:69 184:253 185:252 186:121 212:253 213:252 214:69 240:253 241:252 242:69 267:106 268:253 269:231 270:37 295:179 296:255 297:196 322:17 323:234 324:253 325:92 350:93 351:252 352:253 353:92 378:93 379:252 380:253 381:92 406:93 407:252 408:232 409:8 434:208 435:253 436:116 462:207 463:252 464:116 490:207 491:252 492:32 517:57 518:244 519:252 545:122 546:252 547:252 573:185 574:253 575:253 601:184 602:252 603:252 629:101 630:252 631:252 657:13 658:173 659:252 660:43 686:9 687:232 688:116 +1 156:73 157:253 158:253 159:253 160:124 184:73 185:251 186:251 187:251 188:251 212:99 213:251 214:251 215:251 216:225 240:253 241:251 242:251 243:251 244:71 266:79 267:180 268:253 269:251 270:251 271:173 272:20 294:110 295:253 296:255 297:253 298:216 322:109 323:251 324:253 325:251 326:215 350:109 351:251 352:253 353:251 354:215 378:109 379:251 380:253 381:251 382:137 406:109 407:251 408:253 409:251 410:35 433:37 434:253 435:253 436:255 437:253 438:35 461:140 462:251 463:251 464:253 465:168 466:15 488:125 489:246 490:251 491:251 492:190 493:15 516:144 517:251 518:251 519:251 520:180 543:53 544:221 545:251 546:251 547:251 548:51 571:125 572:253 573:253 574:253 575:201 598:105 599:253 600:251 601:251 602:188 603:30 626:180 627:253 628:251 629:251 630:142 654:180 655:253 656:251 657:235 658:82 682:180 683:253 684:251 685:215 +1 124:111 125:255 126:48 152:162 153:253 154:237 155:63 180:206 181:253 182:253 183:183 208:87 209:217 210:253 211:205 237:90 238:253 239:238 240:60 265:37 266:225 267:253 268:89 294:206 295:253 296:159 322:206 323:253 324:226 350:206 351:253 352:226 378:206 379:253 380:226 406:206 407:253 408:226 434:206 435:253 436:226 462:206 463:253 464:226 490:206 491:253 492:226 518:206 519:253 520:237 521:45 546:206 547:253 548:253 549:109 574:173 575:253 576:253 577:109 602:69 603:253 604:253 605:109 630:64 631:248 632:253 633:109 659:112 660:253 661:109 +0 99:70 100:255 101:165 102:114 127:122 128:253 129:253 130:253 131:120 155:165 156:253 157:253 158:253 159:234 160:52 183:99 184:253 185:253 186:253 187:253 188:228 189:26 209:60 210:168 211:238 212:202 213:174 214:253 215:253 216:253 217:127 235:91 236:81 237:1 238:215 239:128 240:28 241:12 242:181 243:253 244:253 245:175 246:3 262:18 263:204 264:253 265:77 270:7 271:253 272:253 273:253 274:54 289:54 290:248 291:253 292:253 293:143 298:1 299:127 300:253 301:253 302:188 317:104 318:253 319:253 320:253 321:20 327:81 328:249 329:253 330:191 345:192 346:253 347:253 348:218 349:5 356:203 357:253 358:208 359:21 372:56 373:237 374:253 375:250 376:100 384:104 385:253 386:253 387:75 400:76 401:253 402:253 403:224 412:119 413:253 414:253 415:75 428:80 429:253 430:253 431:103 439:4 440:241 441:253 442:218 443:32 456:213 457:253 458:253 459:103 467:125 468:253 469:253 470:191 484:213 485:253 486:253 487:103 494:3 495:176 496:253 497:253 498:135 512:213 513:253 514:253 515:103 521:9 522:162 523:253 524:253 525:226 526:37 540:179 541:253 542:253 543:135 548:46 549:157 550:253 551:253 552:253 553:63 568:23 569:188 570:253 571:249 572:179 573:179 574:179 575:179 576:233 577:253 578:253 579:233 580:156 581:10 597:51 598:235 599:253 600:253 601:253 602:253 603:253 604:253 605:251 606:232 607:120 626:16 627:124 628:253 629:253 630:253 631:253 632:152 633:104 +1 124:29 125:197 126:255 127:84 152:85 153:251 154:253 155:83 180:86 181:253 182:254 183:253 208:85 209:251 210:253 211:251 236:86 237:253 238:254 239:253 240:169 264:85 265:251 266:253 267:251 268:168 292:86 293:253 294:254 295:253 296:169 320:28 321:196 322:253 323:251 324:168 349:169 350:254 351:253 352:169 377:168 378:253 379:251 380:168 405:169 406:254 407:253 408:169 433:168 434:253 435:251 436:168 462:254 463:253 464:254 465:139 490:253 491:251 492:253 493:251 518:254 519:253 520:254 521:253 522:57 546:253 547:251 548:253 549:251 550:168 574:198 575:253 576:254 577:253 578:114 602:85 603:251 604:253 605:251 630:85 631:253 632:254 633:253 658:28 659:83 660:196 661:83 +1 159:31 160:210 161:253 162:163 187:198 188:252 189:252 190:162 213:10 214:86 215:242 216:252 217:252 218:66 241:164 242:252 243:252 244:252 245:188 246:8 268:53 269:242 270:252 271:252 272:225 273:14 296:78 297:252 298:252 299:252 300:204 323:56 324:231 325:252 326:252 327:212 328:35 351:157 352:252 353:252 354:252 355:37 377:8 378:132 379:253 380:252 381:252 382:230 383:24 405:45 406:252 407:253 408:252 409:154 410:55 427:7 428:55 433:107 434:253 435:255 436:228 437:53 454:15 455:24 456:23 460:110 461:242 462:252 463:228 464:59 482:57 483:83 487:88 488:247 489:252 490:252 491:140 514:15 515:189 516:252 517:252 518:252 542:74 543:252 544:252 545:238 546:90 570:178 571:252 572:252 573:189 597:40 598:217 599:252 600:252 601:59 625:75 626:252 627:252 628:252 629:85 630:61 653:62 654:239 655:252 656:156 657:14 682:178 683:252 684:14 +1 131:159 132:255 133:122 158:167 159:228 160:253 161:121 185:64 186:236 187:251 188:205 189:110 212:48 213:158 214:251 215:251 216:178 217:39 240:190 241:251 242:251 243:251 267:96 268:253 269:253 270:253 271:153 295:194 296:251 297:251 298:211 299:74 322:80 323:174 324:251 325:251 326:140 327:47 349:16 350:181 351:253 352:251 353:219 354:23 377:64 378:251 379:253 380:251 381:204 382:19 405:223 406:253 407:255 408:233 409:48 431:20 432:174 433:244 434:251 435:253 436:109 437:31 459:96 460:189 461:251 462:251 463:126 464:31 486:24 487:106 488:251 489:235 490:188 491:100 514:96 515:251 516:251 517:228 518:59 542:255 543:253 544:253 545:213 546:36 569:100 570:253 571:251 572:251 573:85 574:23 596:32 597:127 598:253 599:235 600:126 601:15 624:104 625:251 626:253 627:240 628:79 652:83 653:193 654:253 655:220 +0 153:92 154:191 155:178 156:253 157:242 158:141 159:104 160:29 180:26 181:253 182:252 183:252 184:252 185:253 186:252 187:252 188:252 189:108 190:19 206:57 207:123 208:222 209:253 210:252 211:252 212:252 213:168 214:224 215:252 216:252 217:253 218:84 233:176 234:243 235:252 236:252 237:253 238:252 239:252 240:252 242:19 243:153 244:252 245:253 246:209 247:25 259:10 260:128 261:255 262:253 263:244 264:225 265:114 266:194 267:253 268:178 272:163 273:254 274:253 275:168 287:85 288:252 289:253 290:189 291:56 294:19 295:133 296:9 300:38 301:253 302:252 303:168 314:19 315:191 316:252 317:194 318:19 329:253 330:252 331:234 332:22 342:107 343:252 344:252 345:13 357:253 358:252 359:252 360:128 370:169 371:253 372:241 385:141 386:253 387:253 388:140 397:19 398:225 399:252 400:139 413:66 414:252 415:252 416:139 425:29 426:252 427:252 428:52 441:29 442:252 443:252 444:139 453:29 454:252 455:252 456:28 469:29 470:252 471:252 472:40 481:141 482:253 483:253 484:91 497:154 498:253 499:168 509:66 510:252 511:252 512:165 525:253 526:252 527:168 537:19 538:224 539:252 540:252 552:126 553:253 554:252 555:80 566:169 567:252 568:252 569:214 570:38 579:126 580:249 581:253 582:151 583:6 594:26 595:223 596:253 597:254 598:253 599:128 600:29 604:13 605:41 606:216 607:253 608:253 609:226 610:38 623:122 624:252 625:253 626:252 627:252 628:252 629:169 630:169 631:169 632:206 633:253 634:252 635:252 636:202 637:38 651:19 652:56 653:168 654:224 655:252 656:252 657:253 658:252 659:252 660:252 661:253 662:233 663:130 664:6 682:94 683:139 684:190 685:153 686:252 687:164 688:139 689:28 690:22 +1 128:53 129:250 130:255 131:25 156:167 157:253 158:253 159:25 182:3 183:123 184:247 185:253 186:253 187:25 210:9 211:253 212:253 213:253 214:253 215:25 238:9 239:253 240:253 241:253 242:253 243:25 266:9 267:253 268:253 269:253 270:180 271:13 294:9 295:253 296:253 297:253 298:104 322:9 323:253 324:253 325:253 326:104 350:15 351:253 352:253 353:253 354:104 378:184 379:253 380:253 381:228 382:68 406:184 407:253 408:253 409:182 433:103 434:251 435:253 436:253 437:12 461:106 462:253 463:253 464:253 465:8 488:24 489:238 490:253 491:253 492:253 493:8 516:27 517:253 518:253 519:253 520:253 521:8 544:27 545:253 546:253 547:253 548:253 549:8 572:27 573:253 574:253 575:253 576:177 577:4 600:160 601:253 602:253 603:253 604:87 628:202 629:253 630:253 631:219 632:54 656:81 657:253 658:247 659:51 +0 122:63 123:176 124:253 125:253 126:159 127:113 128:63 150:140 151:253 152:252 153:252 154:252 155:252 156:241 157:100 158:66 177:54 178:227 179:253 180:252 181:252 182:252 183:252 184:253 185:252 186:239 187:181 188:57 204:38 205:224 206:252 207:253 208:226 209:246 210:252 211:252 212:253 213:252 214:252 215:252 216:252 217:108 218:3 232:57 233:252 234:252 235:253 236:27 237:88 238:112 239:112 240:112 241:112 242:142 243:252 244:252 245:253 246:152 247:31 260:198 261:253 262:253 263:79 270:32 271:153 272:253 273:255 274:253 275:196 287:76 288:246 289:252 290:127 299:3 300:106 301:253 302:252 303:214 304:28 315:194 316:252 317:252 318:112 329:143 330:252 331:252 332:193 343:225 344:252 345:217 346:37 357:38 358:234 359:252 360:223 370:63 371:240 372:252 373:84 386:146 387:252 388:223 398:114 399:253 400:228 401:47 414:147 415:253 416:253 417:112 426:159 427:252 428:195 442:225 443:252 444:252 445:112 454:253 455:252 456:195 470:225 471:252 472:230 473:25 482:159 483:252 484:202 485:10 497:92 498:243 499:252 500:208 510:113 511:252 512:252 513:161 524:79 525:253 526:252 527:220 528:37 538:114 539:253 540:253 541:253 542:174 543:63 550:26 551:128 552:253 553:255 554:253 555:133 566:12 567:228 568:252 569:252 570:252 571:241 572:100 573:85 574:76 576:85 577:131 578:231 579:252 580:252 581:253 582:129 583:6 595:97 596:208 597:252 598:252 599:253 600:252 601:252 602:246 603:197 604:253 605:252 606:252 607:252 608:220 609:133 610:6 624:19 625:99 626:239 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:245 635:223 636:99 654:63 655:112 656:112 657:221 658:252 659:252 660:253 661:127 662:87 +0 153:12 154:136 155:254 156:255 157:195 158:115 159:3 180:6 181:175 182:253 183:196 184:160 185:252 186:253 187:15 208:130 209:253 210:234 211:4 213:27 214:205 215:232 216:40 235:54 236:246 237:253 238:68 242:24 243:243 244:106 262:3 263:134 264:235 265:99 266:4 271:132 272:247 273:77 290:56 291:253 292:62 299:23 300:233 301:129 318:179 319:183 320:4 328:182 329:220 345:21 346:232 347:59 356:95 357:232 358:21 373:128 374:183 385:228 386:85 401:187 402:124 413:228 414:186 429:187 430:124 441:228 442:104 457:187 458:124 469:169 470:184 485:187 486:124 497:203 498:150 513:187 514:124 524:10 525:220 526:39 541:187 542:155 552:111 553:201 569:129 570:228 571:7 579:12 580:181 581:76 598:234 599:166 600:9 606:24 607:209 608:106 626:139 627:250 628:167 629:11 630:2 631:11 632:11 633:129 634:227 635:90 636:11 655:95 656:247 657:253 658:178 659:253 660:253 661:244 662:86 684:47 685:175 686:253 687:232 688:149 689:40 +1 128:255 129:253 130:57 156:253 157:251 158:225 159:56 183:169 184:254 185:253 186:254 187:84 211:168 212:253 213:251 214:253 215:83 238:85 239:253 240:254 241:253 242:169 266:85 267:251 268:253 269:251 270:56 294:141 295:253 296:254 297:253 322:253 323:251 324:253 325:251 350:254 351:253 352:254 353:253 378:253 379:251 380:253 381:251 406:254 407:253 408:254 409:196 433:114 434:253 435:251 436:253 437:83 461:169 462:254 463:253 464:226 465:56 489:168 490:253 491:251 492:168 516:85 517:253 518:254 519:253 544:85 545:251 546:253 547:251 572:254 573:253 574:254 575:253 600:253 601:251 602:253 603:251 628:254 629:253 630:254 631:253 656:139 657:251 658:253 659:138 +0 151:23 152:167 153:208 154:254 155:255 156:129 157:19 179:151 180:253 181:253 182:253 183:253 184:253 185:209 186:26 207:181 208:253 209:253 210:253 211:227 212:181 213:253 214:207 215:22 235:227 236:253 237:253 238:253 239:92 240:38 241:226 242:253 243:129 244:2 263:193 264:253 265:253 266:248 267:62 269:50 270:253 271:253 272:45 291:170 292:253 293:253 294:135 297:12 298:208 299:253 300:119 318:16 319:232 320:253 321:253 322:21 326:60 327:253 328:185 346:164 347:253 348:253 349:224 350:14 354:14 355:217 356:247 357:62 373:3 374:193 375:253 376:250 377:64 383:199 384:253 385:179 401:67 402:253 403:253 404:205 411:98 412:253 413:188 429:151 430:253 431:245 432:43 439:63 440:250 441:188 457:151 458:253 459:243 468:244 469:222 470:22 485:151 486:253 487:217 496:244 497:253 498:115 512:3 513:195 514:253 515:134 524:156 525:253 526:150 541:140 542:253 543:134 552:239 553:253 554:139 569:44 570:253 571:134 579:53 580:246 581:237 582:32 597:8 598:200 599:229 600:40 606:25 607:225 608:253 609:188 626:120 627:250 628:230 629:58 630:17 632:12 633:42 634:213 635:253 636:238 637:84 655:151 656:253 657:253 658:217 659:179 660:206 661:253 662:253 663:196 664:118 683:18 684:58 685:145 686:152 687:253 688:214 689:145 690:74 691:7 +1 130:24 131:150 132:233 133:38 156:14 157:89 158:253 159:254 160:254 161:71 183:78 184:203 185:254 186:254 187:254 188:232 189:77 190:54 191:8 209:12 210:155 211:240 212:254 213:223 214:76 215:254 216:254 217:254 218:254 219:68 235:3 236:101 237:216 238:254 239:227 240:122 241:26 242:110 243:254 244:254 245:254 246:184 247:100 262:46 263:222 264:254 265:254 266:179 267:48 270:181 271:254 272:254 273:146 274:6 288:2 289:145 290:248 291:254 292:182 293:111 294:4 297:3 298:250 299:254 300:206 301:3 315:6 316:144 317:254 318:254 319:171 325:125 326:254 327:252 328:80 342:6 343:142 344:254 345:179 346:95 347:4 352:61 353:246 354:254 355:150 370:64 371:254 372:177 373:14 380:124 381:254 382:246 383:32 398:108 399:97 400:15 407:24 408:226 409:254 410:116 435:177 436:255 437:254 438:5 463:196 464:254 465:99 466:1 490:3 491:199 492:254 493:79 518:129 519:254 520:254 521:23 546:178 547:254 548:192 549:8 550:3 551:43 573:11 574:198 575:254 576:128 577:66 578:130 579:225 595:137 596:202 597:106 598:84 599:84 600:84 601:112 602:254 603:254 604:254 605:254 606:212 607:151 623:172 624:254 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:254 633:162 634:75 651:12 652:106 653:177 654:254 655:254 656:254 657:235 658:135 659:100 660:17 661:2 +0 125:120 126:253 127:253 128:63 151:38 152:131 153:246 154:252 155:252 156:203 157:15 179:222 180:252 181:252 182:252 183:252 184:166 185:38 205:4 206:107 207:253 208:252 209:252 210:252 211:252 212:253 213:224 214:137 215:26 233:107 234:252 235:253 236:252 237:220 238:128 239:252 240:253 241:252 242:252 243:239 244:140 261:170 262:253 263:255 264:168 267:79 268:192 269:253 270:253 271:253 272:253 273:255 274:90 288:51 289:243 290:252 291:215 292:33 296:12 297:74 298:233 299:252 300:252 301:253 302:195 303:19 316:166 317:252 318:252 319:31 326:43 327:149 328:195 329:253 330:252 331:177 332:19 343:57 344:234 345:252 346:252 357:237 358:252 359:252 360:180 361:13 371:85 372:252 373:252 374:173 385:50 386:237 387:252 388:252 389:112 399:226 400:253 401:240 402:63 414:163 415:253 416:253 417:112 426:38 427:234 428:252 429:176 442:85 443:252 444:252 445:158 454:113 455:252 456:252 457:84 470:19 471:209 472:252 473:252 482:207 483:252 484:252 485:84 498:10 499:203 500:252 501:236 510:253 511:252 512:252 513:84 526:85 527:252 528:252 529:112 538:114 539:253 540:253 541:146 553:51 554:159 555:253 556:240 557:63 566:75 567:243 568:252 569:249 570:146 579:57 580:85 581:238 582:252 583:252 584:99 595:116 596:252 597:252 598:252 599:198 600:197 601:165 602:57 603:57 604:57 605:182 606:197 607:234 608:252 609:253 610:233 611:164 612:19 623:28 624:84 625:180 626:252 627:253 628:252 629:252 630:252 631:252 632:253 633:252 634:252 635:252 636:252 637:225 638:71 653:13 654:112 655:253 656:252 657:252 658:252 659:252 660:253 661:252 662:252 663:157 664:112 +1 127:155 128:253 129:126 155:253 156:251 157:141 158:4 183:253 184:251 185:251 186:31 211:253 212:251 213:251 214:31 239:253 240:251 241:251 242:31 267:255 268:253 269:253 270:31 293:8 294:131 295:253 296:251 297:235 298:27 321:64 322:251 323:253 324:251 325:126 349:64 350:251 351:253 352:251 353:126 377:64 378:251 379:253 380:251 381:126 405:64 406:253 407:255 408:221 433:182 434:251 435:253 436:200 460:64 461:236 462:251 463:253 464:62 487:8 488:158 489:251 490:251 491:169 492:8 515:32 516:251 517:251 518:251 519:158 543:32 544:253 545:253 546:253 547:159 571:32 572:251 573:251 574:251 575:39 599:32 600:251 601:251 602:251 627:32 628:251 629:251 630:251 631:100 655:32 656:251 657:251 658:251 +0 101:88 102:127 103:5 126:19 127:58 128:20 129:14 130:217 131:19 152:7 153:146 154:247 155:253 156:235 157:27 158:84 159:81 180:126 181:253 182:164 183:19 184:15 187:156 188:9 208:214 209:222 210:34 215:234 216:58 235:59 236:254 237:116 243:235 244:58 263:141 264:251 265:72 271:151 272:140 291:224 292:233 299:136 300:223 319:254 320:218 327:136 328:253 347:254 348:135 355:136 356:253 374:23 375:255 376:114 383:137 384:231 402:98 403:254 404:122 411:136 412:155 430:98 431:254 432:106 439:166 440:155 458:98 459:254 460:128 467:234 468:193 486:98 487:254 488:135 494:61 495:248 496:118 515:255 516:238 517:18 521:13 522:224 523:254 524:58 543:201 544:253 545:128 546:2 548:5 549:150 550:253 551:167 552:9 571:18 572:226 573:253 574:49 575:31 576:156 577:253 578:228 579:13 600:147 601:253 602:243 603:241 604:254 605:227 606:43 628:5 629:126 630:245 631:253 632:231 633:46 +0 127:37 128:141 129:156 130:156 131:194 132:194 133:47 153:11 154:132 155:239 156:253 157:253 158:253 159:253 160:254 161:181 180:25 181:172 182:253 183:235 184:167 185:78 186:93 187:174 188:254 189:247 190:54 207:26 208:210 209:253 210:237 211:90 216:201 217:253 218:78 235:192 236:253 237:237 238:58 244:156 245:253 246:78 262:141 263:254 264:235 265:53 269:19 270:5 272:156 273:254 274:78 289:46 290:254 291:253 292:92 296:17 297:226 298:217 299:49 300:148 301:253 302:78 317:165 318:254 319:239 320:24 324:20 325:253 326:253 327:58 328:18 329:115 330:24 344:37 345:248 346:254 347:91 352:2 353:117 354:250 355:163 356:91 372:77 373:253 374:254 375:39 382:196 383:253 384:173 400:159 401:254 402:218 403:15 410:77 411:254 412:255 413:61 428:234 429:253 430:113 438:21 439:226 440:254 441:135 455:25 456:240 457:253 458:68 467:195 468:254 469:135 483:79 484:253 485:253 495:195 496:254 497:135 511:79 512:253 513:253 514:76 523:195 524:254 525:99 540:212 541:254 542:209 543:9 550:10 551:209 552:196 553:15 568:54 569:253 570:254 571:137 572:36 576:2 577:20 578:168 579:253 580:60 596:28 597:235 598:254 599:253 600:199 601:124 602:79 603:79 604:167 605:253 606:253 607:185 608:30 625:15 626:117 627:217 628:253 629:253 630:253 631:254 632:253 633:240 634:109 635:12 655:27 656:126 657:208 658:253 659:193 660:147 661:40 +0 154:32 155:134 156:218 157:254 158:254 159:254 160:217 161:84 176:44 177:208 178:215 179:156 180:35 181:119 182:236 183:246 184:136 185:91 186:69 187:151 188:249 189:246 190:78 203:44 204:230 205:254 206:254 207:254 208:254 209:254 210:196 211:48 216:60 217:224 218:210 219:24 231:118 232:254 233:202 234:19 235:201 236:254 237:181 238:9 245:35 246:233 247:168 259:193 260:223 261:34 263:59 264:163 265:236 266:15 274:140 275:205 276:8 286:60 287:254 288:176 293:38 302:54 303:237 304:80 314:59 315:254 316:93 331:131 332:200 342:59 343:240 344:24 359:79 360:214 370:59 371:234 387:67 388:248 389:54 398:59 399:234 416:235 417:58 426:60 427:235 443:79 444:255 445:59 454:59 455:251 456:66 471:79 472:250 473:54 482:59 483:254 484:108 499:146 500:214 510:5 511:203 512:187 513:3 526:4 527:188 528:199 539:118 540:254 541:57 554:96 555:254 556:117 567:16 568:237 569:224 570:14 581:14 582:187 583:206 584:8 596:88 597:252 598:186 599:16 608:16 609:187 610:252 611:125 625:100 626:254 627:237 628:94 629:24 635:13 636:214 637:254 638:166 653:3 654:57 655:215 656:248 657:241 658:235 659:197 660:137 661:137 662:137 663:231 664:238 665:155 666:25 684:57 685:155 686:246 687:254 688:254 689:254 690:254 691:147 692:36 +1 124:102 125:252 126:252 127:41 152:102 153:250 154:250 155:202 180:102 181:250 182:250 183:232 184:91 208:102 209:250 210:250 211:212 212:29 236:102 237:252 238:252 239:254 240:150 264:102 265:250 266:250 267:252 268:149 292:102 293:250 294:250 295:252 296:149 320:102 321:250 322:250 323:252 324:231 325:80 349:152 350:252 351:254 352:252 353:100 377:151 378:250 379:252 380:250 381:100 405:151 406:250 407:252 408:250 409:100 433:151 434:250 435:252 436:250 437:100 461:123 462:243 463:254 464:252 465:100 490:202 491:252 492:250 493:100 518:80 519:252 520:250 521:190 522:30 547:252 548:250 549:250 550:49 575:255 576:252 577:252 578:252 579:214 580:31 603:171 604:250 605:250 606:250 607:252 608:190 609:40 631:20 632:160 633:250 634:250 635:252 636:250 637:100 660:20 661:170 662:250 663:212 664:49 665:20 +0 124:20 125:121 126:197 127:253 128:64 151:23 152:200 153:252 154:252 155:252 156:184 157:6 178:25 179:197 180:252 181:252 182:252 183:252 184:253 185:228 186:107 187:15 205:26 206:196 207:252 208:252 209:252 210:252 211:252 212:253 213:252 214:252 215:219 216:178 217:21 233:186 234:252 235:238 236:94 237:67 238:224 239:217 240:53 241:109 242:245 243:252 244:252 245:213 246:63 260:98 261:242 262:252 263:101 266:39 267:31 270:109 271:128 272:241 273:252 274:207 275:97 287:17 288:230 289:252 290:241 291:56 300:109 301:252 302:252 303:229 304:17 314:13 315:192 316:252 317:243 318:96 328:25 329:127 330:252 331:252 332:120 342:121 343:252 344:252 345:165 357:125 358:252 359:252 360:246 361:70 370:190 371:252 372:252 373:39 385:26 386:210 387:252 388:252 389:119 398:255 399:253 400:159 414:22 415:209 416:253 417:183 426:253 427:252 428:103 443:34 444:252 445:252 454:253 455:252 456:26 471:27 472:252 473:252 482:253 483:252 484:168 485:13 499:70 500:252 501:209 510:147 511:252 512:252 513:75 526:68 527:233 528:252 529:119 538:121 539:252 540:252 541:189 542:40 552:15 553:82 554:231 555:252 556:214 557:31 566:38 567:135 568:248 569:252 570:231 571:145 572:41 573:41 574:41 575:41 576:20 577:24 578:37 579:83 580:194 581:252 582:252 583:212 584:33 596:83 597:213 598:252 599:252 600:252 601:252 602:252 603:252 604:204 605:213 606:243 607:252 608:252 609:252 610:212 611:34 625:34 626:140 627:238 628:248 629:252 630:252 631:252 632:253 633:252 634:252 635:241 636:238 637:238 638:75 656:82 657:119 658:119 659:119 660:120 661:119 662:119 663:19 +1 127:20 128:254 129:255 130:37 155:19 156:253 157:253 158:134 183:19 184:253 185:253 186:246 187:125 211:76 212:253 213:253 214:253 215:158 239:207 240:253 241:253 242:253 243:158 267:207 268:253 269:253 270:253 271:158 294:48 295:223 296:253 297:253 298:243 299:106 322:141 323:253 324:253 325:253 326:113 349:65 350:237 351:253 352:253 353:253 354:36 377:76 378:253 379:253 380:253 381:253 382:36 405:76 406:253 407:253 408:253 409:253 410:36 433:76 434:253 435:253 436:253 437:118 438:4 460:4 461:148 462:253 463:253 464:253 465:103 488:10 489:253 490:253 491:253 492:253 493:103 516:10 517:253 518:253 519:253 520:173 521:7 544:10 545:253 546:253 547:253 548:168 572:143 573:253 574:253 575:239 576:49 600:198 601:253 602:253 603:234 615:140 628:198 629:253 630:253 631:234 656:198 657:253 658:253 659:234 +0 235:40 236:37 238:7 239:77 240:137 241:136 242:136 243:136 244:136 245:40 246:6 261:16 262:135 263:254 264:233 266:152 267:215 268:96 269:140 270:155 271:118 272:230 273:254 274:158 275:68 288:19 289:164 290:254 291:114 294:235 295:140 301:99 302:230 303:254 304:186 305:14 315:70 316:226 317:242 318:121 322:104 323:195 324:38 330:33 331:179 332:253 333:140 342:41 343:241 344:198 345:43 359:24 360:209 361:223 370:164 371:250 372:66 388:136 389:253 398:254 399:158 416:136 417:215 426:255 427:76 442:5 443:127 444:246 445:133 454:254 455:122 469:5 470:150 471:247 472:91 473:9 482:254 483:165 495:13 496:79 497:194 498:216 499:84 510:111 511:251 512:87 519:16 520:25 521:40 522:107 523:186 524:213 525:117 526:25 538:14 539:185 540:235 541:142 542:23 546:91 547:157 548:231 549:207 550:126 551:49 569:143 570:195 571:255 572:254 573:254 574:244 575:157 576:76 599:39 600:39 601:39 602:33 +1 128:166 129:255 130:187 131:6 156:165 157:253 158:253 159:13 183:15 184:191 185:253 186:253 187:13 211:49 212:253 213:253 214:253 215:13 239:141 240:253 241:253 242:169 243:4 266:4 267:189 268:253 269:249 270:53 294:69 295:253 296:253 297:246 322:69 323:253 324:253 325:246 350:118 351:253 352:253 353:124 378:206 379:253 380:231 381:21 405:66 406:241 407:253 408:199 433:105 434:253 435:253 436:89 460:3 461:228 462:253 463:252 464:86 488:111 489:253 490:253 491:205 516:166 517:253 518:253 519:75 543:43 544:249 545:253 546:193 547:9 570:4 571:160 572:253 573:253 574:184 598:37 599:253 600:253 601:253 602:88 626:140 627:253 628:253 629:186 630:18 654:14 655:253 656:253 657:27 +1 128:117 129:128 155:2 156:199 157:127 183:81 184:254 185:87 211:116 212:254 213:48 239:175 240:234 241:18 266:5 267:230 268:168 294:80 295:255 296:142 322:80 323:255 324:142 350:80 351:251 352:57 378:129 379:239 406:164 407:209 433:28 434:245 435:159 461:64 462:254 463:144 489:84 490:254 491:80 517:143 518:254 519:30 544:3 545:225 546:200 572:48 573:254 574:174 600:48 601:254 602:174 628:93 629:254 630:129 656:53 657:234 658:41 +1 129:159 130:142 156:11 157:220 158:141 184:78 185:254 186:141 212:111 213:254 214:109 240:196 241:221 242:15 267:26 268:221 269:159 295:63 296:254 297:159 323:178 324:254 325:93 350:7 351:191 352:254 353:97 378:42 379:255 380:254 381:41 406:42 407:254 408:195 409:10 434:141 435:255 436:78 461:11 462:202 463:254 464:59 489:86 490:254 491:254 492:59 517:142 518:254 519:248 520:52 545:142 546:254 547:195 573:142 574:254 575:164 601:142 602:254 603:77 629:142 630:254 631:131 657:77 658:172 659:5 +0 124:66 125:254 126:254 127:58 128:60 129:59 130:59 131:50 151:73 152:233 153:253 154:253 155:148 156:254 157:253 158:253 159:232 160:73 179:156 180:253 181:253 182:253 183:117 184:255 185:253 186:253 187:253 188:223 189:176 190:162 205:37 206:116 207:246 208:253 209:180 210:18 211:4 212:18 213:109 214:241 215:253 216:253 217:253 218:236 219:28 233:235 234:253 235:253 236:245 237:107 242:109 243:170 244:253 245:253 246:253 247:174 261:235 262:253 263:253 264:233 271:15 272:156 273:253 274:253 275:223 276:72 287:10 288:156 289:250 290:253 291:253 292:67 300:99 301:253 302:253 303:253 304:127 305:5 315:118 316:253 317:253 318:253 319:204 320:26 328:68 329:223 330:253 331:253 332:253 333:57 342:32 343:191 344:253 345:253 346:253 347:97 357:156 358:253 359:253 360:253 361:57 370:59 371:253 372:253 373:253 374:253 375:97 385:36 386:224 387:253 388:253 389:57 398:60 399:254 400:255 401:254 402:156 413:37 414:226 415:254 416:254 417:58 426:59 427:253 428:253 429:253 430:154 441:156 442:253 443:253 444:253 445:57 454:59 455:253 456:253 457:253 458:154 469:156 470:253 471:253 472:253 473:57 482:59 483:253 484:253 485:253 486:246 487:90 496:16 497:171 498:253 499:253 500:231 501:49 510:59 511:253 512:253 513:253 514:253 515:156 516:91 524:99 525:253 526:253 527:222 528:71 538:59 539:253 540:253 541:253 542:253 543:253 544:245 545:109 551:145 552:194 553:253 554:253 555:174 566:9 567:38 568:174 569:251 570:253 571:253 572:253 573:241 574:215 575:215 576:217 577:215 578:215 579:250 580:253 581:253 582:221 583:26 597:235 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:253 607:253 608:253 609:204 610:26 625:108 626:116 627:200 628:253 629:253 630:253 631:253 632:254 633:253 634:253 635:253 636:199 637:44 655:36 656:57 657:118 658:253 659:253 660:58 661:57 662:57 663:57 664:35 +1 129:101 130:222 131:84 157:225 158:252 159:84 184:89 185:246 186:208 187:19 212:128 213:252 214:195 239:79 240:253 241:252 242:195 267:141 268:255 269:253 270:133 294:26 295:240 296:253 297:252 298:55 322:60 323:252 324:253 325:154 326:12 349:7 350:178 351:252 352:253 353:27 377:57 378:252 379:252 380:253 381:27 405:57 406:253 407:253 408:204 409:15 433:104 434:252 435:252 436:94 460:19 461:209 462:252 463:252 488:101 489:252 490:252 491:157 516:225 517:252 518:252 519:112 544:226 545:253 546:240 547:63 572:225 573:252 574:223 600:225 601:252 602:223 628:225 629:252 630:242 631:75 656:146 657:252 658:236 659:50 +0 124:41 125:254 126:254 127:157 128:34 129:34 130:218 131:255 132:206 133:34 134:18 151:53 152:238 153:252 154:252 155:252 156:252 157:252 158:252 159:252 160:252 161:252 162:162 163:26 178:66 179:220 180:252 181:252 182:252 183:209 184:153 185:223 186:252 187:252 188:252 189:252 190:252 191:98 206:166 207:252 208:252 209:252 210:252 211:141 213:85 214:230 215:252 216:252 217:252 218:252 219:98 234:166 235:252 236:252 237:252 238:252 239:141 242:73 243:102 244:252 245:252 246:252 247:98 262:166 263:252 264:252 265:252 266:191 267:30 271:5 272:97 273:252 274:252 275:220 276:51 289:123 290:245 291:252 292:252 293:202 294:14 300:56 301:252 302:252 303:252 304:65 316:18 317:154 318:252 319:252 320:241 328:56 329:252 330:252 331:252 332:65 343:21 344:146 345:252 346:252 347:252 348:241 356:56 357:252 358:252 359:252 360:65 371:67 372:252 373:252 374:252 375:252 376:241 384:56 385:252 386:252 387:252 388:65 399:67 400:252 401:252 402:252 403:252 404:116 412:56 413:252 414:252 415:252 416:65 427:67 428:252 429:252 430:252 431:252 432:20 440:56 441:252 442:252 443:252 444:65 455:67 456:252 457:252 458:252 459:87 460:4 468:56 469:252 470:252 471:124 472:11 483:67 484:252 485:252 486:252 487:54 494:19 495:236 496:245 497:252 498:252 499:98 511:67 512:252 513:252 514:252 515:97 516:5 521:39 522:219 523:252 524:252 525:252 526:252 527:98 539:67 540:252 541:252 542:252 543:252 544:102 545:89 546:89 547:89 548:89 549:203 550:252 551:252 552:252 553:252 554:209 555:64 567:67 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:252 576:252 577:252 578:252 579:252 580:226 581:130 582:68 595:67 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:252 604:252 605:252 606:252 607:239 608:77 623:17 624:65 625:163 626:252 627:252 628:252 629:252 630:252 631:252 632:252 633:252 634:96 635:59 653:17 654:176 655:252 656:252 657:252 658:252 659:155 660:32 661:32 662:6 +0 96:56 97:247 98:121 124:24 125:242 126:245 127:122 153:231 154:253 155:253 156:104 157:12 181:90 182:253 183:253 184:254 185:221 186:120 187:120 188:85 206:67 207:75 208:36 209:11 210:56 211:222 212:254 213:253 214:253 215:253 216:245 217:207 218:36 233:86 234:245 235:249 236:105 239:44 240:224 241:230 242:253 243:253 244:253 245:253 246:214 247:10 260:8 261:191 262:253 263:143 269:29 270:119 271:119 272:158 273:253 274:253 275:94 288:15 289:253 290:226 291:48 300:4 301:183 302:253 303:248 304:56 316:42 317:253 318:178 329:179 330:253 331:184 332:14 344:164 345:253 346:178 357:179 358:253 359:163 371:61 372:254 373:254 374:179 384:76 385:254 386:254 387:164 399:60 400:253 401:253 402:178 411:29 412:206 413:253 414:253 415:40 427:60 428:253 429:253 430:178 439:120 440:253 441:253 442:245 443:13 455:60 456:253 457:253 458:178 467:120 468:253 469:239 470:63 483:60 484:253 485:253 486:178 494:14 495:238 496:253 497:179 511:18 512:190 513:253 514:231 515:70 521:43 522:184 523:253 524:253 525:74 540:86 541:253 542:253 543:239 544:134 545:8 548:56 549:163 550:253 551:253 552:213 553:35 568:16 569:253 570:253 571:253 572:253 573:240 574:239 575:239 576:247 577:253 578:253 579:210 580:27 596:4 597:59 598:204 599:253 600:253 601:253 602:253 603:253 604:254 605:253 606:250 607:110 626:31 627:122 628:253 629:253 630:253 631:253 632:255 633:217 634:98 +0 125:19 126:164 127:253 128:255 129:253 130:118 131:59 132:36 153:78 154:251 155:251 156:253 157:251 158:251 159:251 160:199 161:45 180:14 181:198 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:204 190:26 208:5 209:117 210:251 211:251 212:243 213:212 214:239 215:251 216:251 217:251 218:218 236:95 237:251 238:251 239:251 240:120 242:175 243:251 244:251 245:251 246:231 263:97 264:237 265:251 266:251 267:251 270:67 271:240 272:251 273:251 274:243 275:108 290:8 291:163 292:251 293:251 294:240 295:81 299:68 300:251 301:251 302:251 303:179 304:9 317:13 318:145 319:251 320:251 321:226 322:80 327:39 328:251 329:251 330:251 331:251 332:115 345:144 346:251 347:251 348:251 349:173 355:18 356:167 357:251 358:251 359:251 360:115 373:233 374:251 375:251 376:251 377:173 384:98 385:251 386:251 387:251 388:115 400:176 401:253 402:253 403:216 404:179 412:99 413:253 414:253 415:253 416:116 427:55 428:210 429:251 430:251 431:96 440:98 441:251 442:251 443:214 444:62 455:117 456:251 457:251 458:251 459:96 467:28 468:204 469:251 470:237 471:53 482:55 483:241 484:251 485:251 486:160 487:7 494:28 495:222 496:251 497:251 498:231 510:59 511:251 512:251 513:251 514:153 520:23 521:98 522:204 523:251 524:251 525:251 526:156 538:59 539:251 540:251 541:251 542:153 546:85 547:155 548:179 549:251 550:251 551:251 552:251 553:154 554:15 566:59 567:251 568:251 569:251 570:236 571:214 572:214 573:214 574:234 575:251 576:253 577:251 578:251 579:248 580:156 581:15 594:41 595:209 596:251 597:251 598:251 599:251 600:251 601:251 602:251 603:251 604:253 605:251 606:196 607:146 623:54 624:115 625:241 626:251 627:251 628:251 629:251 630:251 631:251 632:253 633:187 634:35 653:83 654:251 655:251 656:251 657:251 658:251 659:101 660:57 661:31 +1 129:232 130:255 131:107 156:58 157:244 158:253 159:106 184:95 185:253 186:253 187:106 212:95 213:253 214:253 215:106 240:95 241:253 242:249 243:69 268:144 269:253 270:192 295:97 296:233 297:253 298:66 323:195 324:253 325:253 326:5 350:38 351:232 352:253 353:182 354:2 377:10 378:160 379:253 380:231 381:53 405:42 406:253 407:253 408:158 433:141 434:253 435:253 436:115 460:75 461:245 462:253 463:183 464:4 487:1 488:147 489:253 490:251 491:58 515:20 516:253 517:253 518:180 543:202 544:253 545:226 546:27 571:243 572:253 573:212 598:85 599:251 600:253 601:173 626:209 627:253 628:244 629:57 654:169 655:253 656:174 +1 127:63 128:128 129:2 155:63 156:254 157:123 183:63 184:254 185:179 211:63 212:254 213:179 239:63 240:254 241:179 267:142 268:254 269:179 295:187 296:254 297:158 323:187 324:254 325:55 350:68 351:235 352:254 353:55 378:181 379:254 380:254 381:55 406:181 407:254 408:202 409:14 434:181 435:254 436:186 462:181 463:254 464:146 490:181 491:254 492:62 518:181 519:254 520:62 546:181 547:254 548:62 574:181 575:255 576:62 602:181 603:254 604:241 605:52 630:181 631:254 632:222 633:30 658:181 659:224 660:34 +1 130:131 131:255 132:184 133:15 157:99 158:247 159:253 160:182 161:15 185:124 186:253 187:253 188:253 189:38 212:9 213:171 214:253 215:253 216:140 217:1 240:47 241:253 242:253 243:251 244:117 267:43 268:219 269:253 270:253 271:153 295:78 296:253 297:253 298:253 299:84 323:97 324:253 325:253 326:244 327:74 350:69 351:243 352:253 353:253 354:183 377:10 378:168 379:253 380:253 381:215 382:34 405:31 406:253 407:253 408:253 409:129 433:107 434:253 435:253 436:242 437:67 460:24 461:204 462:253 463:253 464:187 488:95 489:253 490:253 491:201 492:25 516:239 517:253 518:253 519:176 543:119 544:251 545:253 546:253 547:138 570:30 571:212 572:253 573:252 574:165 575:8 598:193 599:253 600:253 601:222 626:193 627:253 628:253 629:189 654:193 655:253 656:201 657:27 +0 125:57 126:255 127:253 128:198 129:85 153:168 154:253 155:251 156:253 157:251 158:169 159:56 180:86 181:253 182:254 183:253 184:254 185:253 186:254 187:253 188:57 208:197 209:251 210:253 211:251 212:253 213:251 214:253 215:251 216:225 217:56 235:169 236:255 237:253 238:226 239:56 241:114 242:254 243:253 244:254 245:84 262:57 263:224 264:253 265:251 266:56 270:139 271:251 272:253 273:83 290:141 291:253 292:255 293:84 298:57 299:225 300:254 301:196 318:253 319:251 320:253 321:83 327:168 328:253 329:83 345:169 346:254 347:253 348:169 355:169 356:254 357:253 358:169 373:168 374:253 375:251 376:56 383:168 384:253 385:251 386:56 401:169 402:254 403:84 412:254 413:253 429:168 430:253 431:83 440:253 441:251 456:29 457:197 458:254 459:84 467:169 468:254 469:196 484:85 485:251 486:253 487:83 494:57 495:224 496:253 497:83 512:57 513:225 514:254 515:139 521:57 522:141 523:253 524:254 525:84 541:168 542:253 543:251 544:169 545:56 547:114 548:169 549:224 550:253 551:251 552:253 553:83 569:169 570:254 571:253 572:254 573:253 574:254 575:253 576:254 577:253 578:254 579:253 580:226 581:56 597:56 598:253 599:251 600:253 601:251 602:253 603:251 604:253 605:251 606:253 607:251 608:56 626:169 627:225 628:254 629:253 630:254 631:253 632:254 633:253 634:226 635:56 655:56 656:253 657:251 658:253 659:251 660:84 661:83 662:56 +0 127:12 128:105 129:224 130:255 131:247 132:22 155:131 156:254 157:254 158:243 159:252 160:76 182:131 183:225 184:254 185:224 186:48 187:136 208:13 209:109 210:252 211:254 212:254 213:254 214:197 215:76 235:9 236:181 237:254 238:254 239:240 240:229 241:237 242:254 243:252 244:152 245:21 262:9 263:143 264:254 265:254 266:226 267:36 269:22 270:138 271:254 272:254 273:188 289:13 290:181 291:254 292:254 293:250 294:64 298:2 299:53 300:236 301:252 302:131 317:102 318:254 319:254 320:254 321:111 328:56 329:243 330:251 331:42 344:30 345:186 346:254 347:254 348:206 349:29 357:199 358:254 359:91 372:92 373:254 374:254 375:237 376:13 385:134 386:254 387:91 400:133 401:254 402:254 403:126 413:134 414:250 415:17 428:187 429:254 430:237 431:23 441:200 442:183 456:187 457:254 458:213 467:2 468:134 469:252 470:101 484:183 485:254 486:133 495:14 496:254 497:234 498:34 512:92 513:254 514:161 522:84 523:204 524:254 525:56 540:92 541:254 542:229 549:85 550:252 551:252 552:188 553:11 568:56 569:252 570:229 575:3 576:53 577:235 578:253 579:166 597:224 598:245 599:130 600:68 601:68 602:134 603:214 604:254 605:254 606:159 625:141 626:254 627:254 628:254 629:254 630:254 631:254 632:233 633:95 634:3 653:14 654:152 655:254 656:254 657:254 658:186 659:157 660:53 +1 130:226 131:247 132:55 157:99 158:248 159:254 160:230 161:30 185:125 186:254 187:254 188:254 189:38 213:125 214:254 215:254 216:212 217:24 240:18 241:223 242:254 243:252 244:118 268:24 269:254 270:254 271:239 295:27 296:195 297:254 298:254 299:93 323:78 324:254 325:254 326:246 327:74 351:158 352:254 353:254 354:185 378:41 379:239 380:254 381:254 382:43 405:22 406:218 407:254 408:254 409:167 410:9 433:32 434:254 435:254 436:254 437:130 460:24 461:187 462:254 463:254 464:234 465:16 488:189 489:254 490:254 491:254 492:128 515:64 516:247 517:254 518:255 519:219 520:42 543:139 544:254 545:254 546:222 547:40 570:30 571:213 572:254 573:235 574:45 598:194 599:254 600:254 601:223 626:194 627:254 628:254 629:190 654:194 655:254 656:202 657:27 +1 130:166 131:253 132:124 133:53 158:140 159:251 160:251 161:180 185:125 186:246 187:251 188:251 189:51 212:32 213:190 214:251 215:251 216:251 217:103 240:21 241:174 242:251 243:251 244:251 268:73 269:176 270:253 271:253 272:201 296:149 297:251 298:251 299:251 300:71 323:27 324:228 325:251 326:251 327:157 328:10 351:180 352:253 353:251 354:251 355:142 377:27 378:180 379:231 380:253 381:251 382:96 383:41 405:89 406:253 407:253 408:255 409:211 410:25 433:217 434:251 435:251 436:253 437:107 460:21 461:221 462:251 463:251 464:242 465:92 487:32 488:190 489:251 490:251 491:251 492:103 515:202 516:251 517:251 518:251 519:122 542:53 543:255 544:253 545:253 546:221 547:51 570:180 571:253 572:251 573:251 574:142 598:180 599:253 600:251 601:251 602:142 626:180 627:253 628:251 629:157 630:82 654:180 655:253 656:147 657:10 +1 129:17 130:206 131:229 132:44 157:2 158:125 159:254 160:123 185:95 186:254 187:254 188:123 212:78 213:240 214:254 215:254 216:123 240:100 241:254 242:254 243:254 244:123 267:2 268:129 269:254 270:254 271:220 272:20 295:9 296:254 297:254 298:254 299:123 322:22 323:179 324:254 325:254 326:254 327:49 350:83 351:254 352:254 353:254 354:183 355:19 378:136 379:254 380:254 381:254 382:139 404:3 405:111 406:252 407:254 408:254 409:232 410:45 432:67 433:254 434:254 435:254 436:216 437:40 459:14 460:192 461:254 462:254 463:254 464:140 486:23 487:192 488:254 489:254 490:254 491:246 514:77 515:254 516:254 517:255 518:241 519:100 541:65 542:235 543:254 544:254 545:254 546:172 568:30 569:238 570:254 571:254 572:254 573:219 574:26 596:34 597:254 598:254 599:254 600:216 601:41 624:34 625:254 626:254 627:254 628:188 652:12 653:170 654:254 655:254 656:82 +1 130:218 131:253 132:124 157:84 158:236 159:251 160:251 184:63 185:236 186:251 187:251 188:122 212:73 213:251 214:251 215:251 216:173 240:202 241:251 242:251 243:251 244:71 267:53 268:255 269:253 270:253 271:253 272:72 295:180 296:253 297:251 298:251 299:188 300:30 323:180 324:253 325:251 326:251 327:142 350:47 351:211 352:253 353:251 354:235 355:82 377:27 378:211 379:251 380:253 381:251 382:215 405:89 406:253 407:253 408:255 409:253 410:164 433:217 434:251 435:251 436:253 437:168 438:15 460:21 461:221 462:251 463:251 464:253 465:107 487:32 488:190 489:251 490:251 491:251 492:221 493:61 515:73 516:251 517:251 518:251 519:251 520:180 543:255 544:253 545:253 546:253 547:201 570:105 571:253 572:251 573:251 574:251 575:71 598:180 599:253 600:251 601:246 602:137 603:10 626:180 627:253 628:251 629:215 654:180 655:253 656:251 657:86 +1 124:102 125:180 126:1 152:140 153:254 154:130 180:140 181:254 182:204 208:140 209:254 210:204 236:72 237:254 238:204 264:25 265:231 266:250 267:135 292:11 293:211 294:254 295:222 321:101 322:254 323:250 324:15 349:96 350:254 351:254 352:95 377:2 378:251 379:254 380:95 405:2 406:251 407:254 408:95 433:96 434:254 435:254 436:95 461:53 462:253 463:254 464:139 490:250 491:254 492:235 493:27 518:201 519:254 520:254 521:128 546:80 547:254 548:254 549:139 574:65 575:254 576:254 577:139 602:150 603:254 604:254 605:139 630:229 631:254 632:254 633:43 658:52 659:196 660:168 661:9 +0 128:87 129:208 130:249 155:27 156:212 157:254 158:195 182:118 183:225 184:254 185:254 186:232 187:147 188:46 209:115 210:248 211:254 212:254 213:254 214:254 215:254 216:230 217:148 218:12 236:18 237:250 238:254 239:245 240:226 241:254 242:254 243:254 244:254 245:254 246:148 263:92 264:205 265:254 266:250 267:101 268:20 269:194 270:254 271:254 272:254 273:254 274:229 275:53 291:152 292:254 293:254 294:94 297:14 298:124 299:187 300:254 301:254 302:254 303:213 318:95 319:252 320:254 321:206 322:15 327:3 328:6 329:51 330:231 331:254 332:94 345:50 346:246 347:254 348:254 349:20 358:200 359:254 360:96 372:21 373:184 374:254 375:254 376:147 377:2 386:200 387:254 388:96 400:177 401:254 402:254 403:218 404:33 413:16 414:211 415:254 416:96 427:11 428:219 429:254 430:251 431:92 441:84 442:254 443:232 444:44 455:101 456:254 457:254 458:141 469:162 470:254 471:231 472:42 483:235 484:254 485:227 486:42 496:51 497:238 498:254 499:213 511:235 512:254 513:199 524:160 525:254 526:229 527:52 539:235 540:254 541:199 549:10 550:84 551:150 552:253 553:254 554:147 567:235 568:254 569:213 570:20 575:17 576:63 577:158 578:254 579:254 580:254 581:155 582:12 595:122 596:248 597:254 598:204 599:98 600:42 601:177 602:180 603:200 604:254 605:254 606:253 607:213 608:82 609:10 624:203 625:254 626:254 627:254 628:254 629:254 630:254 631:254 632:251 633:219 634:94 652:35 653:221 654:254 655:254 656:254 657:254 658:254 659:217 660:95 +1 126:134 127:230 154:133 155:231 156:10 182:133 183:253 184:96 210:133 211:253 212:96 238:133 239:253 240:183 266:133 267:253 268:217 294:133 295:253 296:217 322:133 323:253 324:217 350:133 351:253 352:217 378:133 379:253 380:217 406:134 407:254 408:218 434:133 435:253 436:159 462:133 463:253 464:199 490:156 491:253 492:96 518:254 519:247 520:73 546:254 547:248 548:74 573:99 574:254 575:245 576:64 600:89 601:230 602:254 603:125 627:140 628:251 629:253 630:243 631:10 655:114 656:242 657:195 658:69 +1 125:29 126:85 127:255 128:139 153:197 154:251 155:253 156:251 181:254 182:253 183:254 184:253 209:253 210:251 211:253 212:251 237:254 238:253 239:254 240:253 265:253 266:251 267:253 268:138 293:254 294:253 295:254 296:196 321:253 322:251 323:253 324:196 349:254 350:253 351:254 352:84 377:253 378:251 379:253 380:196 405:254 406:253 407:254 408:253 433:253 434:251 435:253 436:251 461:254 462:253 463:254 464:253 489:253 490:251 491:253 492:251 517:254 518:253 519:254 520:253 545:253 546:251 547:253 548:251 573:254 574:253 575:254 576:253 601:253 602:251 603:253 604:251 629:57 630:225 631:254 632:253 658:56 659:253 660:251 +1 125:149 126:255 127:254 128:58 153:215 154:253 155:183 156:2 180:41 181:232 182:253 183:181 208:92 209:253 210:253 211:181 236:92 237:253 238:253 239:181 264:92 265:253 266:253 267:181 292:92 293:253 294:253 295:181 320:92 321:253 322:253 323:181 348:92 349:253 350:253 351:181 376:92 377:253 378:253 379:181 404:92 405:253 406:253 407:181 432:92 433:253 434:253 435:181 460:92 461:253 462:253 463:181 488:31 489:228 490:253 491:181 517:198 518:253 519:228 520:54 545:33 546:226 547:253 548:195 549:7 574:199 575:253 576:253 577:75 602:34 603:218 604:253 605:228 606:117 607:14 608:12 631:33 632:219 633:253 634:253 635:253 636:211 660:32 661:123 662:149 663:230 664:41 +1 130:79 131:203 132:141 157:51 158:240 159:240 160:140 185:88 186:252 187:252 188:140 213:197 214:252 215:252 216:140 241:197 242:252 243:252 244:140 268:147 269:253 270:253 271:253 295:38 296:234 297:252 298:242 299:89 323:113 324:252 325:252 326:223 350:16 351:207 352:252 353:252 354:129 377:16 378:203 379:253 380:252 381:220 382:37 405:29 406:253 407:255 408:253 409:56 432:19 433:181 434:252 435:253 436:176 437:6 460:166 461:252 462:252 463:228 464:52 487:10 488:203 489:252 490:252 491:126 514:63 515:178 516:252 517:252 518:173 542:114 543:253 544:253 545:225 570:238 571:252 572:252 573:99 596:7 597:135 598:253 599:252 600:176 601:19 624:29 625:252 626:253 627:252 628:55 652:13 653:189 654:253 655:204 656:25 +1 126:94 127:254 128:75 154:166 155:253 156:231 182:208 183:253 184:147 210:208 211:253 212:116 238:208 239:253 240:168 266:146 267:254 268:222 294:166 295:253 296:116 322:208 323:253 324:116 350:166 351:253 352:158 378:145 379:253 380:231 406:209 407:254 408:169 434:187 435:253 436:168 462:93 463:253 464:116 490:93 491:253 492:116 518:93 519:253 520:116 546:94 547:254 548:179 549:11 574:93 575:253 576:246 577:101 602:145 603:253 604:255 605:92 630:93 631:253 632:246 633:59 658:93 659:253 660:74 +0 127:46 128:105 129:254 130:254 131:224 132:59 133:59 134:9 155:196 156:254 157:253 158:253 159:253 160:253 161:253 162:128 182:96 183:235 184:254 185:253 186:253 187:253 188:253 189:253 190:247 191:122 208:4 209:101 210:244 211:253 212:254 213:234 214:241 215:253 216:253 217:253 218:253 219:186 220:18 236:96 237:253 238:253 239:253 240:232 241:83 242:109 243:170 244:253 245:253 246:253 247:253 248:116 264:215 265:253 266:253 267:253 268:196 271:40 272:253 273:253 274:253 275:253 276:116 290:8 291:141 292:247 293:253 294:253 295:237 296:29 299:6 300:38 301:171 302:253 303:253 304:116 317:13 318:146 319:253 320:253 321:253 322:253 323:57 329:156 330:253 331:253 332:116 345:40 346:253 347:253 348:253 349:253 350:178 351:27 357:156 358:253 359:253 360:116 372:136 373:204 374:253 375:253 376:253 377:192 378:27 385:156 386:253 387:253 388:116 399:28 400:195 401:254 402:254 403:254 404:250 405:135 412:99 413:255 414:254 415:254 416:117 427:118 428:253 429:253 430:253 431:253 432:142 439:19 440:170 441:253 442:253 443:216 444:62 454:42 455:212 456:253 457:253 458:253 459:253 460:38 466:124 467:188 468:253 469:253 470:253 471:174 482:59 483:253 484:253 485:253 486:237 487:93 488:3 491:31 492:40 493:130 494:247 495:253 496:253 497:253 498:204 499:13 510:59 511:253 512:253 513:253 514:154 518:54 519:218 520:254 521:253 522:253 523:253 524:253 525:253 526:38 538:59 539:253 540:253 541:253 542:215 543:156 544:156 545:156 546:209 547:253 548:255 549:253 550:253 551:253 552:192 553:97 554:15 566:55 567:242 568:253 569:253 570:253 571:253 572:253 573:253 574:253 575:253 576:254 577:253 578:253 579:204 580:23 595:118 596:253 597:253 598:253 599:253 600:253 601:253 602:253 603:253 604:254 605:216 606:174 607:13 623:54 624:116 625:243 626:253 627:253 628:253 629:253 630:253 631:146 632:117 633:62 653:53 654:132 655:253 656:253 657:192 658:57 659:13 +1 125:42 126:232 127:254 128:58 153:86 154:253 155:253 156:58 181:86 182:253 183:253 184:58 209:206 210:253 211:253 212:58 237:215 238:253 239:253 240:58 265:215 266:253 267:253 268:58 293:215 294:253 295:253 296:58 321:215 322:253 323:253 324:58 349:215 350:253 351:253 352:58 377:215 378:253 379:253 380:58 405:215 406:253 407:253 408:58 433:188 434:253 435:253 436:85 461:86 462:253 463:253 464:200 465:12 489:29 490:223 491:253 492:253 493:151 518:209 519:253 520:253 521:194 546:128 547:253 548:253 549:200 550:8 574:32 575:213 576:253 577:253 578:152 579:6 603:32 604:221 605:253 606:253 607:153 608:5 632:90 633:215 634:253 635:253 636:151 661:59 662:253 663:253 664:84 +1 156:60 157:229 158:38 184:187 185:254 186:78 211:121 212:252 213:254 214:78 239:197 240:254 241:206 242:6 267:197 268:254 269:202 294:27 295:218 296:233 297:62 322:117 323:254 324:195 350:203 351:254 352:195 377:64 378:244 379:254 380:195 405:79 406:254 407:255 408:161 433:79 434:254 435:254 436:65 461:79 462:254 463:241 464:52 489:79 490:254 491:189 517:79 518:254 519:189 545:79 546:254 547:189 573:79 574:254 575:189 601:79 602:254 603:194 604:5 629:35 630:219 631:254 632:72 658:34 659:223 660:195 687:129 688:195 +1 101:11 102:150 103:72 129:37 130:251 131:71 157:63 158:251 159:71 185:217 186:251 187:71 213:217 214:251 215:71 240:145 241:253 242:253 243:72 267:42 268:206 269:251 270:251 271:71 295:99 296:251 297:251 298:251 299:71 323:253 324:251 325:251 326:251 327:71 350:130 351:253 352:251 353:251 354:251 355:71 377:110 378:253 379:255 380:253 381:253 382:253 383:72 405:109 406:251 407:253 408:251 409:251 410:188 411:30 433:109 434:251 435:253 436:251 437:246 438:123 460:16 461:170 462:251 463:253 464:251 465:215 488:37 489:251 490:251 491:253 492:251 493:86 516:218 517:253 518:253 519:255 520:253 521:35 543:84 544:236 545:251 546:251 547:253 548:168 549:15 571:144 572:251 573:251 574:251 575:190 576:15 599:144 600:251 601:251 602:251 603:180 626:53 627:221 628:251 629:251 630:251 631:180 +0 127:45 128:254 129:254 130:254 131:148 132:24 133:9 154:43 155:254 156:252 157:252 158:252 159:252 160:252 161:121 162:13 181:58 182:237 183:254 184:252 185:252 186:252 187:252 188:252 189:252 190:68 208:69 209:224 210:252 211:254 212:252 213:252 214:252 215:252 216:252 217:252 218:135 219:17 235:75 236:216 237:252 238:252 239:254 240:231 241:168 242:252 243:252 244:252 245:252 246:252 247:45 262:77 263:212 264:252 265:252 266:252 267:242 268:93 269:32 270:114 271:177 272:252 273:252 274:252 275:158 276:12 289:75 290:212 291:252 292:252 293:252 294:252 295:231 299:116 300:252 301:252 302:252 303:252 304:21 316:69 317:216 318:252 319:252 320:252 321:252 322:252 323:62 327:116 328:252 329:252 330:252 331:252 332:21 344:93 345:252 346:252 347:252 348:252 349:252 350:62 355:21 356:158 357:252 358:252 359:252 360:21 371:64 372:239 373:252 374:252 375:252 376:252 377:252 378:21 384:139 385:252 386:252 387:252 388:21 398:5 399:87 400:254 401:254 402:254 403:254 404:237 405:41 411:11 412:150 413:254 414:254 415:254 416:22 425:5 426:85 427:252 428:252 429:252 430:252 431:222 432:55 439:116 440:252 441:252 442:252 443:214 444:18 453:24 454:252 455:252 456:252 457:252 458:252 459:91 466:26 467:153 468:252 469:252 470:252 471:45 481:24 482:252 483:252 484:252 485:252 486:252 487:91 492:18 493:93 494:151 495:252 496:252 497:252 498:184 499:28 509:24 510:252 511:252 512:252 513:252 514:252 515:164 516:116 517:116 518:116 519:117 520:141 521:252 522:252 523:252 524:252 525:252 526:68 537:24 538:252 539:252 540:252 541:252 542:252 543:252 544:252 545:252 546:252 547:254 548:252 549:252 550:252 551:252 552:252 553:163 554:31 565:9 566:121 567:252 568:252 569:252 570:252 571:252 572:252 573:252 574:252 575:254 576:252 577:252 578:252 579:178 580:91 581:33 594:13 595:119 596:252 597:252 598:252 599:252 600:252 601:252 602:252 603:254 604:252 605:252 606:184 607:37 623:13 624:121 625:252 626:252 627:252 628:252 629:252 630:252 631:254 632:214 633:45 634:28 652:8 653:21 654:21 655:169 656:252 657:252 658:41 659:22 660:18 +0 125:218 126:253 127:253 128:255 129:149 130:62 151:42 152:144 153:236 154:251 155:251 156:253 157:251 158:236 159:144 160:144 179:99 180:251 181:251 182:251 183:225 184:253 185:251 186:251 187:251 188:251 189:166 190:16 206:79 207:253 208:251 209:251 210:204 211:41 212:143 213:205 214:251 215:251 216:251 217:253 218:169 219:15 233:79 234:231 235:253 236:251 237:225 238:41 241:41 242:226 243:251 244:251 245:253 246:251 247:164 260:37 261:253 262:253 263:255 264:253 265:35 271:79 272:232 273:255 274:253 275:227 276:42 288:140 289:251 290:251 291:253 292:168 293:15 300:77 301:253 302:251 303:251 304:142 315:21 316:221 317:251 318:251 319:164 320:15 329:227 330:251 331:251 332:236 333:61 342:32 343:190 344:251 345:251 346:251 357:73 358:251 359:251 360:251 361:71 370:73 371:251 372:251 373:251 374:251 385:73 386:251 387:251 388:251 389:71 398:73 399:253 400:253 401:253 402:201 413:73 414:253 415:253 416:253 417:72 426:176 427:251 428:251 429:251 430:71 441:73 442:251 443:251 444:251 445:71 454:253 455:251 456:251 457:157 458:10 469:73 470:251 471:251 472:251 473:71 482:253 483:251 484:251 485:142 497:150 498:251 499:251 500:204 501:41 510:124 511:251 512:251 513:220 514:180 524:130 525:253 526:251 527:225 528:41 538:73 539:253 540:253 541:253 542:253 543:73 544:73 545:10 549:42 550:73 551:150 552:253 553:255 554:253 555:216 566:31 567:189 568:251 569:251 570:251 571:253 572:251 573:159 574:144 575:144 576:145 577:206 578:251 579:251 580:251 581:253 582:168 583:92 595:20 596:195 597:251 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:251 608:225 609:164 610:15 624:21 625:142 626:220 627:253 628:251 629:251 630:251 631:251 632:253 633:251 634:251 635:204 636:41 654:51 655:72 656:174 657:251 658:251 659:251 660:253 661:147 662:71 663:41 +0 127:60 128:96 129:96 130:48 153:16 154:171 155:228 156:253 157:251 158:220 159:51 160:32 181:127 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:80 207:24 208:182 209:236 210:251 211:211 212:189 213:236 214:251 215:251 216:251 217:242 218:193 234:100 235:194 236:251 237:251 238:211 239:35 241:71 242:173 243:251 244:251 245:253 246:240 247:158 248:19 261:64 262:253 263:255 264:253 265:205 266:19 271:40 272:218 273:255 274:253 275:253 276:91 288:16 289:186 290:251 291:253 292:247 293:110 300:39 301:233 302:251 303:251 304:188 315:16 316:189 317:251 318:251 319:205 320:110 329:48 330:220 331:251 332:220 333:48 343:72 344:251 345:251 346:251 347:158 358:51 359:251 360:251 361:232 371:190 372:251 373:251 374:251 375:59 386:32 387:251 388:251 389:251 398:96 399:253 400:253 401:253 402:95 414:32 415:253 416:253 417:193 426:214 427:251 428:251 429:204 430:23 442:52 443:251 444:251 445:94 454:253 455:251 456:251 457:109 469:48 470:221 471:251 472:219 473:47 482:253 483:251 484:251 485:70 497:234 498:251 499:251 500:188 510:253 511:251 512:251 513:188 523:40 524:158 525:253 526:251 527:172 528:70 539:191 540:253 541:253 542:253 543:96 544:24 549:12 550:174 551:253 552:253 553:255 554:221 567:71 568:251 569:251 570:251 571:253 572:205 573:190 574:190 575:190 576:191 577:197 578:251 579:251 580:231 581:221 582:93 595:16 596:126 597:236 598:251 599:253 600:251 601:251 602:251 603:251 604:253 605:251 606:251 607:140 608:47 625:67 626:188 627:189 628:188 629:188 630:188 631:188 632:189 633:188 634:109 635:4 +0 126:32 127:202 128:255 129:253 130:253 131:175 132:21 152:84 153:144 154:190 155:251 156:253 157:251 158:251 159:251 160:174 176:6 177:37 178:166 179:218 180:236 181:251 182:251 183:251 184:253 185:251 186:251 187:251 188:251 189:156 204:115 205:251 206:251 207:253 208:251 209:251 210:251 211:251 212:253 213:251 214:251 215:251 216:251 217:180 231:105 232:241 233:251 234:251 235:253 236:251 237:251 238:251 239:122 240:72 241:71 242:71 243:148 244:251 245:180 258:73 259:253 260:253 261:253 262:253 263:202 264:253 265:253 266:143 286:31 287:189 288:251 289:251 290:251 291:31 292:189 293:251 294:142 314:63 315:236 316:251 317:251 318:96 320:124 321:246 322:142 330:21 331:166 332:21 342:73 343:251 344:251 345:251 346:71 349:217 350:142 357:32 358:190 359:251 360:142 370:73 371:251 372:251 373:251 374:71 377:217 378:142 385:73 386:251 387:251 388:142 398:73 399:253 400:253 401:253 402:72 405:156 406:103 413:73 414:253 415:253 416:253 417:72 426:73 427:251 428:251 429:251 430:174 441:73 442:251 443:251 444:251 445:71 454:73 455:251 456:251 457:251 458:251 469:73 470:251 471:251 472:251 473:71 482:42 483:205 484:251 485:251 486:251 487:79 497:73 498:251 499:251 500:251 501:71 511:41 512:226 513:251 514:251 515:232 516:77 525:73 526:251 527:251 528:251 529:71 540:166 541:253 542:253 543:255 544:253 545:227 546:73 547:21 553:125 554:253 555:253 556:143 568:16 569:169 570:251 571:253 572:251 573:251 574:251 575:174 576:105 579:63 580:144 581:253 582:251 583:251 584:142 597:15 598:35 599:253 600:251 601:251 602:251 603:251 604:243 605:217 606:217 607:231 608:251 609:253 610:251 611:220 612:20 627:143 628:142 629:236 630:251 631:251 632:253 633:251 634:251 635:251 636:251 637:253 638:251 639:137 657:61 658:71 659:200 660:253 661:251 662:251 663:251 664:251 665:201 666:71 667:10 +1 130:218 131:170 132:108 157:32 158:227 159:252 160:232 185:129 186:252 187:252 188:252 212:1 213:253 214:252 215:252 216:168 240:144 241:253 242:252 243:236 244:62 268:144 269:253 270:252 271:215 296:144 297:253 298:252 299:112 323:21 324:206 325:253 326:252 327:71 351:99 352:253 353:255 354:119 378:63 379:242 380:252 381:253 382:35 406:94 407:252 408:252 409:154 410:10 433:145 434:237 435:252 436:252 461:255 462:253 463:253 464:108 487:11 488:155 489:253 490:252 491:179 492:15 514:11 515:150 516:252 517:253 518:200 519:20 542:73 543:252 544:252 545:253 546:97 569:47 570:233 571:253 572:253 596:1 597:149 598:252 599:252 600:252 624:1 625:252 626:252 627:246 628:132 652:1 653:169 654:252 655:132 +1 130:116 131:255 132:123 157:29 158:213 159:253 160:122 185:189 186:253 187:253 188:122 213:189 214:253 215:253 216:122 241:189 242:253 243:253 244:122 267:2 268:114 269:243 270:253 271:186 272:19 295:100 296:253 297:253 298:253 299:48 323:172 324:253 325:253 326:253 327:48 351:172 352:253 353:253 354:182 355:19 378:133 379:251 380:253 381:175 382:4 405:107 406:251 407:253 408:253 409:65 432:26 433:194 434:253 435:253 436:214 437:40 459:105 460:205 461:253 462:253 463:125 464:40 487:139 488:253 489:253 490:253 491:81 514:41 515:231 516:253 517:253 518:159 519:16 541:65 542:155 543:253 544:253 545:172 546:4 569:124 570:253 571:253 572:253 573:98 597:124 598:253 599:253 600:214 601:41 624:22 625:207 626:253 627:253 628:139 653:124 654:253 655:162 656:9 From 745e496c59cfece2fcd6120ecc366dcab07b293a Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 22 Apr 2014 14:27:49 -0700 Subject: [PATCH 337/397] [Fix #204] Eliminate delay between binding and log checking **Bug**: In the existing history server, there is a `spark.history.updateInterval` seconds delay before application logs show up on the UI. **Cause**: This is because the following events happen in this order: (1) The background thread that checks for logs starts, but realizes the server has not yet bound and so waits for N seconds, (2) server binds, (3) N seconds later the background thread finds that the server has finally bound to a port, and so finally checks for application logs. **Fix**: This PR forces the log checking thread to start immediately after binding. It also documents two relevant environment variables that are currently missing. Author: Andrew Or Closes #441 from andrewor14/history-server-fix and squashes the following commits: b2eb46e [Andrew Or] Document SPARK_PUBLIC_DNS and SPARK_HISTORY_OPTS for the history server e8d1fbc [Andrew Or] Eliminate delay between binding and checking for logs --- .../spark/deploy/history/HistoryServer.scala | 5 +++++ docs/monitoring.md | 19 +++++++++++++++---- 2 files changed, 20 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index cf64700f9098c..b8f56234d37ec 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -98,6 +98,11 @@ class HistoryServer( def initialize() { attachPage(new HistoryPage(this)) attachHandler(createStaticHandler(STATIC_RESOURCE_DIR, "/static")) + } + + /** Bind to the HTTP server behind this web interface. */ + override def bind() { + super.bind() logCheckingThread.start() } diff --git a/docs/monitoring.md b/docs/monitoring.md index 4c91c3a5929bf..144be3daf1208 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -39,22 +39,33 @@ You can start a the history server by executing: The base logging directory must be supplied, and should contain sub-directories that each represents an application's event logs. This creates a web interface at -`http://:18080` by default. The history server depends on the following variables: +`http://:18080` by default. The history server can be configured as follows: - + + + + + + + + +
      Environment VariableMeaning
      SPARK_DAEMON_MEMORYMemory to allocate to the history server. (default: 512m).Memory to allocate to the history server (default: 512m).
      SPARK_DAEMON_JAVA_OPTS JVM options for the history server (default: none).
      SPARK_PUBLIC_DNS + The public address for the history server. If this is not set, links to application history + may use the internal address of the server, resulting in broken links (default: none). +
      SPARK_HISTORY_OPTS + spark.history.* configuration options for the history server (default: none). +
      -Further, the history server can be configured as follows: - From 0ea0b1a2d66044af97ab84bf3014ff8dfe91fee4 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Tue, 22 Apr 2014 14:28:41 -0700 Subject: [PATCH 338/397] Fix compilation on Hadoop 2.4.x. Author: Marcelo Vanzin Closes #483 from vanzin/yarn-2.4 and squashes the following commits: 0fc57d8 [Marcelo Vanzin] Fix compilation on Hadoop 2.4.x. --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index c00b63669ca8e..d264ecbeaf7d8 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -458,7 +458,8 @@ object ClientBase { } /** Add entry to the classpath. */ - def addClasspathEntry(path: String) = Apps.addToEnvironment(env, Environment.CLASSPATH.name, path) + def addClasspathEntry(path: String) = YarnSparkHadoopUtil.addToEnvironment(env, + Environment.CLASSPATH.name, path, File.pathSeparator) /** Add entry to the classpath. Interpreted as a path relative to the working directory. */ def addPwdClasspathEntry(entry: String) = addClasspathEntry(Environment.PWD.$() + Path.SEPARATOR + entry) From ea8cea82a02099bb66f1e77b757e4d96cc31d6e2 Mon Sep 17 00:00:00 2001 From: Kan Zhang Date: Tue, 22 Apr 2014 15:05:12 -0700 Subject: [PATCH 339/397] [SPARK-1570] Fix classloading in JavaSQLContext.applySchema I think I hit a class loading issue when running JavaSparkSQL example using spark-submit in local mode. Author: Kan Zhang Closes #484 from kanzhang/SPARK-1570 and squashes the following commits: feaaeba [Kan Zhang] [SPARK-1570] Fix classloading in JavaSQLContext.applySchema --- .../org/apache/spark/sql/api/java/JavaSQLContext.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala index 26922f7f336e2..a7347088794a8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/api/java/JavaSQLContext.scala @@ -28,6 +28,7 @@ import org.apache.spark.sql.catalyst.expressions.{AttributeReference, GenericRow import org.apache.spark.sql.catalyst.types._ import org.apache.spark.sql.parquet.ParquetRelation import org.apache.spark.sql.execution.{ExistingRdd, SparkLogicalPlan} +import org.apache.spark.util.Utils /** * The entry point for executing Spark SQL queries from a Java program. @@ -84,10 +85,11 @@ class JavaSQLContext(sparkContext: JavaSparkContext) { */ def applySchema(rdd: JavaRDD[_], beanClass: Class[_]): JavaSchemaRDD = { val schema = getSchema(beanClass) - val className = beanClass.getCanonicalName + val className = beanClass.getName val rowRdd = rdd.rdd.mapPartitions { iter => // BeanInfo is not serializable so we must rediscover it remotely for each partition. - val localBeanInfo = Introspector.getBeanInfo(Class.forName(className)) + val localBeanInfo = Introspector.getBeanInfo( + Class.forName(className, true, Utils.getContextOrSparkClassLoader)) val extractors = localBeanInfo.getPropertyDescriptors.filterNot(_.getName == "class").map(_.getReadMethod) From 995fdc96bcd2c540804401eaab009a777d7d7aa9 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Tue, 22 Apr 2014 19:22:06 -0700 Subject: [PATCH 340/397] Assorted clean-up for Spark-on-YARN. In particular when the HADOOP_CONF_DIR is not not specified. Author: Patrick Wendell Closes #488 from pwendell/hadoop-cleanup and squashes the following commits: fe95f13 [Patrick Wendell] Changes based on Andrew's feeback 18d09c1 [Patrick Wendell] Review comments from Andrew 17929cc [Patrick Wendell] Assorted clean-up for Spark-on-YARN. --- conf/spark-env.sh.template | 2 ++ .../org/apache/spark/deploy/SparkSubmitArguments.scala | 9 +++++++++ docs/hadoop-third-party-distributions.md | 9 ++------- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 8 +++++--- 4 files changed, 18 insertions(+), 10 deletions(-) diff --git a/conf/spark-env.sh.template b/conf/spark-env.sh.template index 177a21cc0377f..f906be611a931 100755 --- a/conf/spark-env.sh.template +++ b/conf/spark-env.sh.template @@ -5,6 +5,7 @@ # Options read when launching programs locally with # ./bin/run-example or ./bin/spark-submit +# - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files # - SPARK_LOCAL_IP, to set the IP address Spark binds to on this node # - SPARK_PUBLIC_DNS, to set the public dns name of the driver program # - SPARK_CLASSPATH, default classpath entries to append @@ -17,6 +18,7 @@ # - MESOS_NATIVE_LIBRARY, to point to your libmesos.so if you use Mesos # Options read in YARN client mode +# - HADOOP_CONF_DIR, to point Spark towards Hadoop configuration files # - SPARK_EXECUTOR_INSTANCES, Number of workers to start (Default: 2) # - SPARK_EXECUTOR_CORES, Number of cores for the workers (Default: 1). # - SPARK_EXECUTOR_MEMORY, Memory per Worker (e.g. 1000M, 2G) (Default: 1G) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index 02502adfbd0c4..cc976565cc72f 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -116,6 +116,15 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { if (args.length == 0) printUsageAndExit(-1) if (primaryResource == null) SparkSubmit.printErrorAndExit("Must specify a primary resource") if (mainClass == null) SparkSubmit.printErrorAndExit("Must specify a main class with --class") + + if (master.startsWith("yarn")) { + val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") + val testing = sys.env.contains("SPARK_TESTING") + if (!hasHadoopEnv && !testing) { + throw new Exception(s"When running with master '$master' " + + "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.") + } + } } override def toString = { diff --git a/docs/hadoop-third-party-distributions.md b/docs/hadoop-third-party-distributions.md index de6a2b0a43bd5..454877a7fa8a5 100644 --- a/docs/hadoop-third-party-distributions.md +++ b/docs/hadoop-third-party-distributions.md @@ -110,10 +110,5 @@ The location of these configuration files varies across CDH and HDP versions, bu a common location is inside of `/etc/hadoop/conf`. Some tools, such as Cloudera Manager, create configurations on-the-fly, but offer a mechanisms to download copies of them. -There are a few ways to make these files visible to Spark: - -* You can copy these files into `$SPARK_HOME/conf` and they will be included in Spark's -classpath automatically. -* If you are running Spark on the same nodes as Hadoop _and_ your distribution includes both -`hdfs-site.xml` and `core-site.xml` in the same directory, you can set `HADOOP_CONF_DIR` -in `$SPARK_HOME/spark-env.sh` to that directory. +To make these files visible to Spark, set `HADOOP_CONF_DIR` in `$SPARK_HOME/spark-env.sh` +to a location containing the configuration files. diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index d264ecbeaf7d8..b403292d9cb2e 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -375,9 +375,11 @@ object ClientBase { val classpathEntries = Option(conf.getStrings( YarnConfiguration.YARN_APPLICATION_CLASSPATH)).getOrElse( getDefaultYarnApplicationClasspath()) - for (c <- classpathEntries) { - YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, c.trim, - File.pathSeparator) + if (classpathEntries != null) { + for (c <- classpathEntries) { + YarnSparkHadoopUtil.addToEnvironment(env, Environment.CLASSPATH.name, c.trim, + File.pathSeparator) + } } val mrClasspathEntries = Option(conf.getStrings( From 2de573877fbed20092f1b3af20b603b30ba9a940 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Tue, 22 Apr 2014 19:24:03 -0700 Subject: [PATCH 341/397] [Spark-1538] Fix SparkUI incorrectly hiding persisted RDDs **Bug**: After the following command `sc.parallelize(1 to 1000).persist.map(_ + 1).count()` is run, the the persisted RDD is missing from the storage tab of the SparkUI. **Cause**: The command creates two RDDs in one stage, a `ParallelCollectionRDD` and a `MappedRDD`. However, the existing StageInfo only keeps the RDDInfo of the last RDD associated with the stage (`MappedRDD`), and so all RDD information regarding the first RDD (`ParallelCollectionRDD`) is discarded. In this case, we persist the first RDD, but the StorageTab doesn't know about this RDD because it is not encoded in the StageInfo. **Fix**: Record information of all RDDs in StageInfo, instead of just the last RDD (i.e. `stage.rdd`). Since stage boundaries are marked by shuffle dependencies, the solution is to traverse the last RDD's dependency tree, visiting only ancestor RDDs related through a sequence of narrow dependencies. --- This PR also moves RDDInfo to its own file, includes a few style fixes, and adds a unit test for constructing StageInfos. Author: Andrew Or Closes #469 from andrewor14/storage-ui-fix and squashes the following commits: 07fc7f0 [Andrew Or] Add back comment that was accidentally removed (minor) 5d799fe [Andrew Or] Add comment to justify testing of getNarrowAncestors with cycles 9d0e2b8 [Andrew Or] Hide details of getNarrowAncestors from outsiders d2bac8a [Andrew Or] Deal with cycles in RDD dependency graph + add extensive tests 2acb177 [Andrew Or] Move getNarrowAncestors to RDD.scala bfe83f0 [Andrew Or] Backtrace RDD dependency tree to find all RDDs that belong to a Stage --- .../scala/org/apache/spark/TaskContext.scala | 2 +- .../apache/spark/executor/TaskMetrics.scala | 2 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 25 +++ .../apache/spark/scheduler/JobLogger.scala | 2 +- .../apache/spark/scheduler/StageInfo.scala | 19 ++- .../org/apache/spark/storage/RDDInfo.scala | 55 +++++++ .../apache/spark/storage/StorageUtils.scala | 44 +----- .../apache/spark/ui/exec/ExecutorsPage.scala | 6 +- .../apache/spark/ui/storage/StorageTab.scala | 4 +- .../org/apache/spark/util/JsonProtocol.scala | 22 +-- .../org/apache/spark/CacheManagerSuite.scala | 6 +- .../org/apache/spark/PipedRDDSuite.scala | 2 +- .../scala/org/apache/spark/rdd/RDDSuite.scala | 148 +++++++++++++++++- .../spark/scheduler/SparkListenerSuite.scala | 54 ++++++- .../apache/spark/util/JsonProtocolSuite.scala | 8 +- 15 files changed, 318 insertions(+), 81 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/storage/RDDInfo.scala diff --git a/core/src/main/scala/org/apache/spark/TaskContext.scala b/core/src/main/scala/org/apache/spark/TaskContext.scala index dc5a19ecd738e..dc012cc381346 100644 --- a/core/src/main/scala/org/apache/spark/TaskContext.scala +++ b/core/src/main/scala/org/apache/spark/TaskContext.scala @@ -33,7 +33,7 @@ class TaskContext( val attemptId: Long, val runningLocally: Boolean = false, @volatile var interrupted: Boolean = false, - private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty() + private[spark] val taskMetrics: TaskMetrics = TaskMetrics.empty ) extends Serializable { @deprecated("use partitionId", "0.8.1") diff --git a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala index e4f02a4be0b97..350fd74173f65 100644 --- a/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala +++ b/core/src/main/scala/org/apache/spark/executor/TaskMetrics.scala @@ -84,7 +84,7 @@ class TaskMetrics extends Serializable { } private[spark] object TaskMetrics { - def empty(): TaskMetrics = new TaskMetrics + def empty: TaskMetrics = new TaskMetrics } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 5d2ed2b402cd8..596dcb84db7bf 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -20,6 +20,7 @@ package org.apache.spark.rdd import java.util.Random import scala.collection.Map +import scala.collection.mutable import scala.collection.mutable.ArrayBuffer import scala.reflect.{classTag, ClassTag} @@ -229,6 +230,30 @@ abstract class RDD[T: ClassTag]( } } + /** + * Return the ancestors of the given RDD that are related to it only through a sequence of + * narrow dependencies. This traverses the given RDD's dependency tree using DFS, but maintains + * no ordering on the RDDs returned. + */ + private[spark] def getNarrowAncestors: Seq[RDD[_]] = { + val ancestors = new mutable.HashSet[RDD[_]] + + def visit(rdd: RDD[_]) { + val narrowDependencies = rdd.dependencies.filter(_.isInstanceOf[NarrowDependency[_]]) + val narrowParents = narrowDependencies.map(_.rdd) + val narrowParentsNotVisited = narrowParents.filterNot(ancestors.contains) + narrowParentsNotVisited.foreach { parent => + ancestors.add(parent) + visit(parent) + } + } + + visit(this) + + // In case there is a cycle, do not include the root itself + ancestors.filterNot(_ == this).toSeq + } + /** * Compute an RDD partition or read it from a checkpoint if the RDD is checkpointing. */ diff --git a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala index 713aebfa3ce00..a1e21cad48b9b 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/JobLogger.scala @@ -207,7 +207,7 @@ class JobLogger(val user: String, val logDirName: String) extends SparkListener override def onTaskEnd(taskEnd: SparkListenerTaskEnd) { val taskInfo = taskEnd.taskInfo var taskStatus = "TASK_TYPE=%s".format(taskEnd.taskType) - val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty() + val taskMetrics = if (taskEnd.taskMetrics != null) taskEnd.taskMetrics else TaskMetrics.empty taskEnd.reason match { case Success => taskStatus += " STATUS=SUCCESS" recordTaskMetrics(taskEnd.stageId, taskStatus, taskInfo, taskMetrics) diff --git a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala index 9f732f7191465..b42e231e11f91 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/StageInfo.scala @@ -25,7 +25,7 @@ import org.apache.spark.storage.RDDInfo * Stores information about a stage to pass from the scheduler to SparkListeners. */ @DeveloperApi -class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfo: RDDInfo) { +class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddInfos: Seq[RDDInfo]) { /** When this stage was submitted from the DAGScheduler to a TaskScheduler. */ var submissionTime: Option[Long] = None /** Time when all tasks in the stage completed or when the stage was cancelled. */ @@ -41,12 +41,17 @@ class StageInfo(val stageId: Int, val name: String, val numTasks: Int, val rddIn } } -private[spark] -object StageInfo { +private[spark] object StageInfo { + /** + * Construct a StageInfo from a Stage. + * + * Each Stage is associated with one or many RDDs, with the boundary of a Stage marked by + * shuffle dependencies. Therefore, all ancestor RDDs related to this Stage's RDD through a + * sequence of narrow dependencies should also be associated with this Stage. + */ def fromStage(stage: Stage): StageInfo = { - val rdd = stage.rdd - val rddName = Option(rdd.name).getOrElse(rdd.id.toString) - val rddInfo = new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) - new StageInfo(stage.id, stage.name, stage.numTasks, rddInfo) + val ancestorRddInfos = stage.rdd.getNarrowAncestors.map(RDDInfo.fromRdd) + val rddInfos = Seq(RDDInfo.fromRdd(stage.rdd)) ++ ancestorRddInfos + new StageInfo(stage.id, stage.name, stage.numTasks, rddInfos) } } diff --git a/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala new file mode 100644 index 0000000000000..023fd6e4d8baa --- /dev/null +++ b/core/src/main/scala/org/apache/spark/storage/RDDInfo.scala @@ -0,0 +1,55 @@ +/* + * 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.storage + +import org.apache.spark.annotation.DeveloperApi +import org.apache.spark.rdd.RDD +import org.apache.spark.util.Utils + +@DeveloperApi +class RDDInfo( + val id: Int, + val name: String, + val numPartitions: Int, + val storageLevel: StorageLevel) + extends Ordered[RDDInfo] { + + var numCachedPartitions = 0 + var memSize = 0L + var diskSize = 0L + var tachyonSize = 0L + + override def toString = { + import Utils.bytesToString + ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s; " + + "TachyonSize: %s; DiskSize: %s").format( + name, id, storageLevel.toString, numCachedPartitions, numPartitions, + bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) + } + + override def compare(that: RDDInfo) = { + this.id - that.id + } +} + +private[spark] object RDDInfo { + def fromRdd(rdd: RDD[_]): RDDInfo = { + val rddName = Option(rdd.name).getOrElse(rdd.id.toString) + new RDDInfo(rdd.id, rddName, rdd.partitions.size, rdd.getStorageLevel) + } +} diff --git a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala index 7ed371326855d..1eddd1cdc483b 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageUtils.scala @@ -21,60 +21,30 @@ import scala.collection.Map import scala.collection.mutable import org.apache.spark.SparkContext -import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.util.Utils -private[spark] -class StorageStatus( +/** Storage information for each BlockManager. */ +private[spark] class StorageStatus( val blockManagerId: BlockManagerId, val maxMem: Long, val blocks: mutable.Map[BlockId, BlockStatus] = mutable.Map.empty) { - def memUsed() = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) + def memUsed = blocks.values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) def memUsedByRDD(rddId: Int) = rddBlocks.filterKeys(_.rddId == rddId).values.map(_.memSize).reduceOption(_ + _).getOrElse(0L) - def diskUsed() = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) + def diskUsed = blocks.values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) def diskUsedByRDD(rddId: Int) = rddBlocks.filterKeys(_.rddId == rddId).values.map(_.diskSize).reduceOption(_ + _).getOrElse(0L) - def memRemaining : Long = maxMem - memUsed() + def memRemaining: Long = maxMem - memUsed def rddBlocks = blocks.collect { case (rdd: RDDBlockId, status) => (rdd, status) } } -@DeveloperApi -private[spark] -class RDDInfo( - val id: Int, - val name: String, - val numPartitions: Int, - val storageLevel: StorageLevel) - extends Ordered[RDDInfo] { - - var numCachedPartitions = 0 - var memSize = 0L - var diskSize = 0L - var tachyonSize = 0L - - override def toString = { - import Utils.bytesToString - ("RDD \"%s\" (%d) Storage: %s; CachedPartitions: %d; TotalPartitions: %d; MemorySize: %s;" + - "TachyonSize: %s; DiskSize: %s").format( - name, id, storageLevel.toString, numCachedPartitions, numPartitions, - bytesToString(memSize), bytesToString(tachyonSize), bytesToString(diskSize)) - } - - override def compare(that: RDDInfo) = { - this.id - that.id - } -} - -/* Helper methods for storage-related objects */ -private[spark] -object StorageUtils { +/** Helper methods for storage-related objects. */ +private[spark] object StorageUtils { /** * Returns basic information of all RDDs persisted in the given SparkContext. This does not diff --git a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala index c1e69f6cdaffb..6cb43c02b8f08 100644 --- a/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala +++ b/core/src/main/scala/org/apache/spark/ui/exec/ExecutorsPage.scala @@ -32,7 +32,7 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { def render(request: HttpServletRequest): Seq[Node] = { val storageStatusList = listener.storageStatusList val maxMem = storageStatusList.map(_.maxMem).fold(0L)(_ + _) - val memUsed = storageStatusList.map(_.memUsed()).fold(0L)(_ + _) + val memUsed = storageStatusList.map(_.memUsed).fold(0L)(_ + _) val diskSpaceUsed = storageStatusList.flatMap(_.blocks.values.map(_.diskSize)).fold(0L)(_ + _) val execInfo = for (statusId <- 0 until storageStatusList.size) yield getExecInfo(statusId) val execInfoSorted = execInfo.sortBy(_.getOrElse("Executor ID", "")) @@ -106,9 +106,9 @@ private[ui] class ExecutorsPage(parent: ExecutorsTab) extends WebUIPage("") { val execId = status.blockManagerId.executorId val hostPort = status.blockManagerId.hostPort val rddBlocks = status.blocks.size - val memUsed = status.memUsed() + val memUsed = status.memUsed val maxMem = status.maxMem - val diskUsed = status.diskUsed() + val diskUsed = status.diskUsed val activeTasks = listener.executorToTasksActive.getOrElse(execId, 0) val failedTasks = listener.executorToTasksFailed.getOrElse(execId, 0) val completedTasks = listener.executorToTasksComplete.getOrElse(execId, 0) diff --git a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala index c04ef0ae37748..07ec297841f20 100644 --- a/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala +++ b/core/src/main/scala/org/apache/spark/ui/storage/StorageTab.scala @@ -66,8 +66,8 @@ private[ui] class StorageListener(storageStatusListener: StorageStatusListener) } override def onStageSubmitted(stageSubmitted: SparkListenerStageSubmitted) = synchronized { - val rddInfo = stageSubmitted.stageInfo.rddInfo - _rddInfoMap.getOrElseUpdate(rddInfo.id, rddInfo) + val rddInfos = stageSubmitted.stageInfo.rddInfos + rddInfos.foreach { info => _rddInfoMap.getOrElseUpdate(info.id, info) } } override def onStageCompleted(stageCompleted: SparkListenerStageCompleted) = synchronized { diff --git a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala index 465835ea7fe29..9aed3e0985654 100644 --- a/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala +++ b/core/src/main/scala/org/apache/spark/util/JsonProtocol.scala @@ -176,7 +176,7 @@ private[spark] object JsonProtocol { * -------------------------------------------------------------------- */ def stageInfoToJson(stageInfo: StageInfo): JValue = { - val rddInfo = rddInfoToJson(stageInfo.rddInfo) + val rddInfo = JArray(stageInfo.rddInfos.map(rddInfoToJson).toList) val submissionTime = stageInfo.submissionTime.map(JInt(_)).getOrElse(JNothing) val completionTime = stageInfo.completionTime.map(JInt(_)).getOrElse(JNothing) val failureReason = stageInfo.failureReason.map(JString(_)).getOrElse(JNothing) @@ -208,7 +208,8 @@ private[spark] object JsonProtocol { taskMetrics.shuffleReadMetrics.map(shuffleReadMetricsToJson).getOrElse(JNothing) val shuffleWriteMetrics = taskMetrics.shuffleWriteMetrics.map(shuffleWriteMetricsToJson).getOrElse(JNothing) - val updatedBlocks = taskMetrics.updatedBlocks.map { blocks => + val updatedBlocks = + taskMetrics.updatedBlocks.map { blocks => JArray(blocks.toList.map { case (id, status) => ("Block ID" -> id.toString) ~ ("Status" -> blockStatusToJson(status)) @@ -467,13 +468,13 @@ private[spark] object JsonProtocol { val stageId = (json \ "Stage ID").extract[Int] val stageName = (json \ "Stage Name").extract[String] val numTasks = (json \ "Number of Tasks").extract[Int] - val rddInfo = rddInfoFromJson(json \ "RDD Info") + val rddInfos = (json \ "RDD Info").extract[List[JValue]].map(rddInfoFromJson) val submissionTime = Utils.jsonOption(json \ "Submission Time").map(_.extract[Long]) val completionTime = Utils.jsonOption(json \ "Completion Time").map(_.extract[Long]) val failureReason = Utils.jsonOption(json \ "Failure Reason").map(_.extract[String]) val emittedTaskSizeWarning = (json \ "Emitted Task Size Warning").extract[Boolean] - val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfo) + val stageInfo = new StageInfo(stageId, stageName, numTasks, rddInfos) stageInfo.submissionTime = submissionTime stageInfo.completionTime = completionTime stageInfo.failureReason = failureReason @@ -518,13 +519,14 @@ private[spark] object JsonProtocol { Utils.jsonOption(json \ "Shuffle Read Metrics").map(shuffleReadMetricsFromJson) metrics.shuffleWriteMetrics = Utils.jsonOption(json \ "Shuffle Write Metrics").map(shuffleWriteMetricsFromJson) - metrics.updatedBlocks = Utils.jsonOption(json \ "Updated Blocks").map { value => - value.extract[List[JValue]].map { block => - val id = BlockId((block \ "Block ID").extract[String]) - val status = blockStatusFromJson(block \ "Status") - (id, status) + metrics.updatedBlocks = + Utils.jsonOption(json \ "Updated Blocks").map { value => + value.extract[List[JValue]].map { block => + val id = BlockId((block \ "Block ID").extract[String]) + val status = blockStatusFromJson(block \ "Status") + (id, status) + } } - } metrics } diff --git a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala index b86923f07f793..fd5b0906e6765 100644 --- a/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/CacheManagerSuite.scala @@ -60,7 +60,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } @@ -73,7 +73,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(5, 6, 7)) } @@ -87,7 +87,7 @@ class CacheManagerSuite extends FunSuite with BeforeAndAfter with EasyMockSugar whenExecuting(blockManager) { val context = new TaskContext(0, 0, 0, runningLocally = true, interrupted = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val value = cacheManager.getOrCompute(rdd, split, context, StorageLevel.MEMORY_ONLY) assert(value.toList === List(1, 2, 3, 4)) } diff --git a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala index dfe057515efa0..0bb6a6b09c5b5 100644 --- a/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/PipedRDDSuite.scala @@ -179,7 +179,7 @@ class PipedRDDSuite extends FunSuite with SharedSparkContext { val hadoopPart1 = generateFakeHadoopPartition() val pipedRdd = new PipedRDD(nums, "printenv " + varName) val tContext = new TaskContext(0, 0, 0, interrupted = false, runningLocally = false, - taskMetrics = TaskMetrics.empty()) + taskMetrics = TaskMetrics.empty) val rddIter = pipedRdd.compute(hadoopPart1, tContext) val arr = rddIter.toArray assert(arr(0) == "/some/path") diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 1901330d8b188..d7c90346d88a2 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -17,12 +17,10 @@ package org.apache.spark.rdd -import scala.collection.mutable.HashMap -import scala.collection.parallel.mutable +import scala.collection.mutable.{ArrayBuffer, HashMap} +import scala.reflect.ClassTag import org.scalatest.FunSuite -import org.scalatest.concurrent.Timeouts._ -import org.scalatest.time.{Millis, Span} import org.apache.spark._ import org.apache.spark.SparkContext._ @@ -153,7 +151,7 @@ class RDDSuite extends FunSuite with SharedSparkContext { if (shouldFail) { throw new Exception("injected failure") } else { - return Array(1, 2, 3, 4).iterator + Array(1, 2, 3, 4).iterator } } }.cache() @@ -568,4 +566,144 @@ class RDDSuite extends FunSuite with SharedSparkContext { val ids = ranked.map(_._1).distinct().collect() assert(ids.length === n) } + + test("getNarrowAncestors") { + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.filter(_ % 2 == 0).map(_ + 1) + val rdd3 = rdd2.map(_ - 1).filter(_ < 50).map(i => (i, i)) + val rdd4 = rdd3.reduceByKey(_ + _) + val rdd5 = rdd4.mapValues(_ + 1).mapValues(_ + 2).mapValues(_ + 3) + val ancestors1 = rdd1.getNarrowAncestors + val ancestors2 = rdd2.getNarrowAncestors + val ancestors3 = rdd3.getNarrowAncestors + val ancestors4 = rdd4.getNarrowAncestors + val ancestors5 = rdd5.getNarrowAncestors + + // Simple dependency tree with a single branch + assert(ancestors1.size === 0) + assert(ancestors2.size === 2) + assert(ancestors2.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) + assert(ancestors2.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors3.size === 5) + assert(ancestors3.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 1) + assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + + // Any ancestors before the shuffle are not considered + assert(ancestors4.size === 1) + assert(ancestors4.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) + assert(ancestors5.size === 4) + assert(ancestors5.count(_.isInstanceOf[ShuffledRDD[_, _, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[MapPartitionsRDD[_, _]]) === 1) + assert(ancestors5.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 2) + } + + test("getNarrowAncestors with multiple parents") { + val rdd1 = sc.parallelize(1 to 100, 5) + val rdd2 = sc.parallelize(1 to 200, 10).map(_ + 1) + val rdd3 = sc.parallelize(1 to 300, 15).filter(_ > 50) + val rdd4 = rdd1.map(i => (i, i)) + val rdd5 = rdd2.map(i => (i, i)) + val rdd6 = sc.union(rdd1, rdd2) + val rdd7 = sc.union(rdd1, rdd2, rdd3) + val rdd8 = sc.union(rdd6, rdd7) + val rdd9 = rdd4.join(rdd5) + val ancestors6 = rdd6.getNarrowAncestors + val ancestors7 = rdd7.getNarrowAncestors + val ancestors8 = rdd8.getNarrowAncestors + val ancestors9 = rdd9.getNarrowAncestors + + // Simple dependency tree with multiple branches + assert(ancestors6.size === 3) + assert(ancestors6.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 2) + assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors7.size === 5) + assert(ancestors7.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) + assert(ancestors7.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors7.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + + // Dependency tree with duplicate nodes (e.g. rdd1 should not be reported twice) + assert(ancestors8.size === 7) + assert(ancestors8.count(_.isInstanceOf[MappedRDD[_, _]]) === 1) + assert(ancestors8.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors8.count(_.isInstanceOf[UnionRDD[_]]) === 2) + assert(ancestors8.count(_.isInstanceOf[ParallelCollectionRDD[_]]) === 3) + assert(ancestors8.count(_ == rdd1) === 1) + assert(ancestors8.count(_ == rdd2) === 1) + assert(ancestors8.count(_ == rdd3) === 1) + + // Any ancestors before the shuffle are not considered + assert(ancestors9.size === 2) + assert(ancestors9.count(_.isInstanceOf[CoGroupedRDD[_]]) === 1) + assert(ancestors9.count(_.isInstanceOf[MappedValuesRDD[_, _, _]]) === 1) + } + + /** + * This tests for the pathological condition in which the RDD dependency graph is cyclical. + * + * Since RDD is part of the public API, applications may actually implement RDDs that allow + * such graphs to be constructed. In such cases, getNarrowAncestor should not simply hang. + */ + test("getNarrowAncestors with cycles") { + val rdd1 = new CyclicalDependencyRDD[Int] + val rdd2 = new CyclicalDependencyRDD[Int] + val rdd3 = new CyclicalDependencyRDD[Int] + val rdd4 = rdd3.map(_ + 1).filter(_ > 10).map(_ + 2).filter(_ % 5 > 1) + val rdd5 = rdd4.map(_ + 2).filter(_ > 20) + val rdd6 = sc.union(rdd1, rdd2, rdd3).map(_ + 4).union(rdd5).union(rdd4) + + // Simple cyclical dependency + rdd1.addDependency(new OneToOneDependency[Int](rdd2)) + rdd2.addDependency(new OneToOneDependency[Int](rdd1)) + val ancestors1 = rdd1.getNarrowAncestors + val ancestors2 = rdd2.getNarrowAncestors + assert(ancestors1.size === 1) + assert(ancestors1.count(_ == rdd2) === 1) + assert(ancestors1.count(_ == rdd1) === 0) + assert(ancestors2.size === 1) + assert(ancestors2.count(_ == rdd1) === 1) + assert(ancestors2.count(_ == rdd2) === 0) + + // Cycle involving a longer chain + rdd3.addDependency(new OneToOneDependency[Int](rdd4)) + val ancestors3 = rdd3.getNarrowAncestors + val ancestors4 = rdd4.getNarrowAncestors + assert(ancestors3.size === 4) + assert(ancestors3.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors3.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors3.count(_ == rdd3) === 0) + assert(ancestors4.size === 4) + assert(ancestors4.count(_.isInstanceOf[MappedRDD[_, _]]) === 2) + assert(ancestors4.count(_.isInstanceOf[FilteredRDD[_]]) === 1) + assert(ancestors4.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) + assert(ancestors4.count(_ == rdd3) === 1) + assert(ancestors4.count(_ == rdd4) === 0) + + // Cycles that do not involve the root + val ancestors5 = rdd5.getNarrowAncestors + assert(ancestors5.size === 6) + assert(ancestors5.count(_.isInstanceOf[MappedRDD[_, _]]) === 3) + assert(ancestors5.count(_.isInstanceOf[FilteredRDD[_]]) === 2) + assert(ancestors5.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 1) + assert(ancestors4.count(_ == rdd3) === 1) + + // Complex cyclical dependency graph (combination of all of the above) + val ancestors6 = rdd6.getNarrowAncestors + assert(ancestors6.size === 12) + assert(ancestors6.count(_.isInstanceOf[UnionRDD[_]]) === 2) + assert(ancestors6.count(_.isInstanceOf[MappedRDD[_, _]]) === 4) + assert(ancestors6.count(_.isInstanceOf[FilteredRDD[_]]) === 3) + assert(ancestors6.count(_.isInstanceOf[CyclicalDependencyRDD[_]]) === 3) + } + + /** A contrived RDD that allows the manual addition of dependencies after creation. */ + private class CyclicalDependencyRDD[T: ClassTag] extends RDD[T](sc, Nil) { + private val mutableDependencies: ArrayBuffer[Dependency[_]] = ArrayBuffer.empty + override def compute(p: Partition, c: TaskContext): Iterator[T] = Iterator.empty + override def getPartitions: Array[Partition] = Array.empty + override def getDependencies: Seq[Dependency[_]] = mutableDependencies + def addDependency(dep: Dependency[_]) { + mutableDependencies += dep + } + } } diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index 36511a9e95474..ab139175e00eb 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -133,20 +133,57 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc val rdd1 = sc.parallelize(1 to 100, 4) val rdd2 = rdd1.map(_.toString) rdd2.setName("Target RDD") - rdd2.count + rdd2.count() assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {1} val (stageInfo, taskInfoMetrics) = listener.stageInfos.head - stageInfo.rddInfo.name should be {"Target RDD"} + stageInfo.rddInfos.size should be {2} + stageInfo.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo.rddInfos.exists(_.name == "Target RDD") should be {true} stageInfo.numTasks should be {4} - stageInfo.rddInfo.numPartitions should be {4} stageInfo.submissionTime should be ('defined) stageInfo.completionTime should be ('defined) taskInfoMetrics.length should be {4} } + test("basic creation of StageInfo with shuffle") { + val listener = new SaveStageAndTaskInfo + sc.addSparkListener(listener) + val rdd1 = sc.parallelize(1 to 100, 4) + val rdd2 = rdd1.filter(_ % 2 == 0).map(i => (i, i)) + val rdd3 = rdd2.reduceByKey(_ + _) + rdd1.setName("Un") + rdd2.setName("Deux") + rdd3.setName("Trois") + + rdd1.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {1} + val stageInfo1 = listener.stageInfos.keys.find(_.stageId == 0).get + stageInfo1.rddInfos.size should be {1} // ParallelCollectionRDD + stageInfo1.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo1.rddInfos.exists(_.name == "Un") should be {true} + listener.stageInfos.clear() + + rdd2.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) + listener.stageInfos.size should be {1} + val stageInfo2 = listener.stageInfos.keys.find(_.stageId == 1).get + stageInfo2.rddInfos.size should be {3} // ParallelCollectionRDD, FilteredRDD, MappedRDD + stageInfo2.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo2.rddInfos.exists(_.name == "Deux") should be {true} + listener.stageInfos.clear() + + rdd3.count() + listener.stageInfos.size should be {2} // Shuffle map stage + result stage + val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 2).get + stageInfo3.rddInfos.size should be {2} // ShuffledRDD, MapPartitionsRDD + stageInfo3.rddInfos.forall(_.numPartitions == 4) should be {true} + stageInfo3.rddInfos.exists(_.name == "Trois") should be {true} + } + test("StageInfo with fewer tasks than partitions") { val listener = new SaveStageAndTaskInfo sc.addSparkListener(listener) @@ -159,7 +196,8 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.size should be {1} val (stageInfo, _) = listener.stageInfos.head stageInfo.numTasks should be {2} - stageInfo.rddInfo.numPartitions should be {4} + stageInfo.rddInfos.size should be {2} + stageInfo.rddInfos.forall(_.numPartitions == 4) should be {true} } test("local metrics") { @@ -167,7 +205,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc.addSparkListener(listener) sc.addSparkListener(new StatsReportListener) // just to make sure some of the tasks take a noticeable amount of time - val w = {i:Int => + val w = { i: Int => if (i == 0) Thread.sleep(100) i @@ -199,7 +237,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc checkNonZeroAvg( taskInfoMetrics.map(_._2.executorDeserializeTime), stageInfo + " executorDeserializeTime") - if (stageInfo.rddInfo.name == d4.name) { + if (stageInfo.rddInfos.exists(_.name == d4.name)) { checkNonZeroAvg( taskInfoMetrics.map(_._2.shuffleReadMetrics.get.fetchWaitTime), stageInfo + " fetchWaitTime") @@ -207,11 +245,11 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc taskInfoMetrics.foreach { case (taskInfo, taskMetrics) => taskMetrics.resultSize should be > (0l) - if (stageInfo.rddInfo.name == d2.name || stageInfo.rddInfo.name == d3.name) { + if (stageInfo.rddInfos.exists(info => info.name == d2.name || info.name == d3.name)) { taskMetrics.shuffleWriteMetrics should be ('defined) taskMetrics.shuffleWriteMetrics.get.shuffleBytesWritten should be > (0l) } - if (stageInfo.rddInfo.name == d4.name) { + if (stageInfo.rddInfos.exists(_.name == d4.name)) { taskMetrics.shuffleReadMetrics should be ('defined) val sm = taskMetrics.shuffleReadMetrics.get sm.totalBlocksFetched should be > (0) diff --git a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala index 16470bb7bf60d..3031015256ec9 100644 --- a/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala +++ b/core/src/test/scala/org/apache/spark/util/JsonProtocolSuite.scala @@ -231,7 +231,10 @@ class JsonProtocolSuite extends FunSuite { assert(info1.submissionTime === info2.submissionTime) assert(info1.completionTime === info2.completionTime) assert(info1.emittedTaskSizeWarning === info2.emittedTaskSizeWarning) - assertEquals(info1.rddInfo, info2.rddInfo) + assert(info1.rddInfos.size === info2.rddInfos.size) + (0 until info1.rddInfos.size).foreach { i => + assertEquals(info1.rddInfos(i), info2.rddInfos(i)) + } } private def assertEquals(info1: RDDInfo, info2: RDDInfo) { @@ -434,7 +437,8 @@ class JsonProtocolSuite extends FunSuite { } private def makeStageInfo(a: Int, b: Int, c: Int, d: Long, e: Long) = { - new StageInfo(a, "greetings", b, makeRddInfo(a, b, c, d, e)) + val rddInfos = (1 to a % 5).map { i => makeRddInfo(a % i, b % i, c % i, d % i, e % i) } + new StageInfo(a, "greetings", b, rddInfos) } private def makeTaskInfo(a: Long, b: Int, c: Long) = { From f3d19a9f1a4e38ff9fb5bf78e04ed5d321219bf6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Tue, 22 Apr 2014 19:35:13 -0700 Subject: [PATCH 342/397] [streaming][SPARK-1578] Removed requirement for TTL in StreamingContext. Since shuffles and RDDs that are out of context are automatically cleaned by Spark core (using ContextCleaner) there is no need for setting the cleaner TTL while creating a StreamingContext. Author: Tathagata Das Closes #491 from tdas/ttl-fix and squashes the following commits: cf01dc7 [Tathagata Das] Removed requirement for TTL in StreamingContext. --- .../spark/streaming/StreamingContext.scala | 15 +------ .../spark/streaming/InputStreamsSuite.scala | 2 +- .../streaming/StreamingContextSuite.scala | 45 +++++-------------- .../spark/streaming/TestSuiteBase.scala | 1 - 4 files changed, 14 insertions(+), 49 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 6d9dc87a707ac..9ba6e02229aaa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -116,11 +116,6 @@ class StreamingContext private[streaming] ( } } - if (MetadataCleaner.getDelaySeconds(sc.conf) < 0) { - throw new SparkException("Spark Streaming cannot be used without setting spark.cleaner.ttl; " - + "set this property before creating a SparkContext (use SPARK_JAVA_OPTS for the shell)") - } - private[streaming] val conf = sc.conf private[streaming] val env = SparkEnv.get @@ -500,8 +495,6 @@ class StreamingContext private[streaming] ( object StreamingContext extends Logging { - private[streaming] val DEFAULT_CLEANER_TTL = 3600 - implicit def toPairDStreamFunctions[K: ClassTag, V: ClassTag](stream: DStream[(K,V)]) = { new PairDStreamFunctions[K, V](stream) } @@ -546,13 +539,7 @@ object StreamingContext extends Logging { def jarOfClass(cls: Class[_]): Option[String] = SparkContext.jarOfClass(cls) private[streaming] def createNewSparkContext(conf: SparkConf): SparkContext = { - // Set the default cleaner delay to an hour if not already set. - // This should be sufficient for even 1 second batch intervals. - if (MetadataCleaner.getDelaySeconds(conf) < 0) { - MetadataCleaner.setDelaySeconds(conf, DEFAULT_CLEANER_TTL) - } - val sc = new SparkContext(conf) - sc + new SparkContext(conf) } private[streaming] def createNewSparkContext( diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 46b7f63b6567e..3bad871b5c580 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -143,7 +143,7 @@ class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { conf.set("spark.streaming.clock", "org.apache.spark.streaming.util.ManualClock") } - // TODO: This test makes assumptions about Thread.sleep() and is flaky + // TODO: This test works in IntelliJ but not through SBT ignore("actor input stream") { // Start the server val testServer = new TestServer() diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala index 6d14b1f785db3..3e2b25af84098 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingContextSuite.scala @@ -38,15 +38,10 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w val batchDuration = Milliseconds(500) val sparkHome = "someDir" val envPair = "key" -> "value" - val ttl = StreamingContext.DEFAULT_CLEANER_TTL + 100 var sc: SparkContext = null var ssc: StreamingContext = null - before { - System.clearProperty("spark.cleaner.ttl") - } - after { if (ssc != null) { ssc.stop() @@ -62,67 +57,51 @@ class StreamingContextSuite extends FunSuite with BeforeAndAfter with Timeouts w ssc = new StreamingContext(master, appName, batchDuration) assert(ssc.sparkContext.conf.get("spark.master") === master) assert(ssc.sparkContext.conf.get("spark.app.name") === appName) - assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) } test("from no conf + spark home") { ssc = new StreamingContext(master, appName, batchDuration, sparkHome, Nil) assert(ssc.conf.get("spark.home") === sparkHome) - assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) } test("from no conf + spark home + env") { ssc = new StreamingContext(master, appName, batchDuration, sparkHome, Nil, Map(envPair)) assert(ssc.conf.getExecutorEnv.exists(_ == envPair)) - assert(MetadataCleaner.getDelaySeconds(ssc.sparkContext.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) - } - - test("from conf without ttl set") { - val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - ssc = new StreamingContext(myConf, batchDuration) - assert(MetadataCleaner.getDelaySeconds(ssc.conf) === - StreamingContext.DEFAULT_CLEANER_TTL) } - test("from conf with ttl set") { + test("from conf with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", ttl.toString) + myConf.set("spark.cleaner.ttl", "10") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === ttl) + assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } - test("from existing SparkContext without ttl set") { + test("from existing SparkContext") { sc = new SparkContext(master, appName) - val exception = intercept[SparkException] { - ssc = new StreamingContext(sc, batchDuration) - } - assert(exception.getMessage.contains("ttl")) + ssc = new StreamingContext(sc, batchDuration) } - test("from existing SparkContext with ttl set") { + test("from existing SparkContext with settings") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", ttl.toString) + myConf.set("spark.cleaner.ttl", "10") ssc = new StreamingContext(myConf, batchDuration) - assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === ttl) + assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } test("from checkpoint") { val myConf = SparkContext.updatedConf(new SparkConf(false), master, appName) - myConf.set("spark.cleaner.ttl", ttl.toString) + myConf.set("spark.cleaner.ttl", "10") val ssc1 = new StreamingContext(myConf, batchDuration) addInputStream(ssc1).register ssc1.start() val cp = new Checkpoint(ssc1, Time(1000)) - assert(MetadataCleaner.getDelaySeconds(cp.sparkConf) === ttl) + assert(cp.sparkConfPairs.toMap.getOrElse("spark.cleaner.ttl", "-1") === "10") ssc1.stop() val newCp = Utils.deserialize[Checkpoint](Utils.serialize(cp)) - assert(MetadataCleaner.getDelaySeconds(newCp.sparkConf) === ttl) + assert(newCp.sparkConf.getInt("spark.cleaner.ttl", -1) === 10) ssc = new StreamingContext(null, newCp, null) - assert(MetadataCleaner.getDelaySeconds(ssc.conf) === ttl) + assert(ssc.conf.getInt("spark.cleaner.ttl", -1) === 10) } test("start and stop state check") { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala index aa2d5c2fc2454..4f63fd37822cb 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/TestSuiteBase.scala @@ -137,7 +137,6 @@ trait TestSuiteBase extends FunSuite with BeforeAndAfter with Logging { val conf = new SparkConf() .setMaster(master) .setAppName(framework) - .set("spark.cleaner.ttl", StreamingContext.DEFAULT_CLEANER_TTL.toString) // Default before function for any streaming test suite. Override this // if you want to add your stuff to "before" (i.e., don't call before { } ) From 662c860ebcec5565493a7dff4812e6b7a37b1d7d Mon Sep 17 00:00:00 2001 From: Xiangrui Meng Date: Tue, 22 Apr 2014 19:38:27 -0700 Subject: [PATCH 343/397] [FIX: SPARK-1376] use --arg instead of --args in SparkSubmit to avoid warning messages Even if users use `--arg`, `SparkSubmit` still uses `--args` for child args internally, which triggers a warning message that may confuse users: ~~~ --args is deprecated. Use --arg instead. ~~~ @sryza Does it look good to you? Author: Xiangrui Meng Closes #485 from mengxr/submit-arg and squashes the following commits: 5e1b9fe [Xiangrui Meng] update test cebbeb7 [Xiangrui Meng] use --arg instead of --args in SparkSubmit to avoid warning messages --- core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala | 2 +- .../test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 1b1e0fce0e0df..a2efddbfe3f69 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -196,7 +196,7 @@ object SparkSubmit { childArgs ++= appArgs.childArgs } else if (clusterManager == YARN) { for (arg <- appArgs.childArgs) { - childArgs += ("--args", arg) + childArgs += ("--arg", arg) } } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index f82d717719b69..657b44668d385 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -110,7 +110,7 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { childArgsStr should include ("--executor-memory 5g") childArgsStr should include ("--driver-memory 4g") childArgsStr should include ("--executor-cores 5") - childArgsStr should include ("--args arg1 --args arg2") + childArgsStr should include ("--arg arg1 --arg arg2") childArgsStr should include ("--queue thequeue") childArgsStr should include ("--files file1.txt,file2.txt") childArgsStr should include ("--archives archive1.txt,archive2.txt") From aa77f8a6a604efe0d02bc8412b3f1ba3903b7a57 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 22 Apr 2014 20:02:33 -0700 Subject: [PATCH 344/397] SPARK-1562 Fix visibility / annotation of Spark SQL APIs Author: Michael Armbrust Closes #489 from marmbrus/sqlDocFixes and squashes the following commits: acee4f3 [Michael Armbrust] Fix visibility / annotation of Spark SQL APIs --- project/SparkBuild.scala | 4 +-- .../org/apache/spark/sql/SchemaRDDLike.scala | 2 +- .../spark/sql/execution/Aggregate.scala | 3 ++ .../apache/spark/sql/execution/Exchange.scala | 7 +++- .../apache/spark/sql/execution/Generate.scala | 3 ++ .../spark/sql/execution/SparkPlan.scala | 13 +++++-- .../sql/execution/SparkSqlSerializer.scala | 8 ++--- .../spark/sql/execution/SparkStrategies.scala | 2 +- .../spark/sql/execution/basicOperators.scala | 34 ++++++++++++++++++- .../apache/spark/sql/execution/debug.scala | 4 +-- .../apache/spark/sql/execution/joins.scala | 18 ++++++++++ .../apache/spark/sql/execution/package.scala | 1 + .../spark/sql/parquet/ParquetTestData.scala | 2 +- .../org/apache/spark/SparkHadoopWriter.scala | 9 ++--- .../spark/sql/hive/HiveMetastoreCatalog.scala | 8 +++-- .../org/apache/spark/sql/hive/HiveQl.scala | 14 ++++---- .../spark/sql/hive/HiveStrategies.scala | 3 +- .../spark/sql/hive/ScriptTransformation.scala | 6 +++- .../org/apache/spark/sql/hive/TestHive.scala | 3 +- .../apache/spark/sql/hive/hiveOperators.scala | 13 +++++-- .../org/apache/spark/sql/hive/hiveUdfs.scala | 18 +++++----- .../spark/sql/hive/CachedTableSuite.scala | 1 + .../sql/hive/InsertIntoHiveTableSuite.scala | 5 ++- .../sql/hive/api/java/JavaHiveSuite.scala | 2 +- .../execution/BigDataBenchmarkSuite.scala | 2 +- .../hive/execution/ConcurrentHiveSuite.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 2 +- .../execution/HiveCompatibilitySuite.scala | 2 +- .../sql/hive/execution/HiveQuerySuite.scala | 2 +- .../hive/execution/HiveResolutionSuite.scala | 4 +-- .../sql/hive/execution/PruningSuite.scala | 2 +- .../spark/sql/parquet/HiveParquetSuite.scala | 3 +- 32 files changed, 143 insertions(+), 59 deletions(-) diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 1ca4e0dc8add1..852358501af56 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -367,7 +367,7 @@ object SparkBuild extends Build { publish := {}, unidocProjectFilter in (ScalaUnidoc, unidoc) := - inAnyProject -- inProjects(repl, examples, tools, yarn, yarnAlpha), + inAnyProject -- inProjects(repl, examples, tools, catalyst, yarn, yarnAlpha), unidocProjectFilter in (JavaUnidoc, unidoc) := inAnyProject -- inProjects(repl, examples, bagel, graphx, catalyst, tools, yarn, yarnAlpha), @@ -457,7 +457,7 @@ object SparkBuild extends Build { 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 + // assumptions about the the expression ids being contiguous. Running tests in parallel breaks // this non-deterministically. TODO: FIX THIS. parallelExecution in Test := false, libraryDependencies ++= Seq( diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala index a390ab6005dda..3a895e15a4508 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDDLike.scala @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical._ /** * Contains functions that are shared between all SchemaRDD types (i.e., Scala, Java) */ -trait SchemaRDDLike { +private[sql] trait SchemaRDDLike { @transient val sqlContext: SQLContext @transient protected[spark] val logicalPlan: LogicalPlan diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala index 3a4f071eebedf..36b3b956da96c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Aggregate.scala @@ -19,12 +19,14 @@ package org.apache.spark.sql.execution import java.util.HashMap +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.SparkContext import org.apache.spark.sql.catalyst.errors._ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical._ /** + * :: DeveloperApi :: * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each * group. * @@ -34,6 +36,7 @@ import org.apache.spark.sql.catalyst.plans.physical._ * @param aggregateExpressions expressions that are computed for each group. * @param child the input data source. */ +@DeveloperApi case class Aggregate( partial: Boolean, groupingExpressions: Seq[Expression], 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 070557e47c4c7..3b4acb72e87b5 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 @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf} import org.apache.spark.rdd.ShuffledRDD import org.apache.spark.sql.Row @@ -26,6 +27,10 @@ import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.catalyst.rules.Rule import org.apache.spark.util.MutablePair +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode { override def outputPartitioning = newPartitioning @@ -81,7 +86,7 @@ case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends Una * [[catalyst.plans.physical.Distribution Distribution]] requirements for each operator by inserting * [[Exchange]] Operators where required. */ -object AddExchange extends Rule[SparkPlan] { +private[sql] object AddExchange extends Rule[SparkPlan] { // TODO: Determine the number of partitions. val numPartitions = 150 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 cff4887936ae1..da1e08be59e23 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 @@ -17,9 +17,11 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, Projection} /** + * :: DeveloperApi :: * 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, which allows the input rows to be joined with @@ -29,6 +31,7 @@ import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, * @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. */ +@DeveloperApi case class Generate( generator: Generator, join: Boolean, diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala index 5d89697db5f99..50124dd407447 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.execution +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.{Logging, Row} import org.apache.spark.sql.catalyst.trees @@ -26,6 +27,10 @@ import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical} import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan +/** + * :: DeveloperApi :: + */ +@DeveloperApi abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { self: Product => @@ -51,6 +56,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { } /** + * :: DeveloperApi :: * Allows already planned SparkQueries to be linked into logical query plans. * * Note that in general it is not valid to use this class to link multiple copies of the same @@ -59,6 +65,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging { * replace the output attributes with new copies of themselves without breaking any attribute * linking. */ +@DeveloperApi case class SparkLogicalPlan(alreadyPlanned: SparkPlan) extends logical.LogicalPlan with MultiInstanceRelation { @@ -77,15 +84,15 @@ case class SparkLogicalPlan(alreadyPlanned: SparkPlan) } } -trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { +private[sql] trait LeafNode extends SparkPlan with trees.LeafNode[SparkPlan] { self: Product => } -trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] { +private[sql] trait UnaryNode extends SparkPlan with trees.UnaryNode[SparkPlan] { self: Product => override def outputPartitioning: Partitioning = child.outputPartitioning } -trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] { +private[sql] trait BinaryNode extends SparkPlan with trees.BinaryNode[SparkPlan] { self: Product => } diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala index c30ae5bcc02d0..5067c14ddffeb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala @@ -27,7 +27,7 @@ import org.apache.spark.serializer.KryoSerializer import org.apache.spark.util.MutablePair import org.apache.spark.util.Utils -class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { +private[sql] class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { override def newKryo(): Kryo = { val kryo = new Kryo() kryo.setRegistrationRequired(false) @@ -50,7 +50,7 @@ class SparkSqlSerializer(conf: SparkConf) extends KryoSerializer(conf) { } } -object SparkSqlSerializer { +private[sql] object SparkSqlSerializer { // TODO (lian) Using KryoSerializer here is workaround, needs further investigation // Using SparkSqlSerializer here makes BasicQuerySuite to fail because of Kryo serialization // related error. @@ -68,7 +68,7 @@ object SparkSqlSerializer { } } -class BigDecimalSerializer extends Serializer[BigDecimal] { +private[sql] 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()) @@ -83,7 +83,7 @@ class BigDecimalSerializer extends Serializer[BigDecimal] { * Maps do not have a no arg constructor and so cannot be serialized by default. So, we serialize * them as `Array[(k,v)]`. */ -class MapSerializer extends Serializer[Map[_,_]] { +private[sql] class MapSerializer extends Serializer[Map[_,_]] { def write(kryo: Kryo, output: Output, map: Map[_,_]) { kryo.writeObject(output, map.flatMap(e => Seq(e._1, e._2)).toArray) } 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 fe8bd5a508820..500fde19719af 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,7 +25,7 @@ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.catalyst.plans.physical._ import org.apache.spark.sql.parquet._ -abstract class SparkStrategies extends QueryPlanner[SparkPlan] { +private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => object HashJoin extends Strategy { 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 eedcc7dda02d7..e4cf2020a1a93 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 @@ -19,6 +19,7 @@ package org.apache.spark.sql.execution import scala.reflect.runtime.universe.TypeTag +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.{HashPartitioner, SparkConf, SparkContext} import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.sql.catalyst.ScalaReflection @@ -27,6 +28,10 @@ import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans.physical.{OrderedDistribution, UnspecifiedDistribution} import org.apache.spark.util.MutablePair +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends UnaryNode { override def output = projectList.map(_.toAttribute) @@ -36,6 +41,10 @@ case class Project(projectList: Seq[NamedExpression], child: SparkPlan) extends } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { override def output = child.output @@ -44,6 +53,10 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan) extends UnaryNode { @@ -53,6 +66,10 @@ case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: override def execute() = child.execute().sample(withReplacement, fraction, seed) } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends SparkPlan { // TODO: attributes output by union should be distinct for nullability purposes override def output = children.head.output @@ -62,12 +79,14 @@ case class Union(children: Seq[SparkPlan])(@transient sc: SparkContext) extends } /** + * :: DeveloperApi :: * Take the first limit elements. Note that the implementation is different depending on whether * this is a terminal operator or not. If it is terminal and is invoked using executeCollect, * this operator uses Spark's take method on the Spark driver. If it is not terminal or is * invoked using execute, we first take the limit on each partition, and then repartition all the * data to a single partition to compute the global limit. */ +@DeveloperApi case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) extends UnaryNode { // TODO: Implement a partition local limit, and use a strategy to generate the proper limit plan: // partition local limit -> exchange into one partition -> partition local limit again @@ -91,10 +110,12 @@ case class Limit(limit: Int, child: SparkPlan)(@transient sc: SparkContext) exte } /** + * :: DeveloperApi :: * Take the first limit elements as defined by the sortOrder. This is logically equivalent to * having a [[Limit]] operator after a [[Sort]] operator. This could have been named TopK, but * Spark's top operator does the opposite in ordering so we name it TakeOrdered to avoid confusion. */ +@DeveloperApi case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) (@transient sc: SparkContext) extends UnaryNode { override def otherCopyArgs = sc :: Nil @@ -111,7 +132,10 @@ case class TakeOrdered(limit: Int, sortOrder: Seq[SortOrder], child: SparkPlan) override def execute() = sc.makeRDD(executeCollect(), 1) } - +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class Sort( sortOrder: Seq[SortOrder], global: Boolean, @@ -134,6 +158,10 @@ case class Sort( override def output = child.output } +/** + * :: DeveloperApi :: + */ +@DeveloperApi object ExistingRdd { def convertToCatalyst(a: Any): Any = a match { case s: Seq[Any] => s.map(convertToCatalyst) @@ -167,6 +195,10 @@ object ExistingRdd { } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class ExistingRdd(output: Seq[Attribute], rdd: RDD[Row]) extends LeafNode { override def execute() = rdd } 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 index 40982f1fffbbf..a0d29100f505a 100644 --- 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 @@ -17,7 +17,7 @@ package org.apache.spark.sql.execution -object DebugQuery { +private[sql] object DebugQuery { def apply(plan: SparkPlan): SparkPlan = { val visited = new collection.mutable.HashSet[Long]() plan transform { @@ -28,7 +28,7 @@ object DebugQuery { } } -case class DebugNode(child: SparkPlan) extends UnaryNode { +private[sql] case class DebugNode(child: SparkPlan) extends UnaryNode { def references = Set.empty def output = child.output def execute() = { 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 c89dae9358bf7..31cc26962ad93 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 @@ -21,14 +21,24 @@ import scala.collection.mutable.{ArrayBuffer, BitSet} import org.apache.spark.SparkContext +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.physical.{ClusteredDistribution, Partitioning} +@DeveloperApi sealed abstract class BuildSide + +@DeveloperApi case object BuildLeft extends BuildSide + +@DeveloperApi case object BuildRight extends BuildSide +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class HashJoin( leftKeys: Seq[Expression], rightKeys: Seq[Expression], @@ -130,6 +140,10 @@ case class HashJoin( } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNode { def output = left.output ++ right.output @@ -138,6 +152,10 @@ case class CartesianProduct(left: SparkPlan, right: SparkPlan) extends BinaryNod } } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class BroadcastNestedLoopJoin( streamed: SparkPlan, broadcast: SparkPlan, joinType: JoinType, condition: Option[Expression]) (@transient sc: SparkContext) 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 index e4a2dec33231d..66237f8f1314b 100644 --- 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 @@ -18,6 +18,7 @@ package org.apache.spark.sql /** + * :: DeveloperApi :: * An execution engine for relational query plans that runs on top Spark and returns RDDs. * * Note that the operators in this package are created automatically by a query planner using a 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 728e3dd1dc02b..f37976f7313c1 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 @@ -28,7 +28,7 @@ import parquet.schema.{MessageType, MessageTypeParser} import org.apache.spark.sql.catalyst.expressions.GenericRow import org.apache.spark.util.Utils -object ParquetTestData { +private[sql] object ParquetTestData { val testSchema = """message myrecord { diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala index 7219c030cb0f0..ab7862f4f9e06 100644 --- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala +++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark +package org.apache.spark.sql.hive import java.io.IOException import java.text.NumberFormat @@ -28,12 +28,13 @@ import org.apache.hadoop.hive.ql.plan.FileSinkDesc import org.apache.hadoop.mapred._ import org.apache.hadoop.io.Writable +import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} + /** * Internal helper class that saves an RDD using a Hive OutputFormat. * It is based on [[SparkHadoopWriter]]. */ -protected[spark] -class SparkHiveHadoopWriter( +private[hive] class SparkHiveHadoopWriter( @transient jobConf: JobConf, fileSinkConf: FileSinkDesc) extends Logging @@ -179,7 +180,7 @@ class SparkHiveHadoopWriter( } } -object SparkHiveHadoopWriter { +private[hive] object SparkHiveHadoopWriter { def createPathFromString(path: String, conf: JobConf): Path = { if (path == null) { throw new IllegalArgumentException("Output path is 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 ca75cecf7d885..6c907887db79e 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 @@ -34,12 +34,13 @@ 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.execution.SparkLogicalPlan +import org.apache.spark.sql.hive.execution.{HiveTableScan, InsertIntoHiveTable} import org.apache.spark.sql.columnar.InMemoryColumnarTableScan /* Implicit conversions */ import scala.collection.JavaConversions._ -class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { +private[hive] class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { import HiveMetastoreTypes._ val client = Hive.get(hive.hiveconf) @@ -171,7 +172,7 @@ class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging { override def unregisterAllTables() = {} } -object HiveMetastoreTypes extends RegexParsers { +private[hive] object HiveMetastoreTypes extends RegexParsers { protected lazy val primitiveType: Parser[DataType] = "string" ^^^ StringType | "float" ^^^ FloatType | @@ -229,7 +230,8 @@ object HiveMetastoreTypes extends RegexParsers { } } -case class MetastoreRelation(databaseName: String, tableName: String, alias: Option[String]) +private[hive] case class MetastoreRelation + (databaseName: String, tableName: String, alias: Option[String]) (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 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 4dac25b3f60e4..1777e96b6713d 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 @@ -36,20 +36,20 @@ import scala.collection.JavaConversions._ * back for Hive to execute natively. Will be replaced with a native command that contains the * cmd string. */ -case object NativePlaceholder extends Command +private[hive] case object NativePlaceholder extends Command -case class DfsCommand(cmd: String) extends Command +private[hive] case class DfsCommand(cmd: String) extends Command -case class ShellCommand(cmd: String) extends Command +private[hive] case class ShellCommand(cmd: String) extends Command -case class SourceCommand(filePath: String) extends Command +private[hive] case class SourceCommand(filePath: String) extends Command -case class AddJar(jarPath: String) extends Command +private[hive] case class AddJar(jarPath: String) extends Command -case class AddFile(filePath: String) extends Command +private[hive] case class AddFile(filePath: String) extends Command /** Provides a mapping from HiveQL statements to catalyst logical plans and expression trees. */ -object HiveQl { +private[hive] object HiveQl { protected val nativeCommands = Seq( "TOK_DESCFUNCTION", "TOK_DESCTABLE", 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 ac817b21a152e..d9a6e0e88932e 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 @@ -23,9 +23,10 @@ import org.apache.spark.sql.catalyst.planning._ import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution._ +import org.apache.spark.sql.hive.execution._ import org.apache.spark.sql.columnar.InMemoryColumnarTableScan -trait HiveStrategies { +private[hive] trait HiveStrategies { // Possibly being too clever with types here... or not clever enough. self: SQLContext#SparkPlanner => 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 2610100043659..610fa9cb843c0 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 @@ -15,23 +15,27 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.hive.execution import java.io.{BufferedReader, InputStreamReader} +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.execution._ +import org.apache.spark.sql.hive.HiveContext /* Implicit conversions */ import scala.collection.JavaConversions._ /** + * :: DeveloperApi :: * 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. */ +@DeveloperApi case class ScriptTransformation( input: Seq[Expression], script: String, diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index b1a26fdabbb85..74110ee27b76c 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.hive.test import java.io.File import java.util.{Set => JavaSet} @@ -34,6 +34,7 @@ import org.apache.spark.{SparkConf, SparkContext} import org.apache.spark.sql.catalyst.analysis._ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, NativeCommand} import org.apache.spark.sql.catalyst.util._ +import org.apache.spark.sql.hive._ /* Implicit conversions */ import scala.collection.JavaConversions._ 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 821fb22112f87..96faebc5a8687 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 @@ -15,7 +15,7 @@ * limitations under the License. */ -package org.apache.spark.sql.hive +package org.apache.spark.sql.hive.execution import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar} import org.apache.hadoop.hive.metastore.MetaStoreUtils @@ -30,23 +30,26 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive.JavaHiveVarcharOb import org.apache.hadoop.io.Writable import org.apache.hadoop.mapred._ +import org.apache.spark.annotation.DeveloperApi import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.types.{BooleanType, DataType} import org.apache.spark.sql.execution._ -import org.apache.spark.{SparkHiveHadoopWriter, TaskContext, SparkException} +import org.apache.spark.sql.hive._ +import org.apache.spark.{TaskContext, SparkException} /* Implicits */ import scala.collection.JavaConversions._ /** + * :: DeveloperApi :: * 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. */ +@DeveloperApi case class HiveTableScan( attributes: Seq[Attribute], relation: MetastoreRelation, @@ -160,6 +163,10 @@ case class HiveTableScan( def output = attributes } +/** + * :: DeveloperApi :: + */ +@DeveloperApi case class InsertIntoHiveTable( table: MetastoreRelation, partition: Map[String, Option[String]], 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 55a4363af6c76..a09270eb7b134 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 @@ -37,7 +37,7 @@ import org.apache.spark.sql.catalyst.types._ /* Implicit conversions */ import scala.collection.JavaConversions._ -object HiveFunctionRegistry +private[hive] object HiveFunctionRegistry extends analysis.FunctionRegistry with HiveFunctionFactory with HiveInspectors { def lookupFunction(name: String, children: Seq[Expression]): Expression = { @@ -99,7 +99,7 @@ object HiveFunctionRegistry } } -trait HiveFunctionFactory { +private[hive] trait HiveFunctionFactory { def getFunctionInfo(name: String) = FunctionRegistry.getFunctionInfo(name) def getFunctionClass(name: String) = getFunctionInfo(name).getFunctionClass def createFunction[UDFType](name: String) = @@ -130,7 +130,7 @@ trait HiveFunctionFactory { } } -abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { +private[hive] abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory { self: Product => type UDFType @@ -148,7 +148,7 @@ abstract class HiveUdf extends Expression with Logging with HiveFunctionFactory override def toString = s"$nodeName#${functionInfo.getDisplayName}(${children.mkString(",")})" } -case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { +private[hive] case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUdf { import org.apache.spark.sql.hive.HiveFunctionRegistry._ type UDFType = UDF @@ -201,7 +201,7 @@ case class HiveSimpleUdf(name: String, children: Seq[Expression]) extends HiveUd } } -case class HiveGenericUdf(name: String, children: Seq[Expression]) +private[hive] case class HiveGenericUdf(name: String, children: Seq[Expression]) extends HiveUdf with HiveInspectors { import org.apache.hadoop.hive.ql.udf.generic.GenericUDF._ @@ -228,7 +228,7 @@ case class HiveGenericUdf(name: String, children: Seq[Expression]) } } -trait HiveInspectors { +private[hive] trait HiveInspectors { def unwrapData(data: Any, oi: ObjectInspector): Any = oi match { case pi: PrimitiveObjectInspector => pi.getPrimitiveJavaObject(data) @@ -329,7 +329,7 @@ trait HiveInspectors { } } -case class HiveGenericUdaf( +private[hive] case class HiveGenericUdaf( name: String, children: Seq[Expression]) extends AggregateExpression with HiveInspectors @@ -371,7 +371,7 @@ case class HiveGenericUdaf( * 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( +private[hive] case class HiveGenericUdtf( name: String, aliasNames: Seq[String], children: Seq[Expression]) @@ -438,7 +438,7 @@ case class HiveGenericUdtf( override def toString = s"$nodeName#$name(${children.mkString(",")})" } -case class HiveUdafFunction( +private[hive] case class HiveUdafFunction( functionName: String, exprs: Seq[Expression], base: AggregateExpression) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala index 79ec1f1cde019..f9a162ef4e3c0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/CachedTableSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive import org.apache.spark.sql.execution.SparkLogicalPlan import org.apache.spark.sql.columnar.InMemoryColumnarTableScan import org.apache.spark.sql.hive.execution.HiveComparisonTest +import org.apache.spark.sql.hive.test.TestHive class CachedTableSuite extends HiveComparisonTest { TestHive.loadTestTable("src") diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala index ad29e06905c1b..833f3502154f3 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertIntoHiveTableSuite.scala @@ -17,12 +17,11 @@ package org.apache.spark.sql.hive -import java.io.File - import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.hive.test.TestHive /* Implicits */ -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ case class TestData(key: Int, value: String) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala index 8137f99b227f4..9c5d7c81f7c09 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/api/java/JavaHiveSuite.scala @@ -21,7 +21,7 @@ import org.scalatest.FunSuite import org.apache.spark.api.java.JavaSparkContext import org.apache.spark.sql.test.TestSQLContext -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.test.TestHive // Implicits import scala.collection.JavaConversions._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 8488f23abd0fd..9b9a823b6e95b 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution import java.io.File -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ /** * A set of test cases based on the big-data-benchmark. 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 index ac87f2cb10d12..23ece7e7cf6e9 100644 --- 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 @@ -18,7 +18,7 @@ package org.apache.spark.sql.hive.execution import org.apache.spark.{SparkConf, SparkContext} -import org.apache.spark.sql.hive.TestHiveContext +import org.apache.spark.sql.hive.test.TestHiveContext import org.scalatest.{BeforeAndAfterAll, FunSuite} class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll { 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 6c91f40d0f925..ea17e6e93b41c 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 @@ -24,7 +24,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeComman import org.apache.spark.sql.catalyst.util._ import org.apache.spark.sql.execution.Sort import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen} -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.test.TestHive /** * Allows the creations of tests that execute the same query against both hive diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index c3cfa3d25a5c2..dfe88b960b639 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.execution import org.scalatest.BeforeAndAfter -import org.apache.spark.sql.hive.TestHive +import org.apache.spark.sql.hive.test.TestHive /** * Runs the test cases that are included in the hive distribution. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala index a09667ac84b01..d224d2ee600ec 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ /** * A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution. diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala index 8883e5b16d4da..67594b57d3dfa 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala @@ -17,8 +17,8 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.TestHive -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ case class Data(a: Int, B: Int, n: Nested) case class Nested(a: Int, B: Int) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala index d9ccb93e23923..25eca397462f7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala @@ -17,7 +17,7 @@ package org.apache.spark.sql.hive.execution -import org.apache.spark.sql.hive.{HiveTableScan, TestHive} +import org.apache.spark.sql.hive.test.TestHive /* Implicit conversions */ import scala.collection.JavaConversions._ diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala index 843c681e0dc48..91ad59d7f82c0 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/parquet/HiveParquetSuite.scala @@ -22,11 +22,10 @@ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} import org.apache.spark.sql.catalyst.expressions.{AttributeReference, Row} import org.apache.spark.sql.catalyst.types.{DataType, StringType, IntegerType} import org.apache.spark.sql.{parquet, SchemaRDD} -import org.apache.spark.sql.hive.TestHive import org.apache.spark.util.Utils // Implicits -import org.apache.spark.sql.hive.TestHive._ +import org.apache.spark.sql.hive.test.TestHive._ class HiveParquetSuite extends FunSuite with BeforeAndAfterAll with BeforeAndAfterEach { From 8e950813334dd8545357f59f4ed77c39ee4180b3 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 22 Apr 2014 21:56:15 -0700 Subject: [PATCH 345/397] SPARK-1494 Don't initialize classes loaded by MIMA excludes. [WIP] Just seeing how Jenkins likes this... Author: Michael Armbrust Closes #494 from marmbrus/mima and squashes the following commits: 6eec616 [Michael Armbrust] Force hive tests to run. acaf682 [Michael Armbrust] Don't initialize loaded classes. --- .../scala/org/apache/spark/tools/GenerateMIMAIgnore.scala | 7 +++---- 1 file changed, 3 insertions(+), 4 deletions(-) diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index 3fb85e1ff73a8..d9978b02a9f3e 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -41,7 +41,7 @@ object GenerateMIMAIgnore { private def classesPrivateWithin(packageName: String): Set[String] = { - val classes = getClasses(packageName, classLoader) + val classes = getClasses(packageName) val privateClasses = mutable.HashSet[String]() def isPackagePrivate(className: String) = { @@ -108,8 +108,7 @@ object GenerateMIMAIgnore { * Scans all classes accessible from the context class loader which belong to the given package * and subpackages both from directories and jars present on the classpath. */ - private def getClasses(packageName: String, - classLoader: ClassLoader = Thread.currentThread().getContextClassLoader): Set[String] = { + private def getClasses(packageName: String): Set[String] = { val path = packageName.replace('.', '/') val resources = classLoader.getResources(path) @@ -128,7 +127,7 @@ object GenerateMIMAIgnore { val jar = new JarFile(new File(jarPath)) val enums = jar.entries().map(_.getName).filter(_.startsWith(packageName)) val classes = for (entry <- enums if entry.endsWith(".class")) - yield Class.forName(entry.replace('/', '.').stripSuffix(".class")) + yield Class.forName(entry.replace('/', '.').stripSuffix(".class"), false, classLoader) classes } } From 39f85e0322cfecefbc30e7d5a30356cfab1e9640 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Tue, 22 Apr 2014 22:19:32 -0700 Subject: [PATCH 346/397] [SQL] SPARK-1571 Mistake in java example code Author: Michael Armbrust Closes #496 from marmbrus/javaBeanBug and squashes the following commits: 644fedd [Michael Armbrust] Bean methods must be public. --- docs/sql-programming-guide.md | 8 ++++---- .../java/org/apache/spark/examples/sql/JavaSparkSQL.java | 8 ++++---- 2 files changed, 8 insertions(+), 8 deletions(-) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index e25379bd7614d..3ca6cc95bd2db 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -143,19 +143,19 @@ public static class Person implements Serializable { private String name; private int age; - String getName() { + public String getName() { return name; } - void setName(String name) { + public void setName(String name) { this.name = name; } - int getAge() { + public int getAge() { return age; } - void setAge(int age) { + public void setAge(int age) { this.age = age; } } diff --git a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java index e8e63d2745692..b5b438e9753a7 100644 --- a/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java +++ b/examples/src/main/java/org/apache/spark/examples/sql/JavaSparkSQL.java @@ -34,19 +34,19 @@ public static class Person implements Serializable { private String name; private int age; - String getName() { + public String getName() { return name; } - void setName(String name) { + public void setName(String name) { this.name = name; } - int getAge() { + public int getAge() { return age; } - void setAge(int age) { + public void setAge(int age) { this.age = age; } } From cd4ed293262e2349794c13467d1737974385c019 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Wed, 23 Apr 2014 10:19:32 -0700 Subject: [PATCH 347/397] SPARK-1119 and other build improvements 1. Makes assembly and examples jar naming consistent in maven/sbt. 2. Updates make-distribution.sh to use Maven and fixes some bugs. 3. Updates the create-release script to call make-distribution script. Author: Patrick Wendell Closes #502 from pwendell/make-distribution and squashes the following commits: 1a97f0d [Patrick Wendell] SPARK-1119 and other build improvements --- assembly/pom.xml | 2 +- bin/compute-classpath.sh | 4 +- bin/run-example | 11 +++-- dev/create-release/create-release.sh | 18 +++---- examples/pom.xml | 2 +- make-distribution.sh | 70 +++++++++++++++++++--------- project/SparkBuild.scala | 4 +- 7 files changed, 70 insertions(+), 41 deletions(-) diff --git a/assembly/pom.xml b/assembly/pom.xml index 923bf47f7076a..bdb38806492a6 100644 --- a/assembly/pom.xml +++ b/assembly/pom.xml @@ -33,7 +33,7 @@ scala-${scala.binary.version} - ${project.artifactId}-${project.version}-hadoop${hadoop.version}.jar + spark-assembly-${project.version}-hadoop${hadoop.version}.jar ${project.build.directory}/${spark.jar.dir}/${spark.jar.basename} spark /usr/share/spark diff --git a/bin/compute-classpath.sh b/bin/compute-classpath.sh index 2a2bb376fd71f..3a59f599fd7d2 100755 --- a/bin/compute-classpath.sh +++ b/bin/compute-classpath.sh @@ -50,9 +50,9 @@ if [ -f "$ASSEMBLY_DIR"/spark-assembly*hadoop*-deps.jar ]; then 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"/lib/spark-assembly*hadoop*.jar` else - ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark*-assembly*hadoop*.jar` + ASSEMBLY_JAR=`ls "$ASSEMBLY_DIR"/spark-assembly*hadoop*.jar` fi CLASSPATH="$CLASSPATH:$ASSEMBLY_JAR" fi diff --git a/bin/run-example b/bin/run-example index b2999198a8d41..d8a94f2e31e07 100755 --- a/bin/run-example +++ b/bin/run-example @@ -40,12 +40,15 @@ fi # Figure out the JAR file that our examples were packaged into. This includes a bit of a hack # to avoid the -sources and -doc packages that are built by publish-local. EXAMPLES_DIR="$FWDIR"/examples -SPARK_EXAMPLES_JAR="" -if [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar ]; then - export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/*assembly*[0-9Tg].jar` + +if [ -f "$FWDIR/RELEASE" ]; then + export SPARK_EXAMPLES_JAR=`ls "$FWDIR"/lib/spark-examples-*hadoop*.jar` +elif [ -e "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar ]; then + export SPARK_EXAMPLES_JAR=`ls "$EXAMPLES_DIR"/target/scala-$SCALA_VERSION/spark-examples-*hadoop*.jar` fi + if [[ -z $SPARK_EXAMPLES_JAR ]]; then - echo "Failed to find Spark examples assembly in $FWDIR/examples/target" >&2 + echo "Failed to find Spark examples assembly in $FWDIR/lib or $FWDIR/examples/target" >&2 echo "You need to build Spark with sbt/sbt assembly before running this program" >&2 exit 1 fi diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index fb9d9f9e072a6..ad38c8d53eb2d 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -83,15 +83,15 @@ rm -rf spark-$RELEASE_VERSION make_binary_release() { NAME=$1 - MAVEN_FLAGS=$2 - + FLAGS=$2 cp -r spark spark-$RELEASE_VERSION-bin-$NAME + cd spark-$RELEASE_VERSION-bin-$NAME - export MAVEN_OPTS="-Xmx3g -XX:MaxPermSize=1g -XX:ReservedCodeCacheSize=1g" - mvn $MAVEN_FLAGS -DskipTests clean package - find . -name test-classes -type d | xargs rm -rf - find . -name classes -type d | xargs rm -rf + ./make-distribution.sh $FLAGS --name $NAME --tgz cd .. + cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . + rm -rf spark-$RELEASE_VERSION-bin-$NAME + tar cvzf spark-$RELEASE_VERSION-bin-$NAME.tgz spark-$RELEASE_VERSION-bin-$NAME echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ @@ -105,9 +105,9 @@ make_binary_release() { rm -rf spark-$RELEASE_VERSION-bin-$NAME } -make_binary_release "hadoop1" "-Dhadoop.version=1.0.4" -make_binary_release "cdh4" "-Dhadoop.version=2.0.0-mr1-cdh4.2.0" -make_binary_release "hadoop2" "-Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0" +make_binary_release "hadoop1" "--hadoop 1.0.4" +make_binary_release "cdh4" "--hadoop 2.0.0-mr1-cdh4.2.0" +make_binary_release "hadoop2" "--with-yarn --hadoop 2.2.0" # Copy data echo "Copying release tarballs" diff --git a/examples/pom.xml b/examples/pom.xml index 0b6212b5d1549..704d6df7c57f9 100644 --- a/examples/pom.xml +++ b/examples/pom.xml @@ -187,7 +187,7 @@ maven-shade-plugin false - ${project.build.directory}/scala-${scala.binary.version}/${project.artifactId}-assembly-${project.version}.jar + ${project.build.directory}/scala-${scala.binary.version}/spark-examples-${project.version}-hadoop${hadoop.version}.jar *:* diff --git a/make-distribution.sh b/make-distribution.sh index 5c780fcbda863..83dfc745857ca 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -28,6 +28,8 @@ # --tgz: Additionally creates spark-$VERSION-bin.tar.gz # --hadoop VERSION: Builds against specified version of Hadoop. # --with-yarn: Enables support for Hadoop YARN. +# --with-hive: Enable support for reading Hive tables. +# --name: A moniker for the release target. Defaults to the Hadoop verison. # # Recommended deploy/testing procedure (standalone mode): # 1) Rsync / deploy the dist/ dir to one host @@ -41,25 +43,20 @@ FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -# Get version from SBT -export TERM=dumb # Prevents color codes in SBT output - -VERSIONSTRING=$($FWDIR/sbt/sbt "show version") - +VERSION=$(mvn help:evaluate -Dexpression=project.version |grep -v "INFO") if [ $? == -1 ] ;then - echo -e "You need sbt installed and available on your path." - echo -e "Download sbt from http://www.scala-sbt.org/" + echo -e "You need Maven installed to build Spark." + echo -e "Download Maven from https://maven.apache.org." exit -1; fi -VERSION=$(echo "${VERSIONSTRING}" | tail -1 | cut -f 2 | sed 's/^\([a-zA-Z0-9.-]*\).*/\1/') -echo "Version is ${VERSION}" - # Initialize defaults SPARK_HADOOP_VERSION=1.0.4 SPARK_YARN=false +SPARK_HIVE=false SPARK_TACHYON=false MAKE_TGZ=false +NAME=none # Parse arguments while (( "$#" )); do @@ -71,23 +68,37 @@ while (( "$#" )); do --with-yarn) SPARK_YARN=true ;; + --with-hive) + SPARK_HIVE=true + ;; --with-tachyon) SPARK_TACHYON=true ;; --tgz) MAKE_TGZ=true ;; + --name) + NAME="$2" + shift + ;; esac shift done +if [ "$NAME" == "none" ]; then + NAME=$SPARK_HADOOP_VERSION +fi + +echo "Spark version is $VERSION" + if [ "$MAKE_TGZ" == "true" ]; then - echo "Making spark-$VERSION-hadoop_$SPARK_HADOOP_VERSION-bin.tar.gz" + echo "Making spark-$VERSION-bin-$NAME.tgz" else - echo "Making distribution for Spark $VERSION in $DISTDIR..." + echo "Making distribution for Spark $VERSION in $DISTDIR..." fi echo "Hadoop version set to $SPARK_HADOOP_VERSION" +echo "Release name set to $NAME" if [ "$SPARK_YARN" == "true" ]; then echo "YARN enabled" else @@ -100,20 +111,32 @@ else echo "Tachyon Disabled" fi -# Build fat JAR -export SPARK_HADOOP_VERSION -export SPARK_YARN +# Build uber fat JAR cd $FWDIR -"sbt/sbt" "assembly/assembly" +export MAVEN_OPTS="-Xmx2g -XX:MaxPermSize=512M -XX:ReservedCodeCacheSize=512m" + +if [ "$SPARK_HIVE" == "true" ]; then + MAYBE_HIVE="-Phive" +else + MAYBE_HIVE="" +fi + +if [ "$SPARK_YARN" == "true" ]; then + mvn clean package -DskipTests -Pyarn -Dhadoop.version=$SPARK_HADOOP_VERSION \ + -Dyarn.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE +else + mvn clean package -DskipTests -Dhadoop.version=$SPARK_HADOOP_VERSION $MAYBE_HIVE +fi # Make directories rm -rf "$DISTDIR" -mkdir -p "$DISTDIR/jars" +mkdir -p "$DISTDIR/lib" echo "Spark $VERSION built for Hadoop $SPARK_HADOOP_VERSION" > "$DISTDIR/RELEASE" # Copy jars -cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/jars/" +cp $FWDIR/assembly/target/scala*/*assembly*hadoop*.jar "$DISTDIR/lib/" +cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" # Copy other things mkdir "$DISTDIR"/conf @@ -135,16 +158,16 @@ if [ "$SPARK_TACHYON" == "true" ]; then wget "$TACHYON_URL" tar xf "tachyon-${TACHYON_VERSION}-bin.tar.gz" - cp "tachyon-${TACHYON_VERSION}/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/jars" + cp "tachyon-${TACHYON_VERSION}/target/tachyon-${TACHYON_VERSION}-jar-with-dependencies.jar" "$DISTDIR/lib" mkdir -p "$DISTDIR/tachyon/src/main/java/tachyon/web" cp -r "tachyon-${TACHYON_VERSION}"/{bin,conf,libexec} "$DISTDIR/tachyon" cp -r "tachyon-${TACHYON_VERSION}"/src/main/java/tachyon/web/resources "$DISTDIR/tachyon/src/main/java/tachyon/web" if [[ `uname -a` == Darwin* ]]; then # need to run sed differently on osx - nl=$'\n'; sed -i "" -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\\$nl export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" + nl=$'\n'; sed -i "" -e "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\\$nl export TACHYON_JAR=\$TACHYON_HOME/../lib/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" else - sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../jars/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" + sed -i "s|export TACHYON_JAR=\$TACHYON_HOME/target/\(.*\)|# This is set for spark's make-distribution\n export TACHYON_JAR=\$TACHYON_HOME/../lib/\1|" "$DISTDIR/tachyon/libexec/tachyon-config.sh" fi popd > /dev/null @@ -152,8 +175,9 @@ if [ "$SPARK_TACHYON" == "true" ]; then fi if [ "$MAKE_TGZ" == "true" ]; then - TARDIR="$FWDIR/spark-$VERSION" + TARDIR_NAME=spark-$VERSION-bin-$NAME + TARDIR="$FWDIR/$TARDIR_NAME" cp -r "$DISTDIR" "$TARDIR" - tar -zcf "spark-$VERSION-hadoop_$SPARK_HADOOP_VERSION-bin.tar.gz" -C "$FWDIR" "spark-$VERSION" + tar czf "spark-$VERSION-bin-$NAME.tgz" -C "$FWDIR" "$TARDIR_NAME" rm -rf "$TARDIR" fi diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 852358501af56..b8af2bbd2ef6a 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -411,7 +411,9 @@ object SparkBuild extends Build { ) def examplesSettings = sharedSettings ++ Seq( - name := "spark-examples", + name := "spark-examples", + jarName in assembly <<= version map { + v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", "org.apache.hbase" % "hbase" % HBASE_VERSION excludeAll(excludeNetty, excludeAsm, excludeOldAsm, excludeCommonsLogging), From a66460661303bd652b5c5d32860216cc3ceb7f98 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 23 Apr 2014 14:12:20 -0700 Subject: [PATCH 348/397] SPARK-1583: Fix a bug that using java.util.HashMap by mistake JIRA: https://issues.apache.org/jira/browse/SPARK-1583 Does anyone know why using `java.util.HashMap` rather than `mutable.HashMap`? Some methods of `java.util.HashMap` are not generics and compiler can not help us find similar problems. Author: zsxwing Closes #500 from zsxwing/SPARK-1583 and squashes the following commits: 7bfd74d [zsxwing] SPARK-1583: Fix a bug that using java.util.HashMap by mistake --- .../org/apache/spark/storage/BlockManagerMasterActor.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala index c57b6e8391b13..63fa5d3eb6541 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManagerMasterActor.scala @@ -203,7 +203,7 @@ class BlockManagerMasterActor(val isLocal: Boolean, conf: SparkConf, listenerBus val locations = blockLocations.get(blockId) locations -= blockManagerId if (locations.size == 0) { - blockLocations.remove(locations) + blockLocations.remove(blockId) } } listenerBus.post(SparkListenerBlockManagerRemoved(blockManagerId)) From a967b005c8937a3053e215c952d2172ee3dc300d Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 23 Apr 2014 14:46:30 -0700 Subject: [PATCH 349/397] SPARK-1572 Don't kill Executor if PythonRDD fails while computing parent Previously, the behavior was that if the parent RDD threw any exception other than IOException or FileNotFoundException (which is quite possible for Hadoop input sources), the entire Executor would crash, because the default thread a uncaught exception handler calls System.exit(). This patch avoids two related issues: 1. Always catch exceptions in this reader thread. 2. Don't mask readerException when Python throws an EOFError after worker.shutdownOutput() is called. Author: Aaron Davidson Closes #486 from aarondav/pyspark and squashes the following commits: fbb11e9 [Aaron Davidson] Make sure FileNotFoundExceptions are handled same as before b9acb3e [Aaron Davidson] SPARK-1572 Don't kill Executor if PythonRDD fails while computing parent --- .../apache/spark/api/python/PythonRDD.scala | 32 ++++++++++++------- 1 file changed, 21 insertions(+), 11 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index f9d86fed34d0f..8a843fbb0e66f 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -24,6 +24,7 @@ import java.util.{List => JList, ArrayList => JArrayList, Map => JMap, Collectio import scala.collection.JavaConversions._ import scala.reflect.ClassTag +import scala.util.Try import net.razorvine.pickle.{Pickler, Unpickler} @@ -89,16 +90,22 @@ private[spark] class PythonRDD[T: ClassTag]( dataOut.flush() worker.shutdownOutput() } catch { + case e: java.io.FileNotFoundException => readerException = e - // Kill the Python worker process: - worker.shutdownOutput() + Try(worker.shutdownOutput()) // kill Python worker process + case e: IOException => // This can happen for legitimate reasons if the Python code stops returning data - // before we are done passing elements through, e.g., for take(). Just log a message - // to say it happened. - logInfo("stdin writer to Python finished early") - logDebug("stdin writer to Python finished early", e) + // before we are done passing elements through, e.g., for take(). Just log a message to + // say it happened (as it could also be hiding a real IOException from a data source). + logInfo("stdin writer to Python finished early (may not be an error)", e) + + case e: Exception => + // We must avoid throwing exceptions here, because the thread uncaught exception handler + // will kill the whole executor (see Executor). + readerException = e + Try(worker.shutdownOutput()) // kill Python worker process } } }.start() @@ -152,7 +159,7 @@ private[spark] class PythonRDD[T: ClassTag]( val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj)) + throw new PythonException(new String(obj), readerException) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still // read some accumulator updates: @@ -167,10 +174,13 @@ private[spark] class PythonRDD[T: ClassTag]( Array.empty[Byte] } } catch { - case eof: EOFException => { + case e: Exception if readerException != null => + logError("Python worker exited unexpectedly (crashed)", e) + logError("Python crash may have been caused by prior exception:", readerException) + throw readerException + + case eof: EOFException => throw new SparkException("Python worker exited unexpectedly (crashed)", eof) - } - case e: Throwable => throw e } } @@ -185,7 +195,7 @@ private[spark] class PythonRDD[T: ClassTag]( } /** Thrown for exceptions in user Python code. */ -private class PythonException(msg: String) extends Exception(msg) +private class PythonException(msg: String, cause: Exception) extends RuntimeException(msg, cause) /** * Form an RDD[(Array[Byte], Array[Byte])] from key-value pairs returned from Python. From dd1b7a61d9193c93ab95ab550622259f4bc26f53 Mon Sep 17 00:00:00 2001 From: Marcelo Vanzin Date: Wed, 23 Apr 2014 14:47:38 -0700 Subject: [PATCH 350/397] Honor default fs name when initializing event logger. This is related to SPARK-1459 / PR #375. Without this fix, FileLogger.createLogDir() may try to create the log dir on HDFS, while createWriter() will try to open the log file on the local file system, leading to interesting errors and confusion. Author: Marcelo Vanzin Closes #450 from vanzin/event-file-2 and squashes the following commits: 592cdb3 [Marcelo Vanzin] Honor default fs name when initializing event logger. --- .../scala/org/apache/spark/SparkContext.scala | 48 +++++++++---------- .../scheduler/EventLoggingListener.scala | 9 +++- .../org/apache/spark/util/FileLogger.scala | 17 ++++--- 3 files changed, 41 insertions(+), 33 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index 25ca650a3a37e..c14dce8273bc1 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -216,10 +216,33 @@ class SparkContext(config: SparkConf) extends Logging { private[spark] val ui = new SparkUI(this) ui.bind() + /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ + val hadoopConfiguration: Configuration = { + val env = SparkEnv.get + val hadoopConf = SparkHadoopUtil.get.newConfiguration() + // Explicitly check for S3 environment variables + if (System.getenv("AWS_ACCESS_KEY_ID") != null && + System.getenv("AWS_SECRET_ACCESS_KEY") != null) { + hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) + hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) + } + // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" + conf.getAll.foreach { case (key, value) => + if (key.startsWith("spark.hadoop.")) { + hadoopConf.set(key.substring("spark.hadoop.".length), value) + } + } + val bufferSize = conf.get("spark.buffer.size", "65536") + hadoopConf.set("io.file.buffer.size", bufferSize) + hadoopConf + } + // Optionally log Spark events private[spark] val eventLogger: Option[EventLoggingListener] = { if (conf.getBoolean("spark.eventLog.enabled", false)) { - val logger = new EventLoggingListener(appName, conf) + val logger = new EventLoggingListener(appName, conf, hadoopConfiguration) logger.start() listenerBus.addListener(logger) Some(logger) @@ -294,29 +317,6 @@ class SparkContext(config: SparkConf) extends Logging { postEnvironmentUpdate() postApplicationStart() - /** A default Hadoop Configuration for the Hadoop code (e.g. file systems) that we reuse. */ - val hadoopConfiguration: Configuration = { - val env = SparkEnv.get - val hadoopConf = SparkHadoopUtil.get.newConfiguration() - // Explicitly check for S3 environment variables - if (System.getenv("AWS_ACCESS_KEY_ID") != null && - System.getenv("AWS_SECRET_ACCESS_KEY") != null) { - hadoopConf.set("fs.s3.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - hadoopConf.set("fs.s3n.awsAccessKeyId", System.getenv("AWS_ACCESS_KEY_ID")) - hadoopConf.set("fs.s3.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - hadoopConf.set("fs.s3n.awsSecretAccessKey", System.getenv("AWS_SECRET_ACCESS_KEY")) - } - // Copy any "spark.hadoop.foo=bar" system properties into conf as "foo=bar" - conf.getAll.foreach { case (key, value) => - if (key.startsWith("spark.hadoop.")) { - hadoopConf.set(key.substring("spark.hadoop.".length), value) - } - } - val bufferSize = conf.get("spark.buffer.size", "65536") - hadoopConf.set("io.file.buffer.size", bufferSize) - hadoopConf - } - private[spark] var checkpointDir: Option[String] = None // Thread Local variable that can be used by users to pass information down the stack diff --git a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala index b983c16af14f4..2fe65cd944b67 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/EventLoggingListener.scala @@ -19,6 +19,7 @@ package org.apache.spark.scheduler import scala.collection.mutable +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileSystem, Path} import org.json4s.jackson.JsonMethods._ @@ -36,7 +37,10 @@ import org.apache.spark.util.{FileLogger, JsonProtocol} * spark.eventLog.dir - Path to the directory in which events are logged. * spark.eventLog.buffer.kb - Buffer size to use when writing to output streams */ -private[spark] class EventLoggingListener(appName: String, conf: SparkConf) +private[spark] class EventLoggingListener( + appName: String, + conf: SparkConf, + hadoopConfiguration: Configuration) extends SparkListener with Logging { import EventLoggingListener._ @@ -49,7 +53,8 @@ private[spark] class EventLoggingListener(appName: String, conf: SparkConf) val logDir = logBaseDir + "/" + name private val logger = - new FileLogger(logDir, conf, outputBufferSize, shouldCompress, shouldOverwrite) + new FileLogger(logDir, conf, hadoopConfiguration, outputBufferSize, shouldCompress, + shouldOverwrite) /** * Begin logging events. diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 7d58d1c765180..7d47b2a72aff7 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -22,7 +22,8 @@ import java.net.URI import java.text.SimpleDateFormat import java.util.Date -import org.apache.hadoop.fs.{FSDataOutputStream, Path} +import org.apache.hadoop.conf.Configuration +import org.apache.hadoop.fs.{FileSystem, FSDataOutputStream, Path} import org.apache.spark.{Logging, SparkConf} import org.apache.spark.io.CompressionCodec @@ -37,7 +38,8 @@ import org.apache.spark.io.CompressionCodec */ private[spark] class FileLogger( logDir: String, - conf: SparkConf = new SparkConf, + conf: SparkConf, + hadoopConfiguration: Configuration, outputBufferSize: Int = 8 * 1024, // 8 KB compress: Boolean = false, overwrite: Boolean = true) @@ -85,19 +87,20 @@ private[spark] class FileLogger( private def createWriter(fileName: String): PrintWriter = { val logPath = logDir + "/" + fileName val uri = new URI(logPath) + val defaultFs = FileSystem.getDefaultUri(hadoopConfiguration).getScheme + val isDefaultLocal = (defaultFs == null || defaultFs == "file") /* The Hadoop LocalFileSystem (r1.0.4) has known issues with syncing (HADOOP-7844). * Therefore, for local files, use FileOutputStream instead. */ - val dstream = uri.getScheme match { - case "file" | null => + val dstream = + if ((isDefaultLocal && uri.getScheme == null) || uri.getScheme == "file") { // Second parameter is whether to append new FileOutputStream(uri.getPath, !overwrite) - - case _ => + } else { val path = new Path(logPath) hadoopDataStream = Some(fileSystem.create(path, overwrite)) hadoopDataStream.get - } + } val bstream = new BufferedOutputStream(dstream, outputBufferSize) val cstream = if (compress) compressionCodec.compressedOutputStream(bstream) else bstream From 432201c7ee9e1ea1d70a6418cbad1c5ad2653ed3 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 23 Apr 2014 16:52:49 -0700 Subject: [PATCH 351/397] SPARK-1582 Invoke Thread.interrupt() when cancelling jobs Sometimes executor threads are blocked waiting for IO or monitors, and the current implementation of job cancellation may never recover these threads. By simply invoking Thread.interrupt() during cancellation, we can often safely unblock the threads and use them for subsequent work. Note that this feature must remain optional for now because of a bug in HDFS where Thread.interrupt() may cause nodes to be marked as permanently dead (as the InterruptedException is reinterpreted as an IOException during communication with some node). Author: Aaron Davidson Closes #498 from aarondav/cancel and squashes the following commits: e52b829 [Aaron Davidson] Don't use job.properties when null 82f78bb [Aaron Davidson] Update DAGSchedulerSuite b67f472 [Aaron Davidson] Add comment on why interruptOnCancel is in setJobGroup 4cb9fd6 [Aaron Davidson] SPARK-1582 Invoke Thread.interrupt() when cancelling jobs --- .../scala/org/apache/spark/SparkContext.scala | 15 ++++++++++++++- .../org/apache/spark/api/python/PythonRDD.scala | 1 - .../executor/CoarseGrainedExecutorBackend.scala | 4 ++-- .../org/apache/spark/executor/Executor.scala | 10 +++++----- .../spark/executor/MesosExecutorBackend.scala | 3 ++- .../org/apache/spark/scheduler/DAGScheduler.scala | 6 +++++- .../apache/spark/scheduler/SchedulerBackend.scala | 3 ++- .../scala/org/apache/spark/scheduler/Task.scala | 12 ++++++++++-- .../apache/spark/scheduler/TaskScheduler.scala | 2 +- .../spark/scheduler/TaskSchedulerImpl.scala | 4 ++-- .../org/apache/spark/scheduler/TaskSet.scala | 4 ++-- .../cluster/CoarseGrainedClusterMessage.scala | 3 ++- .../cluster/CoarseGrainedSchedulerBackend.scala | 8 ++++---- .../spark/scheduler/local/LocalBackend.scala | 10 +++++----- .../spark/scheduler/DAGSchedulerSuite.scala | 2 +- 15 files changed, 57 insertions(+), 30 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index c14dce8273bc1..dcb6b6824b0a6 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -381,16 +381,27 @@ class SparkContext(config: SparkConf) extends Logging { * // In a separate thread: * sc.cancelJobGroup("some_job_to_cancel") * }}} + * + * If interruptOnCancel is set to true for the job group, then job cancellation will result + * in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + * that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + * where HDFS may respond to Thread.interrupt() by marking nodes as dead. */ - def setJobGroup(groupId: String, description: String) { + def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean = false) { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, description) setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, groupId) + // Note: Specifying interruptOnCancel in setJobGroup (rather than cancelJobGroup) avoids + // changing several public APIs and allows Spark cancellations outside of the cancelJobGroup + // APIs to also take advantage of this property (e.g., internal job failures or canceling from + // JobProgressTab UI) on a per-job basis. + setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, interruptOnCancel.toString) } /** Clear the current thread's job group ID and its description. */ def clearJobGroup() { setLocalProperty(SparkContext.SPARK_JOB_DESCRIPTION, null) setLocalProperty(SparkContext.SPARK_JOB_GROUP_ID, null) + setLocalProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, null) } // Post init @@ -1244,6 +1255,8 @@ object SparkContext extends Logging { private[spark] val SPARK_JOB_GROUP_ID = "spark.jobGroup.id" + private[spark] val SPARK_JOB_INTERRUPT_ON_CANCEL = "spark.job.interruptOnCancel" + private[spark] val SPARK_UNKNOWN_USER = "" implicit object DoubleAccumulatorParam extends AccumulatorParam[Double] { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 8a843fbb0e66f..0d71fdbb03ec6 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -169,7 +169,6 @@ private[spark] class PythonRDD[T: ClassTag]( val update = new Array[Byte](updateLen) stream.readFully(update) accumulator += Collections.singletonList(update) - } Array.empty[Byte] } diff --git a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala index 6327ac01663f6..9ac7365f47f9f 100644 --- a/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala @@ -69,12 +69,12 @@ private[spark] class CoarseGrainedExecutorBackend( executor.launchTask(this, taskDesc.taskId, taskDesc.serializedTask) } - case KillTask(taskId, _) => + case KillTask(taskId, _, interruptThread) => if (executor == null) { logError("Received KillTask command but executor was null") System.exit(1) } else { - executor.killTask(taskId) + executor.killTask(taskId, interruptThread) } case x: DisassociatedEvent => diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 2bfb9c387e1c9..914bc205cebe2 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -136,10 +136,10 @@ private[spark] class Executor( threadPool.execute(tr) } - def killTask(taskId: Long) { + def killTask(taskId: Long, interruptThread: Boolean) { val tr = runningTasks.get(taskId) if (tr != null) { - tr.kill() + tr.kill(interruptThread) } } @@ -166,11 +166,11 @@ private[spark] class Executor( @volatile private var killed = false @volatile private var task: Task[Any] = _ - def kill() { + def kill(interruptThread: Boolean) { logInfo("Executor is trying to kill task " + taskId) killed = true if (task != null) { - task.kill() + task.kill(interruptThread) } } @@ -257,7 +257,7 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) } - case TaskKilledException => { + case TaskKilledException | _: InterruptedException if task.killed => { logInfo("Executor killed task " + taskId) execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) } diff --git a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala index 6fc702fdb1512..64e24506e8038 100644 --- a/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala +++ b/core/src/main/scala/org/apache/spark/executor/MesosExecutorBackend.scala @@ -76,7 +76,8 @@ private[spark] class MesosExecutorBackend if (executor == null) { logError("Received KillTask but executor was null") } else { - executor.killTask(t.getValue.toLong) + // TODO: Determine the 'interruptOnCancel' property set for the given job. + executor.killTask(t.getValue.toLong, interruptThread = false) } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index c6cbf14e20069..dbde9b591dccc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -1055,6 +1055,10 @@ class DAGScheduler( val error = new SparkException(failureReason) job.listener.jobFailed(error) + val shouldInterruptThread = + if (job.properties == null) false + else job.properties.getProperty(SparkContext.SPARK_JOB_INTERRUPT_ON_CANCEL, "false").toBoolean + // Cancel all independent, running stages. val stages = jobIdToStageIds(job.jobId) if (stages.isEmpty) { @@ -1073,7 +1077,7 @@ class DAGScheduler( // This is the only job that uses this stage, so fail the stage if it is running. val stage = stageIdToStage(stageId) if (runningStages.contains(stage)) { - taskScheduler.cancelTasks(stageId) + taskScheduler.cancelTasks(stageId, shouldInterruptThread) val stageInfo = stageToInfos(stage) stageInfo.stageFailed(failureReason) listenerBus.post(SparkListenerStageCompleted(stageToInfos(stage))) diff --git a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala index f1924a4573b21..6a6d8e609bc39 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/SchedulerBackend.scala @@ -28,5 +28,6 @@ private[spark] trait SchedulerBackend { def reviveOffers(): Unit def defaultParallelism(): Int - def killTask(taskId: Long, executorId: String): Unit = throw new UnsupportedOperationException + def killTask(taskId: Long, executorId: String, interruptThread: Boolean): Unit = + throw new UnsupportedOperationException } diff --git a/core/src/main/scala/org/apache/spark/scheduler/Task.scala b/core/src/main/scala/org/apache/spark/scheduler/Task.scala index a8bcb7dfe2f3c..2ca3479c80efc 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/Task.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/Task.scala @@ -44,8 +44,9 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex final def run(attemptId: Long): T = { context = new TaskContext(stageId, partitionId, attemptId, runningLocally = false) + taskThread = Thread.currentThread() if (_killed) { - kill() + kill(interruptThread = false) } runTask(context) } @@ -62,6 +63,9 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex // Task context, to be initialized in run(). @transient protected var context: TaskContext = _ + // The actual Thread on which the task is running, if any. Initialized in run(). + @volatile @transient private var taskThread: Thread = _ + // A flag to indicate whether the task is killed. This is used in case context is not yet // initialized when kill() is invoked. @volatile @transient private var _killed = false @@ -75,12 +79,16 @@ private[spark] abstract class Task[T](val stageId: Int, var partitionId: Int) ex * Kills a task by setting the interrupted flag to true. This relies on the upper level Spark * code and user code to properly handle the flag. This function should be idempotent so it can * be called multiple times. + * If interruptThread is true, we will also call Thread.interrupt() on the Task's executor thread. */ - def kill() { + def kill(interruptThread: Boolean) { _killed = true if (context != null) { context.interrupted = true } + if (interruptThread && taskThread != null) { + taskThread.interrupt() + } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala index 92616c997e20c..819c35257b5a7 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskScheduler.scala @@ -47,7 +47,7 @@ private[spark] trait TaskScheduler { def submitTasks(taskSet: TaskSet): Unit // Cancel a stage. - def cancelTasks(stageId: Int) + def cancelTasks(stageId: Int, interruptThread: Boolean) // Set the DAG scheduler for upcalls. This is guaranteed to be set before submitTasks is called. def setDAGScheduler(dagScheduler: DAGScheduler): Unit diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index fe72ab3e43146..be19d9b8854c8 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -170,7 +170,7 @@ private[spark] class TaskSchedulerImpl( backend.reviveOffers() } - override def cancelTasks(stageId: Int): Unit = synchronized { + override def cancelTasks(stageId: Int, interruptThread: Boolean): Unit = synchronized { logInfo("Cancelling stage " + stageId) activeTaskSets.find(_._2.stageId == stageId).foreach { case (_, tsm) => // There are two possible cases here: @@ -181,7 +181,7 @@ private[spark] class TaskSchedulerImpl( // simply abort the stage. tsm.runningTasksSet.foreach { tid => val execId = taskIdToExecutorId(tid) - backend.killTask(tid, execId) + backend.killTask(tid, execId, interruptThread) } tsm.abort("Stage %s cancelled".format(stageId)) logInfo("Stage %d was cancelled".format(stageId)) diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala index 03bf76083761f..613fa7850bb25 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSet.scala @@ -31,8 +31,8 @@ private[spark] class TaskSet( val properties: Properties) { val id: String = stageId + "." + attempt - def kill() { - tasks.foreach(_.kill()) + def kill(interruptThread: Boolean) { + tasks.foreach(_.kill(interruptThread)) } override def toString: String = "TaskSet " + id diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala index 4a9a1659d8254..ddbc74e82ac49 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedClusterMessage.scala @@ -30,7 +30,8 @@ private[spark] object CoarseGrainedClusterMessages { // Driver to executors case class LaunchTask(task: TaskDescription) extends CoarseGrainedClusterMessage - case class KillTask(taskId: Long, executor: String) extends CoarseGrainedClusterMessage + case class KillTask(taskId: Long, executor: String, interruptThread: Boolean) + extends CoarseGrainedClusterMessage case class RegisteredExecutor(sparkProperties: Seq[(String, String)]) extends CoarseGrainedClusterMessage diff --git a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala index 7bfc30b4208a3..a6d6b3d26a3c6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/cluster/CoarseGrainedSchedulerBackend.scala @@ -101,8 +101,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A case ReviveOffers => makeOffers() - case KillTask(taskId, executorId) => - executorActor(executorId) ! KillTask(taskId, executorId) + case KillTask(taskId, executorId, interruptThread) => + executorActor(executorId) ! KillTask(taskId, executorId, interruptThread) case StopDriver => sender ! true @@ -207,8 +207,8 @@ class CoarseGrainedSchedulerBackend(scheduler: TaskSchedulerImpl, actorSystem: A driverActor ! ReviveOffers } - override def killTask(taskId: Long, executorId: String) { - driverActor ! KillTask(taskId, executorId) + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { + driverActor ! KillTask(taskId, executorId, interruptThread) } override def defaultParallelism(): Int = { diff --git a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala index 16e2f5cf3076d..43f0e18a0cbe0 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/local/LocalBackend.scala @@ -30,7 +30,7 @@ private case class ReviveOffers() private case class StatusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) -private case class KillTask(taskId: Long) +private case class KillTask(taskId: Long, interruptThread: Boolean) /** * Calls to LocalBackend are all serialized through LocalActor. Using an actor makes the calls on @@ -61,8 +61,8 @@ private[spark] class LocalActor( reviveOffers() } - case KillTask(taskId) => - executor.killTask(taskId) + case KillTask(taskId, interruptThread) => + executor.killTask(taskId, interruptThread) } def reviveOffers() { @@ -99,8 +99,8 @@ private[spark] class LocalBackend(scheduler: TaskSchedulerImpl, val totalCores: override def defaultParallelism() = totalCores - override def killTask(taskId: Long, executorId: String) { - localActor ! KillTask(taskId) + override def killTask(taskId: Long, executorId: String, interruptThread: Boolean) { + localActor ! KillTask(taskId, interruptThread) } override def statusUpdate(taskId: Long, state: TaskState, serializedData: ByteBuffer) { diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index 35a7ac9d049c2..ff69eb7e53f8e 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -58,7 +58,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont taskSet.tasks.foreach(_.epoch = mapOutputTracker.getEpoch) taskSets += taskSet } - override def cancelTasks(stageId: Int) { + override def cancelTasks(stageId: Int, interruptThread: Boolean) { cancelledStages += stageId } override def setDAGScheduler(dagScheduler: DAGScheduler) = {} From 640f9a0efefd42cff86aecd4878a3a57f5ae85fa Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Wed, 23 Apr 2014 17:03:54 -0700 Subject: [PATCH 352/397] [SPARK-1540] Add an optional Ordering parameter to PairRDDFunctions. In https://issues.apache.org/jira/browse/SPARK-1540 we'd like to look at Spark's API to see if we can take advantage of Comparable keys in more places, which will make external spilling more efficient. This PR is a first step towards that that shows how to pass an Ordering when available and still continue functioning otherwise. It does this using a new implicit parameter with a default value of null. The API is currently only in Scala -- in Java we'd have to add new versions of mapToPair and such that take a Comparator, or a new method to add a "type hint" to an RDD. We can address those later though. Unfortunately requiring all keys to be Comparable would not work without requiring RDDs in general to contain only Comparable types. The reason is that methods such as distinct() and intersection() do a shuffle, but should be usable on RDDs of any type. So ordering will have to remain an optimization for the types that can be ordered. I think this isn't a horrible outcome though because one of the nice things about Spark's API is that it works on objects of *any* type, without requiring you to specify a schema or implement Writable or stuff like that. Author: Matei Zaharia This patch had conflicts when merged, resolved by Committer: Reynold Xin Closes #487 from mateiz/ordered-keys and squashes the following commits: bd565f6 [Matei Zaharia] Pass an Ordering to only one version of groupBy because the Scala language spec doesn't allow having an optional parameter on all of them (this was only compiling in Scala 2.10 due to a bug). 4629965 [Matei Zaharia] Add tests for other versions of groupBy 3beae85 [Matei Zaharia] Added a test for implicit orderings 80b7a3b [Matei Zaharia] Add an optional Ordering parameter to PairRDDFunctions. --- .../scala/org/apache/spark/SparkContext.scala | 4 +- .../apache/spark/rdd/PairRDDFunctions.scala | 29 +++++----- .../main/scala/org/apache/spark/rdd/RDD.scala | 46 ++++++++------- .../spark/rdd/SequenceFileRDDFunctions.scala | 4 +- .../apache/spark/ImplicitOrderingSuite.scala | 57 +++++++++++++++++++ .../spark/streaming/StreamingContext.scala | 5 +- .../spark/streaming/dstream/DStream.scala | 10 ++-- .../dstream/PairDStreamFunctions.scala | 18 +++--- 8 files changed, 124 insertions(+), 49 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index dcb6b6824b0a6..e9d2f5757963d 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -1281,8 +1281,10 @@ object SparkContext extends Logging { // TODO: Add AccumulatorParams for other types, e.g. lists and strings - implicit def rddToPairRDDFunctions[K: ClassTag, V: ClassTag](rdd: RDD[(K, V)]) = + implicit def rddToPairRDDFunctions[K, V](rdd: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { new PairRDDFunctions(rdd) + } implicit def rddToAsyncRDDActions[T: ClassTag](rdd: RDD[T]) = new AsyncRDDActions(rdd) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index d250bef6aad0f..d2b9ee427656b 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -52,11 +52,12 @@ import org.apache.spark.util.SerializableHyperLogLog * Extra functions available on RDDs of (key, value) pairs through an implicit conversion. * Import `org.apache.spark.SparkContext._` at the top of your program to use these functions. */ -class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) +class PairRDDFunctions[K, V](self: RDD[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) extends Logging with SparkHadoopMapReduceUtil - with Serializable { - + with Serializable +{ /** * Generic function to combine the elements for each key using a custom set of aggregation * functions. Turns an RDD[(K, V)] into a result of type RDD[(K, C)], for a "combined type" C @@ -77,7 +78,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) mapSideCombine: Boolean = true, serializer: Serializer = null): RDD[(K, C)] = { require(mergeCombiners != null, "mergeCombiners must be defined") // required as of Spark 0.9.0 - if (getKeyClass().isArray) { + if (keyClass.isArray) { if (mapSideCombine) { throw new SparkException("Cannot use map-side combining with array keys.") } @@ -170,7 +171,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def reduceByKeyLocally(func: (V, V) => V): Map[K, V] = { - if (getKeyClass().isArray) { + if (keyClass.isArray) { throw new SparkException("reduceByKeyLocally() does not support array keys") } @@ -288,7 +289,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * Return a copy of the RDD partitioned using the specified partitioner. */ def partitionBy(partitioner: Partitioner): RDD[(K, V)] = { - if (getKeyClass().isArray && partitioner.isInstanceOf[HashPartitioner]) { + if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } if (self.partitioner == partitioner) self else new ShuffledRDD[K, V, (K, V)](self, partitioner) @@ -458,7 +459,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def cogroup[W](other: RDD[(K, W)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other), partitioner) @@ -473,7 +474,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def cogroup[W1, W2](other1: RDD[(K, W1)], other2: RDD[(K, W2)], partitioner: Partitioner) : RDD[(K, (Iterable[V], Iterable[W1], Iterable[W2]))] = { - if (partitioner.isInstanceOf[HashPartitioner] && getKeyClass().isArray) { + if (partitioner.isInstanceOf[HashPartitioner] && keyClass.isArray) { throw new SparkException("Default partitioner cannot partition array keys.") } val cg = new CoGroupedRDD[K](Seq(self, other1, other2), partitioner) @@ -573,7 +574,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * supporting the key and value types K and V in this RDD. */ def saveAsHadoopFile[F <: OutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) + saveAsHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -584,7 +585,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) def saveAsHadoopFile[F <: OutputFormat[K, V]]( path: String, codec: Class[_ <: CompressionCodec]) (implicit fm: ClassTag[F]) { val runtimeClass = fm.runtimeClass - saveAsHadoopFile(path, getKeyClass, getValueClass, runtimeClass.asInstanceOf[Class[F]], codec) + saveAsHadoopFile(path, keyClass, valueClass, runtimeClass.asInstanceOf[Class[F]], codec) } /** @@ -592,7 +593,7 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) * (mapreduce.OutputFormat) object supporting the key and value types K and V in this RDD. */ def saveAsNewAPIHadoopFile[F <: NewOutputFormat[K, V]](path: String)(implicit fm: ClassTag[F]) { - saveAsNewAPIHadoopFile(path, getKeyClass, getValueClass, fm.runtimeClass.asInstanceOf[Class[F]]) + saveAsNewAPIHadoopFile(path, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } /** @@ -782,7 +783,9 @@ class PairRDDFunctions[K: ClassTag, V: ClassTag](self: RDD[(K, V)]) */ def values: RDD[V] = self.map(_._2) - private[spark] def getKeyClass() = implicitly[ClassTag[K]].runtimeClass + private[spark] def keyClass: Class[_] = kt.runtimeClass + + private[spark] def valueClass: Class[_] = vt.runtimeClass - private[spark] def getValueClass() = implicitly[ClassTag[V]].runtimeClass + private[spark] def keyOrdering: Option[Ordering[K]] = Option(ord) } diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 596dcb84db7bf..6c897cc03b641 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -284,7 +284,7 @@ abstract class RDD[T: ClassTag]( /** * Return a new RDD containing the distinct elements in this RDD. */ - def distinct(numPartitions: Int): RDD[T] = + def distinct(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = map(x => (x, null)).reduceByKey((x, y) => x, numPartitions).map(_._1) /** @@ -301,7 +301,7 @@ abstract class RDD[T: ClassTag]( * If you are decreasing the number of partitions in this RDD, consider using `coalesce`, * which can avoid performing a shuffle. */ - def repartition(numPartitions: Int): RDD[T] = { + def repartition(numPartitions: Int)(implicit ord: Ordering[T] = null): RDD[T] = { coalesce(numPartitions, shuffle = true) } @@ -325,7 +325,8 @@ abstract class RDD[T: ClassTag]( * coalesce(1000, shuffle = true) will result in 1000 partitions with the * data distributed using a hash partitioner. */ - def coalesce(numPartitions: Int, shuffle: Boolean = false): RDD[T] = { + def coalesce(numPartitions: Int, shuffle: Boolean = false)(implicit ord: Ordering[T] = null) + : RDD[T] = { if (shuffle) { // include a shuffle step so that our upstream tasks are still distributed new CoalescedRDD( @@ -424,10 +425,11 @@ abstract class RDD[T: ClassTag]( * * Note that this method performs a shuffle internally. */ - def intersection(other: RDD[T]): RDD[T] = + def intersection(other: RDD[T]): RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null))) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return the intersection of this RDD and another one. The output will not contain any duplicate @@ -437,10 +439,12 @@ abstract class RDD[T: ClassTag]( * * @param partitioner Partitioner to use for the resulting RDD */ - def intersection(other: RDD[T], partitioner: Partitioner): RDD[T] = + def intersection(other: RDD[T], partitioner: Partitioner)(implicit ord: Ordering[T] = null) + : RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null)), partitioner) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return the intersection of this RDD and another one. The output will not contain any duplicate @@ -450,10 +454,11 @@ abstract class RDD[T: ClassTag]( * * @param numPartitions How many partitions to use in the resulting RDD */ - def intersection(other: RDD[T], numPartitions: Int): RDD[T] = + def intersection(other: RDD[T], numPartitions: Int): RDD[T] = { this.map(v => (v, null)).cogroup(other.map(v => (v, null)), new HashPartitioner(numPartitions)) .filter { case (_, (leftGroup, rightGroup)) => leftGroup.nonEmpty && rightGroup.nonEmpty } .keys + } /** * Return an RDD created by coalescing all elements within each partition into an array. @@ -467,22 +472,25 @@ abstract class RDD[T: ClassTag]( def cartesian[U: ClassTag](other: RDD[U]): RDD[(T, U)] = new CartesianRDD(sc, this, other) /** - * Return an RDD of grouped items. + * Return an RDD of grouped items. Each group consists of a key and a sequence of elements + * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K): RDD[(K, Iterable[T])] = + def groupBy[K](f: T => K)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy[K](f, defaultPartitioner(this)) /** * Return an RDD of grouped elements. Each group consists of a key and a sequence of elements * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, numPartitions: Int): RDD[(K, Iterable[T])] = + def groupBy[K](f: T => K, numPartitions: Int)(implicit kt: ClassTag[K]): RDD[(K, Iterable[T])] = groupBy(f, new HashPartitioner(numPartitions)) /** - * Return an RDD of grouped items. + * Return an RDD of grouped items. Each group consists of a key and a sequence of elements + * mapping to that key. */ - def groupBy[K: ClassTag](f: T => K, p: Partitioner): RDD[(K, Iterable[T])] = { + def groupBy[K](f: T => K, p: Partitioner)(implicit kt: ClassTag[K], ord: Ordering[K] = null) + : RDD[(K, Iterable[T])] = { val cleanF = sc.clean(f) this.map(t => (cleanF(t), t)).groupByKey(p) } @@ -739,7 +747,7 @@ abstract class RDD[T: ClassTag]( /** * Return an RDD with the elements from `this` that are not in `other`. */ - def subtract(other: RDD[T], p: Partitioner): RDD[T] = { + def subtract(other: RDD[T], p: Partitioner)(implicit ord: Ordering[T] = null): RDD[T] = { if (partitioner == Some(p)) { // Our partitioner knows how to handle T (which, since we have a partitioner, is // really (K, V)) so make a new Partitioner that will de-tuple our fake tuples @@ -847,7 +855,7 @@ abstract class RDD[T: ClassTag]( * Return the count of each unique value in this RDD as a map of (value, count) pairs. The final * combine step happens locally on the master, equivalent to running a single reduce task. */ - def countByValue(): Map[T, Long] = { + def countByValue()(implicit ord: Ordering[T] = null): Map[T, Long] = { if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValue() does not support arrays") } @@ -877,10 +885,10 @@ abstract class RDD[T: ClassTag]( * Approximate version of countByValue(). */ @Experimental - def countByValueApprox( - timeout: Long, - confidence: Double = 0.95 - ): PartialResult[Map[T, BoundedDouble]] = { + def countByValueApprox(timeout: Long, confidence: Double = 0.95) + (implicit ord: Ordering[T] = null) + : PartialResult[Map[T, BoundedDouble]] = + { if (elementClassTag.runtimeClass.isArray) { throw new SparkException("countByValueApprox() does not support arrays") } @@ -1030,13 +1038,13 @@ abstract class RDD[T: ClassTag]( * Returns the max of this RDD as defined by the implicit Ordering[T]. * @return the maximum element of the RDD * */ - def max()(implicit ord: Ordering[T]):T = this.reduce(ord.max) + def max()(implicit ord: Ordering[T]): T = this.reduce(ord.max) /** * Returns the min of this RDD as defined by the implicit Ordering[T]. * @return the minimum element of the RDD * */ - def min()(implicit ord: Ordering[T]):T = this.reduce(ord.min) + def min()(implicit ord: Ordering[T]): T = this.reduce(ord.min) /** * Save this RDD as a text file, using string representations of elements. diff --git a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala index 7df9a2960d8a5..9a1efc83cbe6a 100644 --- a/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/SequenceFileRDDFunctions.scala @@ -68,8 +68,8 @@ class SequenceFileRDDFunctions[K <% Writable: ClassTag, V <% Writable : ClassTag val keyClass = getWritableClass[K] val valueClass = getWritableClass[V] - val convertKey = !classOf[Writable].isAssignableFrom(self.getKeyClass) - val convertValue = !classOf[Writable].isAssignableFrom(self.getValueClass) + val convertKey = !classOf[Writable].isAssignableFrom(self.keyClass) + val convertValue = !classOf[Writable].isAssignableFrom(self.valueClass) logInfo("Saving as sequence file of type (" + keyClass.getSimpleName + "," + valueClass.getSimpleName + ")" ) diff --git a/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala new file mode 100644 index 0000000000000..4bd889135631b --- /dev/null +++ b/core/src/test/scala/org/apache/spark/ImplicitOrderingSuite.scala @@ -0,0 +1,57 @@ +/* + * 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 + +import org.scalatest.FunSuite + +import org.apache.spark.SparkContext._ + +class ImplicitOrderingSuite extends FunSuite with LocalSparkContext { + class NonOrderedClass {} + + class ComparableClass extends Comparable[ComparableClass] { + override def compareTo(o: ComparableClass): Int = ??? + } + + class OrderedClass extends Ordered[OrderedClass] { + override def compare(o: OrderedClass): Int = ??? + } + + // Tests that PairRDDFunctions grabs an implicit Ordering in various cases where it should. + test("basic inference of Orderings"){ + sc = new SparkContext("local", "test") + val rdd = sc.parallelize(1 to 10) + + // Infer orderings after basic maps to particular types + assert(rdd.map(x => (x, x)).keyOrdering.isDefined) + assert(rdd.map(x => (1, x)).keyOrdering.isDefined) + assert(rdd.map(x => (x.toString, x)).keyOrdering.isDefined) + assert(rdd.map(x => (null, x)).keyOrdering.isDefined) + assert(rdd.map(x => (new NonOrderedClass, x)).keyOrdering.isEmpty) + assert(rdd.map(x => (new ComparableClass, x)).keyOrdering.isDefined) + assert(rdd.map(x => (new OrderedClass, x)).keyOrdering.isDefined) + + // Infer orderings for other RDD methods + assert(rdd.groupBy(x => x).keyOrdering.isDefined) + assert(rdd.groupBy(x => new NonOrderedClass).keyOrdering.isEmpty) + assert(rdd.groupBy(x => new ComparableClass).keyOrdering.isDefined) + assert(rdd.groupBy(x => new OrderedClass).keyOrdering.isDefined) + assert(rdd.groupBy((x: Int) => x, 5).keyOrdering.isDefined) + assert(rdd.groupBy((x: Int) => x, new HashPartitioner(5)).keyOrdering.isDefined) + } +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 9ba6e02229aaa..1c89543058211 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -495,7 +495,10 @@ class StreamingContext private[streaming] ( object StreamingContext extends Logging { - implicit def toPairDStreamFunctions[K: ClassTag, V: ClassTag](stream: DStream[(K,V)]) = { + private[streaming] val DEFAULT_CLEANER_TTL = 3600 + + implicit def toPairDStreamFunctions[K, V](stream: DStream[(K, V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K] = null) = { new PairDStreamFunctions[K, V](stream) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index a7e5215437e54..d393cc03cb33e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -488,7 +488,8 @@ abstract class DStream[T: ClassTag] ( * the RDDs with `numPartitions` partitions (Spark's default number of partitions if * `numPartitions` not specified). */ - def countByValue(numPartitions: Int = ssc.sc.defaultParallelism): DStream[(T, Long)] = + def countByValue(numPartitions: Int = ssc.sc.defaultParallelism)(implicit ord: Ordering[T] = null) + : DStream[(T, Long)] = this.map(x => (x, 1L)).reduceByKey((x: Long, y: Long) => x + y, numPartitions) /** @@ -686,9 +687,10 @@ abstract class DStream[T: ClassTag] ( def countByValueAndWindow( windowDuration: Duration, slideDuration: Duration, - numPartitions: Int = ssc.sc.defaultParallelism - ): DStream[(T, Long)] = { - + numPartitions: Int = ssc.sc.defaultParallelism) + (implicit ord: Ordering[T] = null) + : DStream[(T, Long)] = + { this.map(x => (x, 1L)).reduceByKeyAndWindow( (x: Long, y: Long) => x + y, (x: Long, y: Long) => x - y, diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala index 354bc132dcdc0..826bf39e860e1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/PairDStreamFunctions.scala @@ -37,13 +37,13 @@ import org.apache.spark.streaming.{Time, Duration} * Import `org.apache.spark.streaming.StreamingContext._` at the top of your program to use * these functions. */ -class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) - extends Serializable { - +class PairDStreamFunctions[K, V](self: DStream[(K,V)]) + (implicit kt: ClassTag[K], vt: ClassTag[V], ord: Ordering[K]) + extends Serializable +{ private[streaming] def ssc = self.ssc - private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) - = { + private[streaming] def defaultPartitioner(numPartitions: Int = self.ssc.sc.defaultParallelism) = { new HashPartitioner(numPartitions) } @@ -576,7 +576,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsHadoopFiles(prefix, suffix, getKeyClass, getValueClass, + saveAsHadoopFiles(prefix, suffix, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } @@ -607,7 +607,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) prefix: String, suffix: String )(implicit fm: ClassTag[F]) { - saveAsNewAPIHadoopFiles(prefix, suffix, getKeyClass, getValueClass, + saveAsNewAPIHadoopFiles(prefix, suffix, keyClass, valueClass, fm.runtimeClass.asInstanceOf[Class[F]]) } @@ -630,7 +630,7 @@ class PairDStreamFunctions[K: ClassTag, V: ClassTag](self: DStream[(K,V)]) self.foreachRDD(saveFunc) } - private def getKeyClass() = implicitly[ClassTag[K]].runtimeClass + private def keyClass: Class[_] = kt.runtimeClass - private def getValueClass() = implicitly[ClassTag[V]].runtimeClass + private def valueClass: Class[_] = vt.runtimeClass } From 4b2bab1d08a6b790be94717bbdd643d896d85c16 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Wed, 23 Apr 2014 21:59:33 -0700 Subject: [PATCH 353/397] [Hot Fix #469] Fix flaky test in SparkListenerSuite The two modified tests may fail if the race condition does not bid in our favor... Author: Andrew Or Closes #516 from andrewor14/stage-info-test-fix and squashes the following commits: b4b6100 [Andrew Or] Add/replace missing waitUntilEmpty() calls to listener bus --- .../org/apache/spark/scheduler/SparkListenerSuite.scala | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index ab139175e00eb..ba048ced32a93 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -50,9 +50,9 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc (1 to 5).foreach { _ => bus.post(SparkListenerJobEnd(0, JobSucceeded)) } assert(counter.count === 0) - // Starting listener bus should flush all buffered events (asynchronously, hence the sleep) + // Starting listener bus should flush all buffered events bus.start() - Thread.sleep(1000) + assert(bus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) assert(counter.count === 5) // After listener bus has stopped, posting events should not increment counter @@ -177,6 +177,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc listener.stageInfos.clear() rdd3.count() + assert(sc.listenerBus.waitUntilEmpty(WAIT_TIMEOUT_MILLIS)) listener.stageInfos.size should be {2} // Shuffle map stage + result stage val stageInfo3 = listener.stageInfos.keys.find(_.stageId == 2).get stageInfo3.rddInfos.size should be {2} // ShuffledRDD, MapPartitionsRDD From d485eecb7233dd339ae85a6f58f1c0686dd2037d Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Wed, 23 Apr 2014 22:00:22 -0700 Subject: [PATCH 354/397] Update Java api for setJobGroup with interruptOnCancel Also adds a unit test. Author: Aaron Davidson Closes #522 from aarondav/cancel2 and squashes the following commits: 565c253 [Aaron Davidson] Update Java api for setJobGroup with interruptOnCancel 65b33d8 [Aaron Davidson] Add unit test for Thread interruption on cancellation --- .../spark/api/java/JavaSparkContext.scala | 15 ++++++++ .../apache/spark/JobCancellationSuite.scala | 36 +++++++++++++++++-- 2 files changed, 49 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala index bda9272b43393..8b95cda511643 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaSparkContext.scala @@ -570,6 +570,21 @@ class JavaSparkContext(val sc: SparkContext) extends JavaSparkContextVarargsWork * // In a separate thread: * sc.cancelJobGroup("some_job_to_cancel"); * }}} + * + * If interruptOnCancel is set to true for the job group, then job cancellation will result + * in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + * that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + * where HDFS may respond to Thread.interrupt() by marking nodes as dead. + */ + def setJobGroup(groupId: String, description: String, interruptOnCancel: Boolean): Unit = + sc.setJobGroup(groupId, description, interruptOnCancel) + + /** + * Assigns a group ID to all the jobs started by this thread until the group ID is set to a + * different value or cleared. + * + * @see `setJobGroup(groupId: String, description: String, interruptThread: Boolean)`. + * This method sets interruptOnCancel to false. */ def setJobGroup(groupId: String, description: String): Unit = sc.setJobGroup(groupId, description) diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 7a39d1af9e2d5..16cfdf11c4a38 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -21,7 +21,7 @@ import java.util.concurrent.Semaphore import scala.concurrent.Await import scala.concurrent.ExecutionContext.Implicits.global -import scala.concurrent.duration.Duration +import scala.concurrent.duration._ import scala.concurrent.future import org.scalatest.{BeforeAndAfter, FunSuite} @@ -101,18 +101,50 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(10); i }.count() } + // Block until both tasks of job A have started and cancel job A. + sem.acquire(2) + sc.clearJobGroup() val jobB = sc.parallelize(1 to 100, 2).countAsync() + sc.cancelJobGroup("jobA") + val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + assert(e.getMessage contains "cancel") + + // Once A is cancelled, job B should finish fairly quickly. + assert(jobB.get() === 100) + } + + + test("job group with interruption") { + sc = new SparkContext("local[2]", "test") + + // Add a listener to release the semaphore once any tasks are launched. + val sem = new Semaphore(0) + sc.addSparkListener(new SparkListener { + override def onTaskStart(taskStart: SparkListenerTaskStart) { + sem.release() + } + }) + + // jobA is the one to be cancelled. + val jobA = future { + sc.setJobGroup("jobA", "this is a job to be cancelled", interruptOnCancel = true) + sc.parallelize(1 to 10000, 2).map { i => Thread.sleep(100000); i }.count() + } // Block until both tasks of job A have started and cancel job A. sem.acquire(2) + + sc.clearJobGroup() + val jobB = sc.parallelize(1 to 100, 2).countAsync() sc.cancelJobGroup("jobA") - val e = intercept[SparkException] { Await.result(jobA, Duration.Inf) } + val e = intercept[SparkException] { Await.result(jobA, 5.seconds) } assert(e.getMessage contains "cancel") // Once A is cancelled, job B should finish fairly quickly. assert(jobB.get() === 100) } + /* test("two jobs sharing the same stage") { // sem1: make sure cancel is issued after some tasks are launched From 1d6abe3a4b58f28fc4e0e690e02c19b2568ce1ee Mon Sep 17 00:00:00 2001 From: Ankur Dave Date: Wed, 23 Apr 2014 22:01:13 -0700 Subject: [PATCH 355/397] Mark all fields of EdgePartition, Graph, and GraphOps transient These classes are only serializable to work around closure capture, so their fields should all be marked `@transient` to avoid wasteful serialization. This PR supersedes apache/spark#519 and fixes the same bug. Author: Ankur Dave Closes #520 from ankurdave/graphx-transient and squashes the following commits: 6431760 [Ankur Dave] Mark all fields of EdgePartition, Graph, and GraphOps `@transient` --- .../src/main/scala/org/apache/spark/graphx/Graph.scala | 6 +++--- .../main/scala/org/apache/spark/graphx/GraphOps.scala | 10 +++++----- .../org/apache/spark/graphx/impl/EdgePartition.scala | 8 ++++---- 3 files changed, 12 insertions(+), 12 deletions(-) diff --git a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala index 45349692cbf6c..50395868902dc 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/Graph.scala @@ -46,7 +46,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * @note vertex ids are unique. * @return an RDD containing the vertices in this graph */ - val vertices: VertexRDD[VD] + @transient val vertices: VertexRDD[VD] /** * An RDD containing the edges and their associated attributes. The entries in the RDD contain @@ -59,7 +59,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * along with their vertex data. * */ - val edges: EdgeRDD[ED] + @transient val edges: EdgeRDD[ED] /** * An RDD containing the edge triplets, which are edges along with the vertex data associated with @@ -77,7 +77,7 @@ abstract class Graph[VD: ClassTag, ED: ClassTag] protected () extends Serializab * val numInvalid = graph.triplets.map(e => if (e.src.data == e.dst.data) 1 else 0).sum * }}} */ - val triplets: RDD[EdgeTriplet[VD, ED]] + @transient val triplets: RDD[EdgeTriplet[VD, ED]] /** * Caches the vertices and edges associated with this graph at the specified storage level. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala index 5635287694ee2..4997fbc3cbcd8 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/GraphOps.scala @@ -34,28 +34,28 @@ import scala.util.Random class GraphOps[VD: ClassTag, ED: ClassTag](graph: Graph[VD, ED]) extends Serializable { /** The number of edges in the graph. */ - lazy val numEdges: Long = graph.edges.count() + @transient lazy val numEdges: Long = graph.edges.count() /** The number of vertices in the graph. */ - lazy val numVertices: Long = graph.vertices.count() + @transient lazy val numVertices: Long = graph.vertices.count() /** * The in-degree of each vertex in the graph. * @note Vertices with no in-edges are not returned in the resulting RDD. */ - lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) + @transient lazy val inDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.In) /** * The out-degree of each vertex in the graph. * @note Vertices with no out-edges are not returned in the resulting RDD. */ - lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) + @transient lazy val outDegrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Out) /** * The degree of each vertex in the graph. * @note Vertices with no edges are not returned in the resulting RDD. */ - lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) + @transient lazy val degrees: VertexRDD[Int] = degreesRDD(EdgeDirection.Either) /** * Computes the neighboring vertex degrees. diff --git a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala index 2e05f5d4e4969..b7c472e905a9b 100644 --- a/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala +++ b/graphx/src/main/scala/org/apache/spark/graphx/impl/EdgePartition.scala @@ -34,10 +34,10 @@ import org.apache.spark.graphx.util.collection.PrimitiveKeyOpenHashMap */ private[graphx] class EdgePartition[@specialized(Char, Int, Boolean, Byte, Long, Float, Double) ED: ClassTag]( - val srcIds: Array[VertexId], - val dstIds: Array[VertexId], - val data: Array[ED], - val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable { + @transient val srcIds: Array[VertexId], + @transient val dstIds: Array[VertexId], + @transient val data: Array[ED], + @transient val index: PrimitiveKeyOpenHashMap[VertexId, Int]) extends Serializable { /** * Reverse all the edges in this partition. From 6ab7578067e3bb78b64f99fd67c97e9607050ffe Mon Sep 17 00:00:00 2001 From: zsxwing Date: Wed, 23 Apr 2014 22:36:02 -0700 Subject: [PATCH 356/397] SPARK-1589: Fix the incorrect compare JIRA: https://issues.apache.org/jira/browse/SPARK-1589 Author: zsxwing Closes #508 from zsxwing/SPARK-1589 and squashes the following commits: 570c67a [zsxwing] SPARK-1589: Fix the incorrect compare --- .../main/scala/org/apache/spark/rdd/PairRDDFunctions.scala | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala index d2b9ee427656b..5efb4388f6c71 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PairRDDFunctions.scala @@ -292,7 +292,11 @@ class PairRDDFunctions[K, V](self: RDD[(K, V)]) if (keyClass.isArray && partitioner.isInstanceOf[HashPartitioner]) { throw new SparkException("Default partitioner cannot partition array keys.") } - if (self.partitioner == partitioner) self else new ShuffledRDD[K, V, (K, V)](self, partitioner) + if (self.partitioner == Some(partitioner)) { + self + } else { + new ShuffledRDD[K, V, (K, V)](self, partitioner) + } } /** From bb68f47745eec2954814d3da277a672d5cf89980 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Wed, 23 Apr 2014 22:47:59 -0700 Subject: [PATCH 357/397] [Fix #79] Replace Breakable For Loops By While Loops Author: Sandeep Closes #503 from techaddict/fix-79 and squashes the following commits: e3f6746 [Sandeep] Style changes 07a4f6b [Sandeep] for loop to While loop 0a6d8e9 [Sandeep] Breakable for loop to While loop --- .../spark/mllib/tree/DecisionTree.scala | 60 ++++++++++--------- 1 file changed, 31 insertions(+), 29 deletions(-) diff --git a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala index 3019447ce4cd9..f68076f426259 100644 --- a/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala +++ b/mllib/src/main/scala/org/apache/spark/mllib/tree/DecisionTree.scala @@ -17,8 +17,6 @@ package org.apache.spark.mllib.tree -import scala.util.control.Breaks._ - import org.apache.spark.annotation.Experimental import org.apache.spark.{Logging, SparkContext} import org.apache.spark.SparkContext._ @@ -82,31 +80,34 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo * still survived the filters of the parent nodes. */ - // TODO: Convert for loop to while loop - breakable { - for (level <- 0 until maxDepth) { - - logDebug("#####################################") - logDebug("level = " + level) - logDebug("#####################################") - - // Find best split for all nodes at a level. - val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, - level, filters, splits, bins) - - for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { - // Extract info for nodes at the current level. - extractNodeInfo(nodeSplitStats, level, index, nodes) - // Extract info for nodes at the next lower level. - extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, - filters) - logDebug("final best split = " + nodeSplitStats._1) - } - require(scala.math.pow(2, level) == splitsStatsForLevel.length) - // Check whether all the nodes at the current level at leaves. - val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) - logDebug("all leaf = " + allLeaf) - if (allLeaf) break // no more tree construction + var level = 0 + var break = false + while (level < maxDepth && !break) { + + logDebug("#####################################") + logDebug("level = " + level) + logDebug("#####################################") + + // Find best split for all nodes at a level. + val splitsStatsForLevel = DecisionTree.findBestSplits(input, parentImpurities, strategy, + level, filters, splits, bins) + + for ((nodeSplitStats, index) <- splitsStatsForLevel.view.zipWithIndex) { + // Extract info for nodes at the current level. + extractNodeInfo(nodeSplitStats, level, index, nodes) + // Extract info for nodes at the next lower level. + extractInfoForLowerLevels(level, index, maxDepth, nodeSplitStats, parentImpurities, + filters) + logDebug("final best split = " + nodeSplitStats._1) + } + require(scala.math.pow(2, level) == splitsStatsForLevel.length) + // Check whether all the nodes at the current level at leaves. + val allLeaf = splitsStatsForLevel.forall(_._2.gain <= 0) + logDebug("all leaf = " + allLeaf) + if (allLeaf) { + break = true // no more tree construction + } else { + level += 1 } } @@ -146,8 +147,8 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo parentImpurities: Array[Double], filters: Array[List[Filter]]): Unit = { // 0 corresponds to the left child node and 1 corresponds to the right child node. - // TODO: Convert to while loop - for (i <- 0 to 1) { + var i = 0 + while (i <= 1) { // Calculate the index of the node from the node level and the index at the current level. val nodeIndex = scala.math.pow(2, level + 1).toInt - 1 + 2 * index + i if (level < maxDepth - 1) { @@ -166,6 +167,7 @@ class DecisionTree (private val strategy: Strategy) extends Serializable with Lo logDebug("Filter = " + filter) } } + i += 1 } } } From dd681f502eafe39cfb8a5a62ea2d28016ac6013d Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Wed, 23 Apr 2014 23:20:55 -0700 Subject: [PATCH 358/397] SPARK-1587 Fix thread leak mvn test fails (intermittently) due to thread leak - since scalatest runs all tests in same vm. Author: Mridul Muralidharan Closes #504 from mridulm/resource_leak_fixes and squashes the following commits: a5d10d0 [Mridul Muralidharan] Prevent thread leaks while running tests : cleanup all threads when SparkContext.stop is invoked. Causes tests to fail 7b5e19c [Mridul Muralidharan] Prevent NPE while running tests --- .../apache/spark/metrics/MetricsSystem.scala | 22 ++++++++------- .../spark/scheduler/TaskSchedulerImpl.scala | 1 + .../apache/spark/storage/BlockManager.scala | 2 ++ .../spark/storage/DiskBlockManager.scala | 28 +++++++++++-------- .../spark/storage/ShuffleBlockManager.scala | 4 +++ .../org/apache/spark/ui/JettyUtils.scala | 1 + .../spark/storage/DiskBlockManagerSuite.scala | 5 ++++ 7 files changed, 42 insertions(+), 21 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala index c5bda2078fc14..651511da1b7fe 100644 --- a/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala +++ b/core/src/main/scala/org/apache/spark/metrics/MetricsSystem.scala @@ -129,17 +129,19 @@ private[spark] class MetricsSystem private (val instance: String, sinkConfigs.foreach { kv => val classPath = kv._2.getProperty("class") - try { - val sink = Class.forName(classPath) - .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) - .newInstance(kv._2, registry, securityMgr) - if (kv._1 == "servlet") { - metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) - } else { - sinks += sink.asInstanceOf[Sink] + if (null != classPath) { + try { + val sink = Class.forName(classPath) + .getConstructor(classOf[Properties], classOf[MetricRegistry], classOf[SecurityManager]) + .newInstance(kv._2, registry, securityMgr) + if (kv._1 == "servlet") { + metricsServlet = Some(sink.asInstanceOf[MetricsServlet]) + } else { + sinks += sink.asInstanceOf[Sink] + } + } catch { + case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } - } catch { - case e: Exception => logError("Sink class " + classPath + " cannot be instantialized", e) } } } diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala index be19d9b8854c8..5a68f38bc5844 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSchedulerImpl.scala @@ -356,6 +356,7 @@ private[spark] class TaskSchedulerImpl( if (taskResultGetter != null) { taskResultGetter.stop() } + starvationTimer.cancel() // sleeping for an arbitrary 1 seconds to ensure that messages are sent out. Thread.sleep(1000L) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index f15fa4dd7ffd5..ccd5c5320abe5 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -1021,6 +1021,8 @@ private[spark] class BlockManager( heartBeatTask.cancel() } connectionManager.stop() + shuffleBlockManager.stop() + diskBlockManager.stop() actorSystem.stop(slaveActor) blockInfo.clear() memoryStore.clear() diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 7a24c8f57f43b..054f66a8b7260 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -150,20 +150,26 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") - localDirs.foreach { localDir => - try { - if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) - } catch { - case t: Throwable => - logError("Exception while deleting local spark dir: " + localDir, t) - } - } + stop() + } + }) + } - if (shuffleSender != null) { - shuffleSender.stop() + private[spark] def stop() { + localDirs.foreach { localDir => + if (localDir.isDirectory() && localDir.exists()) { + try { + if (!Utils.hasRootAsShutdownDeleteDir(localDir)) Utils.deleteRecursively(localDir) + } catch { + case t: Throwable => + logError("Exception while deleting local spark dir: " + localDir, t) } } - }) + } + + if (shuffleSender != null) { + shuffleSender.stop() + } } private[storage] def startShuffleBlockSender(port: Int): Int = { diff --git a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala index 4cd4cdbd9909d..35910e552fe86 100644 --- a/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/ShuffleBlockManager.scala @@ -207,6 +207,10 @@ class ShuffleBlockManager(blockManager: BlockManager) extends Logging { private def cleanup(cleanupTime: Long) { shuffleStates.clearOldValues(cleanupTime, (shuffleId, state) => removeShuffleBlocks(shuffleId)) } + + def stop() { + metadataCleaner.cancel() + } } private[spark] diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index 750f5a501c213..fdeb15b5d058a 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -195,6 +195,7 @@ private[spark] object JettyUtils extends Logging { (server, server.getConnectors.head.getLocalPort) case f: Failure[_] => server.stop() + pool.stop() logInfo("Failed to create UI at port, %s. Trying again.".format(currentPort)) logInfo("Error was: " + f.toString) connect((currentPort + 1) % 65536) diff --git a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala index 9b29e2a8a55de..42bfbf1bdfc74 100644 --- a/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala +++ b/core/src/test/scala/org/apache/spark/storage/DiskBlockManagerSuite.scala @@ -53,6 +53,11 @@ class DiskBlockManagerSuite extends FunSuite with BeforeAndAfterEach { shuffleBlockManager.idToSegmentMap.clear() } + override def afterEach() { + diskBlockManager.stop() + shuffleBlockManager.idToSegmentMap.clear() + } + test("basic block creation") { val blockId = new TestBlockId("test") assertSegmentEquals(blockId, blockId.name, 0, 0) From 1fdf659d2fdf23c5562e5dc646d05083062281ed Mon Sep 17 00:00:00 2001 From: Reynold Xin Date: Thu, 24 Apr 2014 00:27:45 -0700 Subject: [PATCH 359/397] SPARK-1601 & SPARK-1602: two bug fixes related to cancellation This should go into 1.0 since it would return wrong data when the bug happens (which is pretty likely if cancellation is used). Test case attached. 1. Do not put partially executed partitions into cache (in task killing). 2. Iterator returned by CacheManager#getOrCompute was not an InterruptibleIterator, and was thus leading to uninterruptible jobs. Thanks @aarondav and @ahirreddy for reporting and helping debug. Author: Reynold Xin Closes #521 from rxin/kill and squashes the following commits: 401033f [Reynold Xin] Merge branch 'master' of https://git-wip-us.apache.org/repos/asf/spark into kill 7a7bdd2 [Reynold Xin] Add a new line in the end of JobCancellationSuite.scala. 35cd9f7 [Reynold Xin] Fixed a bug that partially executed partitions can be put into cache (in task killing). --- .../scala/org/apache/spark/CacheManager.scala | 15 +++++-- .../apache/spark/InterruptibleIterator.scala | 12 +++++- .../apache/spark/TaskKilledException.scala | 23 ++++++++++ .../org/apache/spark/executor/Executor.scala | 8 ++-- .../apache/spark/JobCancellationSuite.scala | 43 ++++++++++++++++--- 5 files changed, 86 insertions(+), 15 deletions(-) create mode 100644 core/src/main/scala/org/apache/spark/TaskKilledException.scala diff --git a/core/src/main/scala/org/apache/spark/CacheManager.scala b/core/src/main/scala/org/apache/spark/CacheManager.scala index c7893f288b4b5..811610c657b62 100644 --- a/core/src/main/scala/org/apache/spark/CacheManager.scala +++ b/core/src/main/scala/org/apache/spark/CacheManager.scala @@ -47,7 +47,12 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { if (loading.contains(key)) { logInfo("Another thread is loading %s, waiting for it to finish...".format(key)) while (loading.contains(key)) { - try {loading.wait()} catch {case _ : Throwable =>} + try { + loading.wait() + } catch { + case e: Exception => + logWarning(s"Got an exception while waiting for another thread to load $key", e) + } } logInfo("Finished waiting for %s".format(key)) /* See whether someone else has successfully loaded it. The main way this would fail @@ -72,7 +77,9 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val computedValues = rdd.computeOrReadCheckpoint(split, context) // Persist the result, so long as the task is not running locally - if (context.runningLocally) { return computedValues } + if (context.runningLocally) { + return computedValues + } // Keep track of blocks with updated statuses var updatedBlocks = Seq[(BlockId, BlockStatus)]() @@ -88,7 +95,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { updatedBlocks = blockManager.put(key, computedValues, storageLevel, tellMaster = true) blockManager.get(key) match { case Some(values) => - new InterruptibleIterator(context, values.asInstanceOf[Iterator[T]]) + values.asInstanceOf[Iterator[T]] case None => logInfo("Failure to store %s".format(key)) throw new Exception("Block manager failed to return persisted valued") @@ -107,7 +114,7 @@ private[spark] class CacheManager(blockManager: BlockManager) extends Logging { val metrics = context.taskMetrics metrics.updatedBlocks = Some(updatedBlocks) - returnValue + new InterruptibleIterator(context, returnValue) } finally { loading.synchronized { diff --git a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala index fd1802ba2f984..ec11dbbffaaf8 100644 --- a/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala +++ b/core/src/main/scala/org/apache/spark/InterruptibleIterator.scala @@ -24,7 +24,17 @@ package org.apache.spark private[spark] class InterruptibleIterator[+T](val context: TaskContext, val delegate: Iterator[T]) extends Iterator[T] { - def hasNext: Boolean = !context.interrupted && delegate.hasNext + def hasNext: Boolean = { + // TODO(aarondav/rxin): Check Thread.interrupted instead of context.interrupted if interrupt + // is allowed. The assumption is that Thread.interrupted does not have a memory fence in read + // (just a volatile field in C), while context.interrupted is a volatile in the JVM, which + // introduces an expensive read fence. + if (context.interrupted) { + throw new TaskKilledException + } else { + delegate.hasNext + } + } def next(): T = delegate.next() } diff --git a/core/src/main/scala/org/apache/spark/TaskKilledException.scala b/core/src/main/scala/org/apache/spark/TaskKilledException.scala new file mode 100644 index 0000000000000..cbd6b2866e4f9 --- /dev/null +++ b/core/src/main/scala/org/apache/spark/TaskKilledException.scala @@ -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. + */ + +package org.apache.spark + +/** + * Exception for a task getting killed. + */ +private[spark] class TaskKilledException extends RuntimeException diff --git a/core/src/main/scala/org/apache/spark/executor/Executor.scala b/core/src/main/scala/org/apache/spark/executor/Executor.scala index 914bc205cebe2..272bcda5f8f2f 100644 --- a/core/src/main/scala/org/apache/spark/executor/Executor.scala +++ b/core/src/main/scala/org/apache/spark/executor/Executor.scala @@ -161,8 +161,6 @@ private[spark] class Executor( class TaskRunner(execBackend: ExecutorBackend, taskId: Long, serializedTask: ByteBuffer) extends Runnable { - object TaskKilledException extends Exception - @volatile private var killed = false @volatile private var task: Task[Any] = _ @@ -200,7 +198,7 @@ private[spark] class Executor( // causes a NonLocalReturnControl exception to be thrown. The NonLocalReturnControl // exception will be caught by the catch block, leading to an incorrect ExceptionFailure // for the task. - throw TaskKilledException + throw new TaskKilledException } attemptedTask = Some(task) @@ -214,7 +212,7 @@ private[spark] class Executor( // If the task has been killed, let's fail it. if (task.killed) { - throw TaskKilledException + throw new TaskKilledException } val resultSer = SparkEnv.get.serializer.newInstance() @@ -257,7 +255,7 @@ private[spark] class Executor( execBackend.statusUpdate(taskId, TaskState.FAILED, ser.serialize(reason)) } - case TaskKilledException | _: InterruptedException if task.killed => { + case _: TaskKilledException | _: InterruptedException if task.killed => { logInfo("Executor killed task " + taskId) execBackend.statusUpdate(taskId, TaskState.KILLED, ser.serialize(TaskKilled)) } diff --git a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala index 16cfdf11c4a38..2c8ef405c944c 100644 --- a/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala +++ b/core/src/test/scala/org/apache/spark/JobCancellationSuite.scala @@ -84,6 +84,35 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf assert(sc.parallelize(1 to 10, 2).count === 10) } + test("do not put partially executed partitions into cache") { + // In this test case, we create a scenario in which a partition is only partially executed, + // and make sure CacheManager does not put that partially executed partition into the + // BlockManager. + import JobCancellationSuite._ + sc = new SparkContext("local", "test") + + // Run from 1 to 10, and then block and wait for the task to be killed. + val rdd = sc.parallelize(1 to 1000, 2).map { x => + if (x > 10) { + taskStartedSemaphore.release() + taskCancelledSemaphore.acquire() + } + x + }.cache() + + val rdd1 = rdd.map(x => x) + + future { + taskStartedSemaphore.acquire() + sc.cancelAllJobs() + taskCancelledSemaphore.release(100000) + } + + intercept[SparkException] { rdd1.count() } + // If the partial block is put into cache, rdd.count() would return a number less than 1000. + assert(rdd.count() === 1000) + } + test("job group") { sc = new SparkContext("local[2]", "test") @@ -114,7 +143,6 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf assert(jobB.get() === 100) } - test("job group with interruption") { sc = new SparkContext("local[2]", "test") @@ -145,15 +173,14 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf assert(jobB.get() === 100) } -/* - test("two jobs sharing the same stage") { + ignore("two jobs sharing the same stage") { // sem1: make sure cancel is issued after some tasks are launched // sem2: make sure the first stage is not finished until cancel is issued val sem1 = new Semaphore(0) val sem2 = new Semaphore(0) sc = new SparkContext("local[2]", "test") - sc.dagScheduler.addSparkListener(new SparkListener { + sc.addSparkListener(new SparkListener { override def onTaskStart(taskStart: SparkListenerTaskStart) { sem1.release() } @@ -179,7 +206,7 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf intercept[SparkException] { f1.get() } intercept[SparkException] { f2.get() } } - */ + def testCount() { // Cancel before launching any tasks { @@ -238,3 +265,9 @@ class JobCancellationSuite extends FunSuite with ShouldMatchers with BeforeAndAf } } } + + +object JobCancellationSuite { + val taskStartedSemaphore = new Semaphore(0) + val taskCancelledSemaphore = new Semaphore(0) +} From 27b2821cf16948962c7a6f513621a1eba60b8cf3 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Thu, 24 Apr 2014 09:57:28 -0700 Subject: [PATCH 360/397] [SPARK-1610] [SQL] Fix Cast to use exact type value when cast from BooleanType to NumericTy... ...pe. `Cast` from `BooleanType` to `NumericType` are all using `Int` value. But it causes `ClassCastException` when the casted value is used by the following evaluation like the code below: ``` scala scala> import org.apache.spark.sql.catalyst._ import org.apache.spark.sql.catalyst._ scala> import types._ import types._ scala> import expressions._ import expressions._ scala> Add(Cast(Literal(true), ShortType), Literal(1.toShort)).eval() java.lang.ClassCastException: java.lang.Integer cannot be cast to java.lang.Short at scala.runtime.BoxesRunTime.unboxToShort(BoxesRunTime.java:102) at scala.math.Numeric$ShortIsIntegral$.plus(Numeric.scala:72) at org.apache.spark.sql.catalyst.expressions.Add$$anonfun$eval$2.apply(arithmetic.scala:58) at org.apache.spark.sql.catalyst.expressions.Add$$anonfun$eval$2.apply(arithmetic.scala:58) at org.apache.spark.sql.catalyst.expressions.Expression.n2(Expression.scala:114) at org.apache.spark.sql.catalyst.expressions.Add.eval(arithmetic.scala:58) at .(:17) at .() at .(:7) at .() at $print() at sun.reflect.NativeMethodAccessorImpl.invoke0(Native Method) at sun.reflect.NativeMethodAccessorImpl.invoke(NativeMethodAccessorImpl.java:62) at sun.reflect.DelegatingMethodAccessorImpl.invoke(DelegatingMethodAccessorImpl.java:43) at java.lang.reflect.Method.invoke(Method.java:483) at scala.tools.nsc.interpreter.IMain$ReadEvalPrint.call(IMain.scala:734) at scala.tools.nsc.interpreter.IMain$Request.loadAndRun(IMain.scala:983) at scala.tools.nsc.interpreter.IMain.loadAndRunReq$1(IMain.scala:573) at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:604) at scala.tools.nsc.interpreter.IMain.interpret(IMain.scala:568) at scala.tools.nsc.interpreter.ILoop.reallyInterpret$1(ILoop.scala:760) at scala.tools.nsc.interpreter.ILoop.interpretStartingWith(ILoop.scala:805) at scala.tools.nsc.interpreter.ILoop.command(ILoop.scala:717) at scala.tools.nsc.interpreter.ILoop.processLine$1(ILoop.scala:581) at scala.tools.nsc.interpreter.ILoop.innerLoop$1(ILoop.scala:588) at scala.tools.nsc.interpreter.ILoop.loop(ILoop.scala:591) at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply$mcZ$sp(ILoop.scala:882) at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:837) at scala.tools.nsc.interpreter.ILoop$$anonfun$process$1.apply(ILoop.scala:837) at scala.tools.nsc.util.ScalaClassLoader$.savingContextLoader(ScalaClassLoader.scala:135) at scala.tools.nsc.interpreter.ILoop.process(ILoop.scala:837) at scala.tools.nsc.MainGenericRunner.runTarget$1(MainGenericRunner.scala:83) at scala.tools.nsc.MainGenericRunner.process(MainGenericRunner.scala:96) at scala.tools.nsc.MainGenericRunner$.main(MainGenericRunner.scala:105) at scala.tools.nsc.MainGenericRunner.main(MainGenericRunner.scala) ``` Author: Takuya UESHIN Closes #533 from ueshin/issues/SPARK-1610 and squashes the following commits: 70f36e8 [Takuya UESHIN] Fix Cast to use exact type value when cast from BooleanType to NumericType. --- .../apache/spark/sql/catalyst/expressions/Cast.scala | 10 +++++----- .../expressions/ExpressionEvaluationSuite.scala | 7 +++++++ 2 files changed, 12 insertions(+), 5 deletions(-) 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 1f3fab09e9566..8b79b0cd65a84 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 @@ -111,7 +111,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toLong catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1L else 0L) case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t)) case DecimalType => nullOrCast[BigDecimal](_, _.toLong) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toLong(b) @@ -131,7 +131,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toShort catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1.toShort else 0.toShort) case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toShort) case DecimalType => nullOrCast[BigDecimal](_, _.toShort) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toShort @@ -141,7 +141,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toByte catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1.toByte else 0.toByte) case TimestampType => nullOrCast[Timestamp](_, t => timestampToLong(t).toByte) case DecimalType => nullOrCast[BigDecimal](_, _.toByte) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toInt(b).toByte @@ -162,7 +162,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toDouble catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1d else 0d) case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t)) case DecimalType => nullOrCast[BigDecimal](_, _.toDouble) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toDouble(b) @@ -172,7 +172,7 @@ case class Cast(child: Expression, dataType: DataType) extends UnaryExpression { case StringType => nullOrCast[String](_, s => try s.toFloat catch { case _: NumberFormatException => null }) - case BooleanType => nullOrCast[Boolean](_, b => if(b) 1 else 0) + case BooleanType => nullOrCast[Boolean](_, b => if(b) 1f else 0f) case TimestampType => nullOrCast[Timestamp](_, t => timestampToDouble(t).toFloat) case DecimalType => nullOrCast[BigDecimal](_, _.toFloat) case x: NumericType => b => x.numeric.asInstanceOf[Numeric[Any]].toFloat(b) diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 2cd0d2b0e1385..4ce0dff9e1586 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -237,6 +237,13 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation("2012-12-11" cast DoubleType, null) checkEvaluation(Literal(123) cast IntegerType, 123) + checkEvaluation(Literal(23d) + Cast(true, DoubleType), 24) + checkEvaluation(Literal(23) + Cast(true, IntegerType), 24) + checkEvaluation(Literal(23f) + Cast(true, FloatType), 24) + checkEvaluation(Literal(BigDecimal(23)) + Cast(true, DecimalType), 24) + checkEvaluation(Literal(23.toByte) + Cast(true, ByteType), 24) + checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24) + intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} } From faeb761cbe37521ec7ee6bbab7bcbc4bb1abb328 Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 24 Apr 2014 09:59:44 -0700 Subject: [PATCH 361/397] Small changes to release script --- dev/create-release/create-release.sh | 4 +--- make-distribution.sh | 1 + 2 files changed, 2 insertions(+), 3 deletions(-) diff --git a/dev/create-release/create-release.sh b/dev/create-release/create-release.sh index ad38c8d53eb2d..f1aa0f8f99e13 100755 --- a/dev/create-release/create-release.sh +++ b/dev/create-release/create-release.sh @@ -92,7 +92,6 @@ make_binary_release() { cp spark-$RELEASE_VERSION-bin-$NAME/spark-$RELEASE_VERSION-bin-$NAME.tgz . rm -rf spark-$RELEASE_VERSION-bin-$NAME - tar cvzf spark-$RELEASE_VERSION-bin-$NAME.tgz spark-$RELEASE_VERSION-bin-$NAME echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --armour \ --output spark-$RELEASE_VERSION-bin-$NAME.tgz.asc \ --detach-sig spark-$RELEASE_VERSION-bin-$NAME.tgz @@ -102,7 +101,6 @@ make_binary_release() { echo $GPG_PASSPHRASE | gpg --passphrase-fd 0 --print-md \ SHA512 spark-$RELEASE_VERSION-bin-$NAME.tgz > \ spark-$RELEASE_VERSION-bin-$NAME.tgz.sha - rm -rf spark-$RELEASE_VERSION-bin-$NAME } make_binary_release "hadoop1" "--hadoop 1.0.4" @@ -114,7 +112,7 @@ echo "Copying release tarballs" ssh $USER_NAME@people.apache.org \ mkdir /home/$USER_NAME/public_html/spark-$RELEASE_VERSION-$RC_NAME rc_folder=spark-$RELEASE_VERSION-$RC_NAME -scp spark* \ +scp spark-* \ $USER_NAME@people.apache.org:/home/$USER_NAME/public_html/$rc_folder/ # Docs diff --git a/make-distribution.sh b/make-distribution.sh index 83dfc745857ca..62a28217740cd 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -141,6 +141,7 @@ cp $FWDIR/examples/target/scala*/spark-examples*.jar "$DISTDIR/lib/" # Copy other things mkdir "$DISTDIR"/conf cp "$FWDIR"/conf/*.template "$DISTDIR"/conf +cp "$FWDIR"/conf/slaves "$DISTDIR"/conf cp -r "$FWDIR/bin" "$DISTDIR" cp -r "$FWDIR/python" "$DISTDIR" cp -r "$FWDIR/sbin" "$DISTDIR" From 6338a93f10e8b5f4365f71afd1086bdde64afad5 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 24 Apr 2014 10:06:18 -0700 Subject: [PATCH 362/397] SPARK-1488. Squash more language feature warnings in new commits by importing implicitConversion A recent commit reintroduced some of the same warnings that SPARK-1488 resolved. These are just a few more of the same changes to remove these warnings. Author: Sean Owen Closes #528 from srowen/SPARK-1488.2 and squashes the following commits: 62d592c [Sean Owen] More feature warnings in tests 4e2e94b [Sean Owen] Squash more language feature warnings in new commits by importing implicitConversion --- .../org/apache/spark/streaming/api/java/JavaInputDStream.scala | 1 + .../apache/spark/streaming/api/java/JavaPairInputDStream.scala | 2 ++ .../spark/streaming/api/java/JavaPairReceiverInputDStream.scala | 1 + .../spark/streaming/api/java/JavaReceiverInputDStream.scala | 1 + .../scala/org/apache/spark/streaming/NetworkReceiverSuite.scala | 1 + .../org/apache/spark/streaming/StreamingListenerSuite.scala | 1 + 6 files changed, 7 insertions(+) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala index 91f8d342d2bf9..ae5e693df93d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaInputDStream.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.api.java +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.InputDStream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala index add858530862b..e6ff8a0cb545f 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairInputDStream.scala @@ -18,6 +18,8 @@ package org.apache.spark.streaming.api.java import org.apache.spark.streaming.dstream.InputDStream + +import scala.language.implicitConversions import scala.reflect.ClassTag /** diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala index 974b3e451642d..294bc2be1898c 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaPairReceiverInputDStream.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.api.java +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.ReceiverInputDStream diff --git a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala index 340ef979808b9..8142dc59ea7c7 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/api/java/JavaReceiverInputDStream.scala @@ -17,6 +17,7 @@ package org.apache.spark.streaming.api.java +import scala.language.implicitConversions import scala.reflect.ClassTag import org.apache.spark.streaming.dstream.ReceiverInputDStream diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index 5c0415ad14ebb..45304c76b0928 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming import java.nio.ByteBuffer import scala.collection.mutable.ArrayBuffer +import scala.language.postfixOps import org.apache.spark.SparkConf import org.apache.spark.storage.{StorageLevel, StreamBlockId} diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 542c697ae3127..458dd3a2b13d8 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -20,6 +20,7 @@ package org.apache.spark.streaming import scala.collection.mutable.ArrayBuffer import scala.concurrent.Future import scala.concurrent.ExecutionContext.Implicits.global +import scala.language.postfixOps import org.apache.spark.storage.StorageLevel import org.apache.spark.streaming.dstream.DStream From 78a49b2532d4751257654dfe55a564bcd10701b3 Mon Sep 17 00:00:00 2001 From: zsxwing Date: Thu, 24 Apr 2014 11:13:40 -0700 Subject: [PATCH 363/397] SPARK-1611: Fix incorrect initialization order in AppendOnlyMap JIRA: https://issues.apache.org/jira/browse/SPARK-1611 Author: zsxwing Closes #534 from zsxwing/SPARK-1611 and squashes the following commits: 96af089 [zsxwing] SPARK-1611: Fix incorrect initialization order in AppendOnlyMap --- .../org/apache/spark/util/collection/AppendOnlyMap.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala index ad38250ad339f..1a6f1c2b55799 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/AppendOnlyMap.scala @@ -40,6 +40,8 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) require(initialCapacity <= (1 << 29), "Can't make capacity bigger than 2^29 elements") require(initialCapacity >= 1, "Invalid initial capacity") + private val LOAD_FACTOR = 0.7 + private var capacity = nextPowerOf2(initialCapacity) private var mask = capacity - 1 private var curSize = 0 @@ -57,8 +59,6 @@ class AppendOnlyMap[K, V](initialCapacity: Int = 64) private var destroyed = false private val destructionMessage = "Map state is invalid from destructive sorting!" - private val LOAD_FACTOR = 0.7 - /** Get the value for a given key */ def apply(key: K): V = { assert(!destroyed, destructionMessage) From bd375094a1480b0ff9c16ab8ddd2dba8731506df Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 24 Apr 2014 11:15:12 -0700 Subject: [PATCH 364/397] Spark 1490 Add kerberos support to the HistoryServer Here I've added the ability for the History server to login from a kerberos keytab file so that the history server can be run as a super user and stay up for along period of time while reading the history files from HDFS. Author: Thomas Graves Closes #513 from tgravescs/SPARK-1490 and squashes the following commits: e204a99 [Thomas Graves] remove extra logging 5418daa [Thomas Graves] fix typo in config 0076b99 [Thomas Graves] Update docs 4d76545 [Thomas Graves] SPARK-1490 Add kerberos support to the HistoryServer --- .../apache/spark/deploy/SparkHadoopUtil.scala | 4 ++++ .../spark/deploy/history/HistoryServer.scala | 16 +++++++++++++ docs/monitoring.md | 24 +++++++++++++++++++ 3 files changed, 44 insertions(+) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala index 9bdbfb33bf54f..498fcc520ac5e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkHadoopUtil.scala @@ -75,6 +75,10 @@ class SparkHadoopUtil { def getSecretKeyFromUserCredentials(key: String): Array[Byte] = { null } + def loginUserFromKeytab(principalName: String, keytabFilename: String) { + UserGroupInformation.loginUserFromKeytab(principalName, keytabFilename) + } + } object SparkHadoopUtil { diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index b8f56234d37ec..d7a3246bcfb47 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -22,6 +22,7 @@ import scala.collection.mutable import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.spark.{Logging, SecurityManager, SparkConf} +import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.scheduler._ import org.apache.spark.ui.{WebUI, SparkUI} import org.apache.spark.ui.JettyUtils._ @@ -257,6 +258,7 @@ object HistoryServer { val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR def main(argStrings: Array[String]) { + initSecurity() val args = new HistoryServerArguments(argStrings) val securityManager = new SecurityManager(conf) val server = new HistoryServer(args.logDir, securityManager, conf) @@ -266,6 +268,20 @@ object HistoryServer { while(true) { Thread.sleep(Int.MaxValue) } server.stop() } + + def initSecurity() { + // If we are accessing HDFS and it has security enabled (Kerberos), we have to login + // from a keytab file so that we can access HDFS beyond the kerberos ticket expiration. + // As long as it is using Hadoop rpc (hdfs://), a relogin will automatically + // occur from the keytab. + if (conf.getBoolean("spark.history.kerberos.enabled", false)) { + // if you have enabled kerberos the following 2 params must be set + val principalName = conf.get("spark.history.kerberos.principal") + val keytabFilename = conf.get("spark.history.kerberos.keytab") + SparkHadoopUtil.get.loginUserFromKeytab(principalName, keytabFilename) + } + } + } diff --git a/docs/monitoring.md b/docs/monitoring.md index 144be3daf1208..347a9b1f1a329 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -91,6 +91,30 @@ represents an application's event logs. This creates a web interface at The port to which the web interface of the history server binds. + + + + + + + + + + + + + + +
      Property NameDefaultMeaning
      spark.history.kerberos.enabledfalse + Indicates whether the history server should use kerberos to login. This is useful + if the history server is accessing HDFS files on a secure Hadoop cluster. If this is + true it looks uses the configs spark.history.kerberos.principal and + spark.history.kerberos.keytab. +
      spark.history.kerberos.principal(none) + Kerberos principal name for the History Server. +
      spark.history.kerberos.keytab(none) + Location of the kerberos keytab file for the History Server. +
      Note that in all of these UIs, the tables are sortable by clicking their headers, From c5c1916dd1b77e22759d58b5b361c56672983e3e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 24 Apr 2014 14:54:01 -0700 Subject: [PATCH 365/397] SPARK-1494 Don't initialize classes loaded by MIMA excludes, attempt 2 [WIP] Looks like scala reflection was invoking the static initializer: ``` ... at org.apache.spark.sql.test.TestSQLContext$.(TestSQLContext.scala:25) at org.apache.spark.sql.test.TestSQLContext$.(TestSQLContext.scala) at java.lang.Class.forName0(Native Method) at java.lang.Class.forName(Class.java:270) at scala.reflect.runtime.JavaMirrors$JavaMirror.javaClass(JavaMirrors.scala:500) at scala.reflect.runtime.JavaMirrors$JavaMirror.tryJavaClass(JavaMirrors.scala:505) at scala.reflect.runtime.SymbolLoaders$PackageScope.lookupEntry(SymbolLoaders.scala:109) ... ``` Need to make sure that this doesn't change the exclusion semantics before merging. Author: Michael Armbrust Closes #526 from marmbrus/mima and squashes the following commits: 8168dea [Michael Armbrust] Spurious change afba262 [Michael Armbrust] Prevent Scala reflection from running static class initializer. --- .../main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala index d9978b02a9f3e..a433e8e2e89f4 100644 --- a/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala +++ b/tools/src/main/scala/org/apache/spark/tools/GenerateMIMAIgnore.scala @@ -50,7 +50,7 @@ object GenerateMIMAIgnore { is a module or class. */ val privateAsClass = mirror - .staticClass(className) + .classSymbol(Class.forName(className, false, classLoader)) .privateWithin .fullName .startsWith(packageName) From a03ac222d84025a1036750e1179136a13f75dea7 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 24 Apr 2014 15:07:23 -0700 Subject: [PATCH 366/397] Fix Scala Style Any comments are welcome Author: Sandeep Closes #531 from techaddict/stylefix-1 and squashes the following commits: 7492730 [Sandeep] Pass 4 98b2428 [Sandeep] fix rxin suggestions b5e2e6f [Sandeep] Pass 3 05932d7 [Sandeep] fix if else styling 2 08690e5 [Sandeep] fix if else styling --- .../scala/org/apache/spark/Accumulators.scala | 7 +- .../spark/deploy/SparkSubmitArguments.scala | 3 +- .../apache/spark/deploy/master/Master.scala | 3 +- .../spark/deploy/worker/DriverRunner.scala | 8 ++- .../spark/deploy/worker/ui/LogPage.scala | 16 +++-- .../apache/spark/storage/BlockManager.scala | 8 ++- .../spark/util/BoundedPriorityQueue.scala | 12 ++-- .../org/apache/spark/util/FileLogger.scala | 4 +- .../scala/org/apache/spark/util/Utils.scala | 3 +- .../scala/org/apache/spark/rdd/RDDSuite.scala | 4 +- .../org/apache/spark/examples/LogQuery.scala | 3 +- .../examples/clickstream/PageViewStream.scala | 7 +- .../apache/spark/graphx/GraphOpsSuite.scala | 7 +- .../apache/spark/repl/SparkExprTyper.scala | 13 ++-- .../sql/parquet/ParquetTableOperations.scala | 5 +- .../sql/parquet/ParquetTableSupport.scala | 7 +- .../streaming/scheduler/ReceiverTracker.scala | 3 +- .../spark/deploy/yarn/ApplicationMaster.scala | 9 +-- .../spark/deploy/yarn/ExecutorLauncher.scala | 3 +- .../deploy/yarn/YarnAllocationHandler.scala | 67 ++++++++++--------- 20 files changed, 109 insertions(+), 83 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/Accumulators.scala b/core/src/main/scala/org/apache/spark/Accumulators.scala index d5f3e3f6ec496..6d652faae149a 100644 --- a/core/src/main/scala/org/apache/spark/Accumulators.scala +++ b/core/src/main/scala/org/apache/spark/Accumulators.scala @@ -104,8 +104,11 @@ class Accumulable[R, T] ( * Set the accumulator's value; only allowed on master. */ def value_= (newValue: R) { - if (!deserialized) value_ = newValue - else throw new UnsupportedOperationException("Can't assign accumulator value in task") + if (!deserialized) { + value_ = newValue + } else { + throw new UnsupportedOperationException("Can't assign accumulator value in task") + } } /** diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index cc976565cc72f..c3e8c6b8c65a9 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -66,8 +66,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { if (k.startsWith("spark")) { defaultProperties(k) = v if (verbose) SparkSubmit.printStream.println(s"Adding default property: $k=$v") - } - else { + } else { SparkSubmit.printWarning(s"Ignoring non-spark config property: $k=$v") } } diff --git a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala index 81f990bfa6513..fdb633bd33608 100644 --- a/core/src/main/scala/org/apache/spark/deploy/master/Master.scala +++ b/core/src/main/scala/org/apache/spark/deploy/master/Master.scala @@ -237,8 +237,7 @@ private[spark] class Master( if (waitingDrivers.contains(d)) { waitingDrivers -= d self ! DriverStateChanged(driverId, DriverState.KILLED, None) - } - else { + } else { // We just notify the worker to kill the driver here. The final bookkeeping occurs // on the return path when the worker submits a state change back to the master // to notify it that the driver was successfully killed. diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala index f918b42c83bc6..662d37871e7a6 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/DriverRunner.scala @@ -91,9 +91,11 @@ private[spark] class DriverRunner( } val state = - if (killed) { DriverState.KILLED } - else if (finalException.isDefined) { DriverState.ERROR } - else { + if (killed) { + DriverState.KILLED + } else if (finalException.isDefined) { + DriverState.ERROR + } else { finalExitCode match { case Some(0) => DriverState.FINISHED case _ => DriverState.FAILED diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala index fec1207948628..8381f59672ea3 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ui/LogPage.scala @@ -89,8 +89,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { Previous {Utils.bytesToString(math.min(byteLength, startByte))} - } - else { + } else { @@ -104,8 +103,7 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { Next {Utils.bytesToString(math.min(byteLength, logLength - endByte))} - } - else { + } else { @@ -137,9 +135,13 @@ private[spark] class LogPage(parent: WorkerWebUI) extends WebUIPage("logPage") { val logLength = file.length() val getOffset = offset.getOrElse(logLength - defaultBytes) val startByte = - if (getOffset < 0) 0L - else if (getOffset > logLength) logLength - else getOffset + if (getOffset < 0) { + 0L + } else if (getOffset > logLength) { + logLength + } else { + getOffset + } val logPageLength = math.min(byteLength, maxBytes) val endByte = math.min(startByte + logPageLength, logLength) (startByte, endByte) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala index ccd5c5320abe5..02ba5ecf52459 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockManager.scala @@ -281,7 +281,9 @@ private[spark] class BlockManager( val onDiskSize = status.diskSize master.updateBlockInfo( blockManagerId, blockId, storageLevel, inMemSize, onDiskSize, inTachyonSize) - } else true + } else { + true + } } /** @@ -676,7 +678,7 @@ private[spark] class BlockManager( tachyonStore.putValues(blockId, iterator, level, false) case ArrayBufferValues(array) => tachyonStore.putValues(blockId, array, level, false) - case ByteBufferValues(bytes) => + case ByteBufferValues(bytes) => bytes.rewind() tachyonStore.putBytes(blockId, bytes, level) } @@ -695,7 +697,7 @@ private[spark] class BlockManager( diskStore.putValues(blockId, iterator, level, askForBytes) case ArrayBufferValues(array) => diskStore.putValues(blockId, array, level, askForBytes) - case ByteBufferValues(bytes) => + case ByteBufferValues(bytes) => bytes.rewind() diskStore.putBytes(blockId, bytes, level) } diff --git a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala index b9f4a5d720b93..1b2b1932e0c3d 100644 --- a/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala +++ b/core/src/main/scala/org/apache/spark/util/BoundedPriorityQueue.scala @@ -43,8 +43,11 @@ private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Orderin } override def +=(elem: A): this.type = { - if (size < maxSize) underlying.offer(elem) - else maybeReplaceLowest(elem) + if (size < maxSize) { + underlying.offer(elem) + } else { + maybeReplaceLowest(elem) + } this } @@ -59,7 +62,8 @@ private[spark] class BoundedPriorityQueue[A](maxSize: Int)(implicit ord: Orderin if (head != null && ord.gt(a, head)) { underlying.poll() underlying.offer(a) - } else false + } else { + false + } } } - diff --git a/core/src/main/scala/org/apache/spark/util/FileLogger.scala b/core/src/main/scala/org/apache/spark/util/FileLogger.scala index 7d47b2a72aff7..1ed3b70bb24fd 100644 --- a/core/src/main/scala/org/apache/spark/util/FileLogger.scala +++ b/core/src/main/scala/org/apache/spark/util/FileLogger.scala @@ -113,7 +113,9 @@ private[spark] class FileLogger( * @param withTime Whether to prepend message with a timestamp */ def log(msg: String, withTime: Boolean = false) { - val writeInfo = if (!withTime) msg else { + val writeInfo = if (!withTime) { + msg + } else { val date = new Date(System.currentTimeMillis()) dateFormat.get.format(date) + ": " + msg } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index a3af4e7b91692..d333e2a88c18c 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -811,8 +811,7 @@ private[spark] object Utils extends Logging { } else { el.getMethodName } - } - else { + } else { firstUserLine = el.getLineNumber firstUserFile = el.getFileName firstUserClass = el.getClassName diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index d7c90346d88a2..2676558bfc86d 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -381,8 +381,8 @@ class RDDSuite extends FunSuite with SharedSparkContext { val prng42 = new Random(42) val prng43 = new Random(43) Array(1, 2, 3, 4, 5, 6).filter{i => - if (i < 4) 0 == prng42.nextInt(3) - else 0 == prng43.nextInt(3)} + if (i < 4) 0 == prng42.nextInt(3) else 0 == prng43.nextInt(3) + } } assert(sample.size === checkSample.size) for (i <- 0 until sample.size) assert(sample(i) === checkSample(i)) diff --git a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala index 820e87d04f3fa..f77a444ff7a9f 100644 --- a/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala +++ b/examples/src/main/scala/org/apache/spark/examples/LogQuery.scala @@ -49,8 +49,7 @@ object LogQuery { System.getenv("SPARK_HOME"), SparkContext.jarOfClass(this.getClass).toSeq) val dataSet = - if (args.length == 2) sc.textFile(args(1)) - else sc.parallelize(exampleApacheLogs) + if (args.length == 2) sc.textFile(args(1)) else sc.parallelize(exampleApacheLogs) // scalastyle:off val apacheLogRegex = """^([\d.]+) (\S+) (\S+) \[([\w\d:/]+\s[+\-]\d{4})\] "(.+?)" (\d{3}) ([\d\-]+) "([^"]+)" "([^"]+)".*""".r diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala index edc769c59734a..673013f7cf948 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/clickstream/PageViewStream.scala @@ -69,8 +69,11 @@ object PageViewStream { val normalCount = statuses.filter(_ == 200).size val errorCount = statuses.size - normalCount val errorRatio = errorCount.toFloat / statuses.size - if (errorRatio > 0.05) {"%s: **%s**".format(zip, errorRatio)} - else {"%s: %s".format(zip, errorRatio)} + if (errorRatio > 0.05) { + "%s: **%s**".format(zip, errorRatio) + } else { + "%s: %s".format(zip, errorRatio) + } } // Return the number unique users in last 15 seconds diff --git a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala index a467ca1ae715a..ea94d4accb63b 100644 --- a/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala +++ b/graphx/src/test/scala/org/apache/spark/graphx/GraphOpsSuite.scala @@ -165,8 +165,11 @@ class GraphOpsSuite extends FunSuite with LocalSparkContext { // not have any edges in the specified direction. assert(edges.count === 50) edges.collect.foreach { - case (vid, edges) => if (vid > 0 && vid < 49) assert(edges.size == 2) - else assert(edges.size == 1) + case (vid, edges) => if (vid > 0 && vid < 49) { + assert(edges.size == 2) + } else { + assert(edges.size == 1) + } } edges.collect.foreach { case (vid, edges) => diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala index dcc139544e2c2..f8432c8af6ed2 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkExprTyper.scala @@ -47,9 +47,13 @@ trait SparkExprTyper extends Logging { var isIncomplete = false reporter.withIncompleteHandler((_, _) => isIncomplete = true) { val trees = codeParser.stmts(line) - if (reporter.hasErrors) Some(Nil) - else if (isIncomplete) None - else Some(trees) + if (reporter.hasErrors) { + Some(Nil) + } else if (isIncomplete) { + None + } else { + Some(trees) + } } } // def parsesAsExpr(line: String) = { @@ -70,8 +74,7 @@ trait SparkExprTyper extends Logging { val sym0 = symbolOfTerm(name) // drop NullaryMethodType val sym = sym0.cloneSymbol setInfo afterTyper(sym0.info.finalResultType) - if (sym.info.typeSymbol eq UnitClass) NoSymbol - else sym + if (sym.info.typeSymbol eq UnitClass) NoSymbol else sym case _ => NoSymbol } } 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 d5846baa72ada..f825ca3c028ef 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 @@ -203,8 +203,9 @@ case class InsertIntoParquetTable( val stageId = sc.newRddId() val taskIdOffset = - if (overwrite) 1 - else { + if (overwrite) { + 1 + } else { FileSystemHelper .findMaxTaskId(NewFileOutputFormat.getOutputPath(job).toString, job.getConfiguration) + 1 } 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 84b1b4609458b..71ba0fecce47a 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 @@ -158,8 +158,11 @@ private[parquet] class CatalystGroupConverter( 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)) + if (ctype == StringType) { + new CatalystPrimitiveStringConverter(this, schema.indexOf(a)) + } else { + new CatalystPrimitiveConverter(this, schema.indexOf(a)) + } case _ => throw new RuntimeException( s"unable to convert datatype ${a.dataType.toString} in CatalystGroupConverter") } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 3d2537f6f23dc..557e0961d5944 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -240,8 +240,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { if (hasLocationPreferences) { val receiversWithPreferences = receivers.map(r => (r, Seq(r.preferredLocation.get))) ssc.sc.makeRDD[Receiver[_]](receiversWithPreferences) - } - else { + } else { ssc.sc.makeRDD(receivers, receivers.size) } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala index 2f74965900baf..fc13dbecb4555 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ApplicationMaster.scala @@ -147,12 +147,12 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, // LOCAL_DIRS => 2.X, YARN_LOCAL_DIRS => 0.23.X val localDirs = Option(System.getenv("YARN_LOCAL_DIRS")) .orElse(Option(System.getenv("LOCAL_DIRS"))) - + localDirs match { case None => throw new Exception("Yarn Local dirs can't be empty") case Some(l) => l } - } + } private def getApplicationAttemptId(): ApplicationAttemptId = { val envs = System.getenv() @@ -321,8 +321,9 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, logInfo("Allocating %d containers to make up for (potentially) lost containers". format(missingExecutorCount)) yarnAllocator.allocateContainers(missingExecutorCount) + } else { + sendProgress() } - else sendProgress() Thread.sleep(sleepTime) } } @@ -361,7 +362,7 @@ class ApplicationMaster(args: ApplicationMasterArguments, conf: Configuration, return } isFinished = true - + logInfo("finishApplicationMaster with " + status) if (registered) { val finishReq = Records.newRecord(classOf[FinishApplicationMasterRequest]) diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala index ea356f33eb998..65b7215afbd4c 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/ExecutorLauncher.scala @@ -243,8 +243,9 @@ class ExecutorLauncher(args: ApplicationMasterArguments, conf: Configuration, sp logInfo("Allocating " + missingExecutorCount + " containers to make up for (potentially ?) lost containers") yarnAllocator.allocateContainers(missingExecutorCount) + } else { + sendProgress() } - else sendProgress() Thread.sleep(sleepTime) } } diff --git a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala index 95f0f9d0ff2bc..856391e52b2df 100644 --- a/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala +++ b/yarn/alpha/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocationHandler.scala @@ -60,12 +60,12 @@ object AllocationType extends Enumeration { */ private[yarn] class YarnAllocationHandler( val conf: Configuration, - val resourceManager: AMRMProtocol, + val resourceManager: AMRMProtocol, val appAttemptId: ApplicationAttemptId, val maxExecutors: Int, val executorMemory: Int, val executorCores: Int, - val preferredHostToCount: Map[String, Int], + val preferredHostToCount: Map[String, Int], val preferredRackToCount: Map[String, Int], val sparkConf: SparkConf) extends Logging { @@ -136,9 +136,10 @@ private[yarn] class YarnAllocationHandler( val containers = hostToContainers.getOrElseUpdate(host, new ArrayBuffer[Container]()) containers += container + } else { + // Add all ignored containers to released list + releasedContainerList.add(container.getId()) } - // Add all ignored containers to released list - else releasedContainerList.add(container.getId()) } // Find the appropriate containers to use. Slightly non trivial groupBy ... @@ -159,8 +160,7 @@ private[yarn] class YarnAllocationHandler( dataLocalContainers.put(candidateHost, remainingContainers) // all consumed remainingContainers = null - } - else if (requiredHostCount > 0) { + } else if (requiredHostCount > 0) { // Container list has more containers than we need for data locality. // Split into two : data local container count of (remainingContainers.size - // requiredHostCount) and rest as remainingContainer @@ -170,7 +170,7 @@ private[yarn] class YarnAllocationHandler( // remainingContainers = remaining // yarn has nasty habit of allocating a tonne of containers on a host - discourage this : - // add remaining to release list. If we have insufficient containers, next allocation + // add remaining to release list. If we have insufficient containers, next allocation // cycle will reallocate (but wont treat it as data local) for (container <- remaining) releasedContainerList.add(container.getId()) remainingContainers = null @@ -182,7 +182,7 @@ private[yarn] class YarnAllocationHandler( if (rack != null){ val maxExpectedRackCount = preferredRackToCount.getOrElse(rack, 0) - val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - + val requiredRackCount = maxExpectedRackCount - allocatedContainersOnRack(rack) - rackLocalContainers.get(rack).getOrElse(List()).size @@ -191,8 +191,7 @@ private[yarn] class YarnAllocationHandler( dataLocalContainers.put(rack, remainingContainers) // All consumed remainingContainers = null - } - else if (requiredRackCount > 0) { + } else if (requiredRackCount > 0) { // container list has more containers than we need for data locality. // Split into two : data local container count of (remainingContainers.size - // requiredRackCount) and rest as remainingContainer @@ -213,7 +212,7 @@ private[yarn] class YarnAllocationHandler( } } - // Now that we have split the containers into various groups, go through them in order : + // Now that we have split the containers into various groups, go through them in order : // first host local, then rack local and then off rack (everything else). // Note that the list we create below tries to ensure that not all containers end up within a // host if there are sufficiently large number of hosts/containers. @@ -238,8 +237,7 @@ private[yarn] class YarnAllocationHandler( releasedContainerList.add(containerId) // reset counter back to old value. numExecutorsRunning.decrementAndGet() - } - else { + } else { // Deallocate + allocate can result in reusing id's wrongly - so use a different counter // (executorIdCounter) val executorId = executorIdCounter.incrementAndGet().toString @@ -293,8 +291,7 @@ private[yarn] class YarnAllocationHandler( // Was this released by us ? If yes, then simply remove from containerSet and move on. if (pendingReleaseContainers.containsKey(containerId)) { pendingReleaseContainers.remove(containerId) - } - else { + } else { // Simply decrement count - next iteration of ReporterThread will take care of allocating. numExecutorsRunning.decrementAndGet() logInfo("Completed container %s (state: %s, exit status: %s)".format( @@ -319,8 +316,11 @@ private[yarn] class YarnAllocationHandler( assert (containerSet != null) containerSet -= containerId - if (containerSet.isEmpty) allocatedHostToContainersMap.remove(host) - else allocatedHostToContainersMap.update(host, containerSet) + if (containerSet.isEmpty) { + allocatedHostToContainersMap.remove(host) + } else { + allocatedHostToContainersMap.update(host, containerSet) + } allocatedContainerToHostMap -= containerId @@ -328,8 +328,11 @@ private[yarn] class YarnAllocationHandler( val rack = YarnAllocationHandler.lookupRack(conf, host) if (rack != null) { val rackCount = allocatedRackCount.getOrElse(rack, 0) - 1 - if (rackCount > 0) allocatedRackCount.put(rack, rackCount) - else allocatedRackCount.remove(rack) + if (rackCount > 0) { + allocatedRackCount.put(rack, rackCount) + } else { + allocatedRackCount.remove(rack) + } } } } @@ -365,10 +368,10 @@ private[yarn] class YarnAllocationHandler( } } - val requestedContainers: ArrayBuffer[ResourceRequest] = + val requestedContainers: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](rackToCounts.size) for ((rack, count) <- rackToCounts){ - requestedContainers += + requestedContainers += createResourceRequest(AllocationType.RACK, rack, count, YarnAllocationHandler.PRIORITY) } @@ -401,11 +404,10 @@ private[yarn] class YarnAllocationHandler( preferredHostToCount.isEmpty) resourceRequests = List(createResourceRequest( AllocationType.ANY, null, numExecutors, YarnAllocationHandler.PRIORITY)) - } - else { - // request for all hosts in preferred nodes and for numExecutors - + } else { + // request for all hosts in preferred nodes and for numExecutors - // candidates.size, request by default allocation policy. - val hostContainerRequests: ArrayBuffer[ResourceRequest] = + val hostContainerRequests: ArrayBuffer[ResourceRequest] = new ArrayBuffer[ResourceRequest](preferredHostToCount.size) for ((candidateHost, candidateCount) <- preferredHostToCount) { val requiredCount = candidateCount - allocatedContainersOnHost(candidateHost) @@ -449,8 +451,7 @@ private[yarn] class YarnAllocationHandler( if (numExecutors > 0) { logInfo("Allocating %d executor containers with %d of memory each.".format(numExecutors, executorMemory + YarnAllocationHandler.MEMORY_OVERHEAD)) - } - else { + } else { logDebug("Empty allocation req .. release : " + releasedContainerList) } @@ -467,7 +468,7 @@ private[yarn] class YarnAllocationHandler( private def createResourceRequest( - requestType: AllocationType.AllocationType, + requestType: AllocationType.AllocationType, resource:String, numExecutors: Int, priority: Int): ResourceRequest = { @@ -528,7 +529,7 @@ private[yarn] class YarnAllocationHandler( if (! retval.isEmpty) { releasedContainerList.removeAll(retval) for (v <- retval) pendingReleaseContainers.put(v, true) - logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + + logInfo("Releasing " + retval.size + " containers. pendingReleaseContainers : " + pendingReleaseContainers) } @@ -539,7 +540,7 @@ private[yarn] class YarnAllocationHandler( object YarnAllocationHandler { val ANY_HOST = "*" - // All requests are issued with same priority : we do not (yet) have any distinction between + // All requests are issued with same priority : we do not (yet) have any distinction between // request types (like map/reduce in hadoop for example) val PRIORITY = 1 @@ -548,7 +549,7 @@ object YarnAllocationHandler { // Host to rack map - saved from allocation requests // We are expecting this not to change. - // Note that it is possible for this to change : and RM will indicate that to us via update + // Note that it is possible for this to change : and RM will indicate that to us via update // response to allocate. But we are punting on handling that for now. private val hostToRack = new ConcurrentHashMap[String, String]() private val rackToHostSet = new ConcurrentHashMap[String, JSet[String]]() @@ -565,7 +566,7 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numExecutors, + args.numExecutors, args.executorMemory, args.executorCores, Map[String, Int](), @@ -587,7 +588,7 @@ object YarnAllocationHandler { conf, resourceManager, appAttemptId, - args.numExecutors, + args.numExecutors, args.executorMemory, args.executorCores, hostToCount, From f99af8529b6969986f0c3e03f6ff9b7bb9d53ece Mon Sep 17 00:00:00 2001 From: CodingCat Date: Thu, 24 Apr 2014 15:55:18 -0700 Subject: [PATCH 367/397] SPARK-1104: kill Process in workerThread of ExecutorRunner As reported in https://spark-project.atlassian.net/browse/SPARK-1104 By @pwendell: "Sometimes due to large shuffles executors will take a long time shutting down. In particular this can happen if large numbers of shuffle files are around (this will be alleviated by SPARK-1103, but nonetheless...). The symptom is you have DEAD workers sitting around in the UI and the existing workers keep trying to re-register but can't because they've been assumed dead." In this patch, I add lines in the handler of InterruptedException in workerThread of executorRunner, so that the process.destroy() and process.waitFor() can only block the workerThread instead of blocking the worker Actor... --------- analysis: process.destroy() is a blocking method, i.e. it only returns when all shutdownHook threads return...so calling it in Worker thread will make Worker block for a long while.... about what will happen on the shutdown hooks when the JVM process is killed: http://www.tutorialspoint.com/java/lang/runtime_addshutdownhook.htm Author: CodingCat Closes #35 from CodingCat/SPARK-1104 and squashes the following commits: 85767da [CodingCat] add null checking and remove unnecessary killProce 3107aeb [CodingCat] address Aaron's comments eb615ba [CodingCat] kill the process when the error happens 0accf2f [CodingCat] set process to null after killed it 1d511c8 [CodingCat] kill Process in workerThread --- .../spark/deploy/worker/ExecutorRunner.scala | 31 +++++++++---------- 1 file changed, 14 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala index f94cd685e8eb0..2051403682737 100644 --- a/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala +++ b/core/src/main/scala/org/apache/spark/deploy/worker/ExecutorRunner.scala @@ -58,30 +58,29 @@ private[spark] class ExecutorRunner( override def run() { fetchAndRunExecutor() } } workerThread.start() - // Shutdown hook that kills actors on shutdown. shutdownHook = new Thread() { override def run() { - if (process != null) { - logInfo("Shutdown hook killing child process.") - process.destroy() - process.waitFor() - } + killProcess() } } Runtime.getRuntime.addShutdownHook(shutdownHook) } + private def killProcess() { + if (process != null) { + logInfo("Killing process!") + process.destroy() + process.waitFor() + } + } + /** Stop this executor runner, including killing the process it launched */ def kill() { if (workerThread != null) { + // the workerThread will kill the child process when interrupted workerThread.interrupt() workerThread = null - if (process != null) { - logInfo("Killing process!") - process.destroy() - process.waitFor() - } state = ExecutorState.KILLED worker ! ExecutorStateChanged(appId, execId, state, None, None) Runtime.getRuntime.removeShutdownHook(shutdownHook) @@ -128,7 +127,6 @@ private[spark] class ExecutorRunner( // parent process for the executor command env.put("SPARK_LAUNCH_WITH_SCALA", "0") process = builder.start() - val header = "Spark Executor Command: %s\n%s\n\n".format( command.mkString("\"", "\" \"", "\""), "=" * 40) @@ -148,14 +146,13 @@ private[spark] class ExecutorRunner( val message = "Command exited with code " + exitCode worker ! ExecutorStateChanged(appId, execId, state, Some(message), Some(exitCode)) } catch { - case interrupted: InterruptedException => + case interrupted: InterruptedException => { logInfo("Runner thread for executor " + fullId + " interrupted") - + killProcess() + } case e: Exception => { logError("Error running executor", e) - if (process != null) { - process.destroy() - } + killProcess() state = ExecutorState.FAILED val message = e.getClass + ": " + e.getMessage worker ! ExecutorStateChanged(appId, execId, state, Some(message), None) From 35e3d199f04fba3230625002a458d43b9578b2e8 Mon Sep 17 00:00:00 2001 From: Arun Ramakrishnan Date: Thu, 24 Apr 2014 17:27:16 -0700 Subject: [PATCH 368/397] SPARK-1438 RDD.sample() make seed param optional copying form previous pull request https://github.com/apache/spark/pull/462 Its probably better to let the underlying language implementation take care of the default . This was easier to do with python as the default value for seed in random and numpy random is None. In Scala/Java side it might mean propagating an Option or null(oh no!) down the chain until where the Random is constructed. But, looks like the convention in some other methods was to use System.nanoTime. So, followed that convention. Conflict with overloaded method in sql.SchemaRDD.sample which also defines default params. sample(fraction, withReplacement=false, seed=math.random) Scala does not allow more than one overloaded to have default params. I believe the author intended to override the RDD.sample method and not overload it. So, changed it. If backward compatible is important, 3 new method can be introduced (without default params) like this sample(fraction) sample(fraction, withReplacement) sample(fraction, withReplacement, seed) Added some tests for the scala RDD takeSample method. Author: Arun Ramakrishnan This patch had conflicts when merged, resolved by Committer: Matei Zaharia Closes #477 from smartnut007/master and squashes the following commits: 07bb06e [Arun Ramakrishnan] SPARK-1438 fixing more space formatting issues b9ebfe2 [Arun Ramakrishnan] SPARK-1438 removing redundant import of random in python rddsampler 8d05b1a [Arun Ramakrishnan] SPARK-1438 RDD . Replace System.nanoTime with a Random generated number. python: use a separate instance of Random instead of seeding language api global Random instance. 69619c6 [Arun Ramakrishnan] SPARK-1438 fix spacing issue 0c247db [Arun Ramakrishnan] SPARK-1438 RDD language apis to support optional seed in RDD methods sample/takeSample --- .../apache/spark/api/java/JavaDoubleRDD.scala | 9 +++++- .../apache/spark/api/java/JavaPairRDD.scala | 9 +++++- .../org/apache/spark/api/java/JavaRDD.scala | 9 +++++- .../apache/spark/api/java/JavaRDDLike.scala | 6 +++- .../spark/rdd/PartitionwiseSampledRDD.scala | 5 +-- .../main/scala/org/apache/spark/rdd/RDD.scala | 11 ++++--- .../scala/org/apache/spark/util/Utils.scala | 2 ++ .../scala/org/apache/spark/rdd/RDDSuite.scala | 21 ++++++++++++- python/pyspark/rdd.py | 13 ++++---- python/pyspark/rddsampler.py | 31 +++++++++---------- .../plans/logical/basicOperators.scala | 2 +- .../org/apache/spark/sql/SchemaRDD.scala | 5 +-- .../spark/sql/execution/basicOperators.scala | 6 ++-- 13 files changed, 88 insertions(+), 41 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala index 4330cef3965ee..a6123bd108c11 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaDoubleRDD.scala @@ -30,6 +30,7 @@ import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel import org.apache.spark.util.StatCounter +import org.apache.spark.util.Utils class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, JavaDoubleRDD] { @@ -133,7 +134,13 @@ class JavaDoubleRDD(val srdd: RDD[scala.Double]) extends JavaRDDLike[JDouble, Ja /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: JDouble, seed: Int): JavaDoubleRDD = + def sample(withReplacement: Boolean, fraction: JDouble): JavaDoubleRDD = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: JDouble, seed: Long): JavaDoubleRDD = fromRDD(srdd.sample(withReplacement, fraction, seed)) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala index b3ec270281ae4..554c065358648 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaPairRDD.scala @@ -39,6 +39,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.{OrderedRDDFunctions, RDD} import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) (implicit val kClassTag: ClassTag[K], implicit val vClassTag: ClassTag[V]) @@ -119,7 +120,13 @@ class JavaPairRDD[K, V](val rdd: RDD[(K, V)]) /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaPairRDD[K, V] = + def sample(withReplacement: Boolean, fraction: Double): JavaPairRDD[K, V] = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaPairRDD[K, V] = new JavaPairRDD[K, V](rdd.sample(withReplacement, fraction, seed)) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala index 327c1552dc941..dc698dea75e43 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDD.scala @@ -24,6 +24,7 @@ import org.apache.spark._ import org.apache.spark.api.java.function.{Function => JFunction} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) extends JavaRDDLike[T, JavaRDD[T]] { @@ -98,7 +99,13 @@ class JavaRDD[T](val rdd: RDD[T])(implicit val classTag: ClassTag[T]) /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): JavaRDD[T] = + def sample(withReplacement: Boolean, fraction: Double): JavaRDD[T] = + sample(withReplacement, fraction, Utils.random.nextLong) + + /** + * Return a sampled subset of this RDD. + */ + def sample(withReplacement: Boolean, fraction: Double, seed: Long): JavaRDD[T] = wrapRDD(rdd.sample(withReplacement, fraction, seed)) /** diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala index 725c423a53e35..574a98636a619 100644 --- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala +++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala @@ -34,6 +34,7 @@ import org.apache.spark.api.java.function.{Function => JFunction, Function2 => J import org.apache.spark.partial.{BoundedDouble, PartialResult} import org.apache.spark.rdd.RDD import org.apache.spark.storage.StorageLevel +import org.apache.spark.util.Utils trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { def wrapRDD(rdd: RDD[T]): This @@ -394,7 +395,10 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable { new java.util.ArrayList(arr) } - def takeSample(withReplacement: Boolean, num: Int, seed: Int): JList[T] = { + def takeSample(withReplacement: Boolean, num: Int): JList[T] = + takeSample(withReplacement, num, Utils.random.nextLong) + + def takeSample(withReplacement: Boolean, num: Int, seed: Long): JList[T] = { import scala.collection.JavaConversions._ val arr: java.util.Collection[T] = rdd.takeSample(withReplacement, num, seed).toSeq new java.util.ArrayList(arr) diff --git a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala index b4e3bb5d75e17..b5b8a5706deb3 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PartitionwiseSampledRDD.scala @@ -23,6 +23,7 @@ import scala.reflect.ClassTag import org.apache.spark.{Partition, TaskContext} import org.apache.spark.util.random.RandomSampler +import org.apache.spark.util.Utils private[spark] class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) @@ -38,14 +39,14 @@ class PartitionwiseSampledRDDPartition(val prev: Partition, val seed: Long) * * @param prev RDD to be sampled * @param sampler a random sampler - * @param seed random seed, default to System.nanoTime + * @param seed random seed * @tparam T input RDD item type * @tparam U sampled RDD item type */ private[spark] class PartitionwiseSampledRDD[T: ClassTag, U: ClassTag]( prev: RDD[T], sampler: RandomSampler[T, U], - @transient seed: Long = System.nanoTime) + @transient seed: Long = Utils.random.nextLong) extends RDD[U](prev) { override def getPartitions: Array[Partition] = { diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index 6c897cc03b641..e8bbfbf01679c 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -341,7 +341,9 @@ abstract class RDD[T: ClassTag]( /** * Return a sampled subset of this RDD. */ - def sample(withReplacement: Boolean, fraction: Double, seed: Int): RDD[T] = { + def sample(withReplacement: Boolean, + fraction: Double, + seed: Long = Utils.random.nextLong): RDD[T] = { require(fraction >= 0.0, "Invalid fraction value: " + fraction) if (withReplacement) { new PartitionwiseSampledRDD[T, T](this, new PoissonSampler[T](fraction), seed) @@ -354,11 +356,11 @@ abstract class RDD[T: ClassTag]( * Randomly splits this RDD with the provided weights. * * @param weights weights for splits, will be normalized if they don't sum to 1 - * @param seed random seed, default to System.nanoTime + * @param seed random seed * * @return split RDDs in an array */ - def randomSplit(weights: Array[Double], seed: Long = System.nanoTime): Array[RDD[T]] = { + def randomSplit(weights: Array[Double], seed: Long = Utils.random.nextLong): Array[RDD[T]] = { val sum = weights.sum val normalizedCumWeights = weights.map(_ / sum).scanLeft(0.0d)(_ + _) normalizedCumWeights.sliding(2).map { x => @@ -366,7 +368,8 @@ abstract class RDD[T: ClassTag]( }.toArray } - def takeSample(withReplacement: Boolean, num: Int, seed: Int): Array[T] = { + def takeSample(withReplacement: Boolean, num: Int, seed: Long = Utils.random.nextLong): Array[T] = + { var fraction = 0.0 var total = 0 val multiplier = 3.0 diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index d333e2a88c18c..084a71c4caebd 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -46,6 +46,8 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, private[spark] object Utils extends Logging { val osName = System.getProperty("os.name") + + val random = new Random() /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { diff --git a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala index 2676558bfc86d..8da9a0da700e0 100644 --- a/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala +++ b/core/src/test/scala/org/apache/spark/rdd/RDDSuite.scala @@ -463,7 +463,13 @@ class RDDSuite extends FunSuite with SharedSparkContext { test("takeSample") { val data = sc.parallelize(1 to 100, 2) - + + for (num <- List(5, 20, 100)) { + val sample = data.takeSample(withReplacement=false, num=num) + assert(sample.size === num) // Got exactly num elements + assert(sample.toSet.size === num) // Elements are distinct + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=false, 20, seed) assert(sample.size === 20) // Got exactly 20 elements @@ -481,6 +487,19 @@ class RDDSuite extends FunSuite with SharedSparkContext { assert(sample.size === 20) // Got exactly 20 elements assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") } + { + val sample = data.takeSample(withReplacement=true, num=20) + assert(sample.size === 20) // Got exactly 100 elements + assert(sample.toSet.size <= 20, "sampling with replacement returned all distinct elements") + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } + { + val sample = data.takeSample(withReplacement=true, num=100) + assert(sample.size === 100) // Got exactly 100 elements + // Chance of getting all distinct elements is astronomically low, so test we got < 100 + assert(sample.toSet.size < 100, "sampling with replacement returned all distinct elements") + assert(sample.forall(x => 1 <= x && x <= 100), "elements not in [1, 100]") + } for (seed <- 1 to 5) { val sample = data.takeSample(withReplacement=true, 100, seed) assert(sample.size === 100) // Got exactly 100 elements diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index 91fc7e637e2c6..d73ab7006e9c7 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -30,6 +30,7 @@ from threading import Thread import warnings import heapq +from random import Random from pyspark.serializers import NoOpSerializer, CartesianDeserializer, \ BatchedSerializer, CloudPickleSerializer, PairDeserializer, pack_long @@ -332,7 +333,7 @@ def distinct(self): .reduceByKey(lambda x, _: x) \ .map(lambda (x, _): x) - def sample(self, withReplacement, fraction, seed): + def sample(self, withReplacement, fraction, seed=None): """ Return a sampled subset of this RDD (relies on numpy and falls back on default random generator if numpy is unavailable). @@ -344,7 +345,7 @@ def sample(self, withReplacement, fraction, seed): return self.mapPartitionsWithIndex(RDDSampler(withReplacement, fraction, seed).func, True) # this is ported from scala/spark/RDD.scala - def takeSample(self, withReplacement, num, seed): + def takeSample(self, withReplacement, num, seed=None): """ Return a fixed-size sampled subset of this RDD (currently requires numpy). @@ -381,13 +382,11 @@ def takeSample(self, withReplacement, num, seed): # If the first sample didn't turn out large enough, keep trying to take samples; # this shouldn't happen often because we use a big multiplier for their initial size. # See: scala/spark/RDD.scala + rand = Random(seed) while len(samples) < total: - if seed > sys.maxint - 2: - seed = -1 - seed += 1 - samples = self.sample(withReplacement, fraction, seed).collect() + samples = self.sample(withReplacement, fraction, rand.randint(0, sys.maxint)).collect() - sampler = RDDSampler(withReplacement, fraction, seed+1) + sampler = RDDSampler(withReplacement, fraction, rand.randint(0, sys.maxint)) sampler.shuffle(samples) return samples[0:total] diff --git a/python/pyspark/rddsampler.py b/python/pyspark/rddsampler.py index aca2ef3b51e98..845a267e311c5 100644 --- a/python/pyspark/rddsampler.py +++ b/python/pyspark/rddsampler.py @@ -19,7 +19,7 @@ import random class RDDSampler(object): - def __init__(self, withReplacement, fraction, seed): + def __init__(self, withReplacement, fraction, seed=None): try: import numpy self._use_numpy = True @@ -27,7 +27,7 @@ def __init__(self, withReplacement, fraction, seed): print >> sys.stderr, "NumPy does not appear to be installed. Falling back to default random generator for sampling." self._use_numpy = False - self._seed = seed + self._seed = seed if seed is not None else random.randint(0, sys.maxint) self._withReplacement = withReplacement self._fraction = fraction self._random = None @@ -38,17 +38,14 @@ def initRandomGenerator(self, split): if self._use_numpy: import numpy self._random = numpy.random.RandomState(self._seed) - for _ in range(0, split): - # discard the next few values in the sequence to have a - # different seed for the different splits - self._random.randint(sys.maxint) else: - import random - random.seed(self._seed) - for _ in range(0, split): - # discard the next few values in the sequence to have a - # different seed for the different splits - random.randint(0, sys.maxint) + self._random = random.Random(self._seed) + + for _ in range(0, split): + # discard the next few values in the sequence to have a + # different seed for the different splits + self._random.randint(0, sys.maxint) + self._split = split self._rand_initialized = True @@ -59,7 +56,7 @@ def getUniformSample(self, split): if self._use_numpy: return self._random.random_sample() else: - return random.uniform(0.0, 1.0) + return self._random.uniform(0.0, 1.0) def getPoissonSample(self, split, mean): if not self._rand_initialized or split != self._split: @@ -73,26 +70,26 @@ def getPoissonSample(self, split, mean): num_arrivals = 1 cur_time = 0.0 - cur_time += random.expovariate(mean) + cur_time += self._random.expovariate(mean) if cur_time > 1.0: return 0 while(cur_time <= 1.0): - cur_time += random.expovariate(mean) + cur_time += self._random.expovariate(mean) num_arrivals += 1 return (num_arrivals - 1) def shuffle(self, vals): - if self._random == None or split != self._split: + if self._random == None: self.initRandomGenerator(0) # this should only ever called on the master so # the split does not matter if self._use_numpy: self._random.shuffle(vals) else: - random.shuffle(vals, self._random) + self._random.shuffle(vals, self._random.random) def func(self, split, iterator): if self._withReplacement: 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 397473e178867..732708e146b04 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 @@ -168,7 +168,7 @@ case class LowerCaseSchema(child: LogicalPlan) extends UnaryNode { def references = Set.empty } -case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: LogicalPlan) +case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: LogicalPlan) extends UnaryNode { def output = child.output diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala index 6cb0e0f61f2d2..ca6e0a696405a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SchemaRDD.scala @@ -256,10 +256,11 @@ class SchemaRDD( * @group Query */ @Experimental + override def sample( - fraction: Double, withReplacement: Boolean = true, - seed: Int = (math.random * 1000).toInt) = + fraction: Double, + seed: Long) = new SchemaRDD(sqlContext, Sample(fraction, withReplacement, seed, logicalPlan)) /** 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 e4cf2020a1a93..d807187a5ffb8 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 @@ -57,9 +57,9 @@ case class Filter(condition: Expression, child: SparkPlan) extends UnaryNode { * :: DeveloperApi :: */ @DeveloperApi -case class Sample(fraction: Double, withReplacement: Boolean, seed: Int, child: SparkPlan) - extends UnaryNode { - +case class Sample(fraction: Double, withReplacement: Boolean, seed: Long, child: SparkPlan) + extends UnaryNode +{ override def output = child.output // TODO: How to pick seed? From 526a518bf32ad55b926a26f16086f445fd0ae29f Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 24 Apr 2014 18:18:22 -0700 Subject: [PATCH 369/397] [SPARK-1592][streaming] Automatically remove streaming input blocks The raw input data is stored as blocks in BlockManagers. Earlier they were cleared by cleaner ttl. Now since streaming does not require cleaner TTL to be set, the block would not get cleared. This increases up the Spark's memory usage, which is not even accounted and shown in the Spark storage UI. It may cause the data blocks to spill over to disk, which eventually slows down the receiving of data (persisting to memory become bottlenecked by writing to disk). The solution in this PR is to automatically remove those blocks. The mechanism to keep track of which BlockRDDs (which has presents the raw data blocks as a RDD) can be safely cleared already exists. Just use it to explicitly remove blocks from BlockRDDs. Author: Tathagata Das Closes #512 from tdas/block-rdd-unpersist and squashes the following commits: d25e610 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist 5f46d69 [Tathagata Das] Merge remote-tracking branch 'apache/master' into block-rdd-unpersist 2c320cd [Tathagata Das] Updated configuration with spark.streaming.unpersist setting. 2d4b2fd [Tathagata Das] Automatically removed input blocks --- .../scala/org/apache/spark/rdd/BlockRDD.scala | 45 +++++++++-- docs/configuration.md | 7 +- .../org/apache/spark/streaming/Time.scala | 2 +- .../spark/streaming/dstream/DStream.scala | 16 +++- .../streaming/BasicOperationsSuite.scala | 76 ++++++++++++++++++- .../spark/streaming/InputStreamsSuite.scala | 13 ---- .../streaming/NetworkReceiverSuite.scala | 1 + 7 files changed, 135 insertions(+), 25 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala index e6c4a6d3794a0..c64da8804d166 100644 --- a/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/BlockRDD.scala @@ -19,24 +19,30 @@ package org.apache.spark.rdd import scala.reflect.ClassTag -import org.apache.spark.{Partition, SparkContext, SparkEnv, TaskContext} +import org.apache.spark._ import org.apache.spark.storage.{BlockId, BlockManager} +import scala.Some private[spark] class BlockRDDPartition(val blockId: BlockId, idx: Int) extends Partition { val index = idx } private[spark] -class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId]) +class BlockRDD[T: ClassTag](@transient sc: SparkContext, @transient val blockIds: Array[BlockId]) extends RDD[T](sc, Nil) { @transient lazy val locations_ = BlockManager.blockIdsToHosts(blockIds, SparkEnv.get) + @volatile private var _isValid = true - override def getPartitions: Array[Partition] = (0 until blockIds.size).map(i => { - new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] - }).toArray + override def getPartitions: Array[Partition] = { + assertValid() + (0 until blockIds.size).map(i => { + new BlockRDDPartition(blockIds(i), i).asInstanceOf[Partition] + }).toArray + } override def compute(split: Partition, context: TaskContext): Iterator[T] = { + assertValid() val blockManager = SparkEnv.get.blockManager val blockId = split.asInstanceOf[BlockRDDPartition].blockId blockManager.get(blockId) match { @@ -47,7 +53,36 @@ class BlockRDD[T: ClassTag](sc: SparkContext, @transient blockIds: Array[BlockId } override def getPreferredLocations(split: Partition): Seq[String] = { + assertValid() locations_(split.asInstanceOf[BlockRDDPartition].blockId) } + + /** + * Remove the data blocks that this BlockRDD is made from. NOTE: This is an + * irreversible operation, as the data in the blocks cannot be recovered back + * once removed. Use it with caution. + */ + private[spark] def removeBlocks() { + blockIds.foreach { blockId => + sc.env.blockManager.master.removeBlock(blockId) + } + _isValid = false + } + + /** + * Whether this BlockRDD is actually usable. This will be false if the data blocks have been + * removed using `this.removeBlocks`. + */ + private[spark] def isValid: Boolean = { + _isValid + } + + /** Check if this BlockRDD is valid. If not valid, exception is thrown. */ + private[spark] def assertValid() { + if (!_isValid) { + throw new SparkException( + "Attempted to use %s after its blocks have been removed!".format(toString)) + } + } } diff --git a/docs/configuration.md b/docs/configuration.md index e7e1dd56cf124..8d3442625b475 100644 --- a/docs/configuration.md +++ b/docs/configuration.md @@ -469,10 +469,13 @@ Apart from these, the following properties are also available, and may be useful spark.streaming.unpersist - false + true Force RDDs generated and persisted by Spark Streaming to be automatically unpersisted from - Spark's memory. Setting this to true is likely to reduce Spark's RDD memory usage. + Spark's memory. The raw input data received by Spark Streaming is also automatically cleared. + Setting this to false will allow the raw data and persisted RDDs to be accessible outside the + streaming application as they will not be cleared automatically. But it comes at the cost of + higher memory usage in Spark. diff --git a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala index 6a6b00a778b48..37b3b28fa01cb 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/Time.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/Time.scala @@ -68,5 +68,5 @@ case class Time(private val millis: Long) { } object Time { - val ordering = Ordering.by((time: Time) => time.millis) + implicit val ordering = Ordering.by((time: Time) => time.millis) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index d393cc03cb33e..f69f69e0c44af 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -25,7 +25,7 @@ import scala.reflect.ClassTag import java.io.{IOException, ObjectInputStream, ObjectOutputStream} import org.apache.spark.Logging -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel import org.apache.spark.util.MetadataCleaner import org.apache.spark.streaming._ @@ -340,13 +340,23 @@ abstract class DStream[T: ClassTag] ( * this to clear their own metadata along with the generated RDDs. */ private[streaming] def clearMetadata(time: Time) { + val unpersistData = ssc.conf.getBoolean("spark.streaming.unpersist", true) val oldRDDs = generatedRDDs.filter(_._1 <= (time - rememberDuration)) logDebug("Clearing references to old RDDs: [" + oldRDDs.map(x => s"${x._1} -> ${x._2.id}").mkString(", ") + "]") generatedRDDs --= oldRDDs.keys - if (ssc.conf.getBoolean("spark.streaming.unpersist", false)) { + if (unpersistData) { logDebug("Unpersisting old RDDs: " + oldRDDs.values.map(_.id).mkString(", ")) - oldRDDs.values.foreach(_.unpersist(false)) + oldRDDs.values.foreach { rdd => + rdd.unpersist(false) + // Explicitly remove blocks of BlockRDD + rdd match { + case b: BlockRDD[_] => + logInfo("Removing blocks of RDD " + b + " of time " + time) + b.removeBlocks() + case _ => + } + } } logDebug("Cleared " + oldRDDs.size + " RDDs that were older than " + (time - rememberDuration) + ": " + oldRDDs.keys.mkString(", ")) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 8aec27e39478a..4792ca1f8ae3e 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -19,7 +19,7 @@ package org.apache.spark.streaming import org.apache.spark.streaming.StreamingContext._ -import org.apache.spark.rdd.RDD +import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.SparkContext._ import util.ManualClock @@ -27,6 +27,8 @@ import org.apache.spark.{SparkContext, SparkConf} import org.apache.spark.streaming.dstream.{WindowedDStream, DStream} import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.reflect.ClassTag +import org.apache.spark.storage.StorageLevel +import scala.collection.mutable class BasicOperationsSuite extends TestSuiteBase { test("map") { @@ -450,6 +452,78 @@ class BasicOperationsSuite extends TestSuiteBase { assert(!stateStream.generatedRDDs.contains(Time(4000))) } + test("rdd cleanup - input blocks and persisted RDDs") { + // Actually receive data over through receiver to create BlockRDDs + + // Start the server + val testServer = new TestServer() + testServer.start() + + // Set up the streaming context and input streams + val ssc = new StreamingContext(conf, batchDuration) + val networkStream = ssc.socketTextStream("localhost", testServer.port, StorageLevel.MEMORY_AND_DISK) + val mappedStream = networkStream.map(_ + ".").persist() + val outputBuffer = new ArrayBuffer[Seq[String]] with SynchronizedBuffer[Seq[String]] + val outputStream = new TestOutputStream(mappedStream, outputBuffer) + + outputStream.register() + ssc.start() + + // Feed data to the server to send to the network receiver + val clock = ssc.scheduler.clock.asInstanceOf[ManualClock] + val input = Seq(1, 2, 3, 4, 5, 6) + + val blockRdds = new mutable.HashMap[Time, BlockRDD[_]] + val persistentRddIds = new mutable.HashMap[Time, Int] + + def collectRddInfo() { // get all RDD info required for verification + networkStream.generatedRDDs.foreach { case (time, rdd) => + blockRdds(time) = rdd.asInstanceOf[BlockRDD[_]] + } + mappedStream.generatedRDDs.foreach { case (time, rdd) => + persistentRddIds(time) = rdd.id + } + } + + Thread.sleep(200) + for (i <- 0 until input.size) { + testServer.send(input(i).toString + "\n") + Thread.sleep(200) + clock.addToTime(batchDuration.milliseconds) + collectRddInfo() + } + + Thread.sleep(200) + collectRddInfo() + logInfo("Stopping server") + testServer.stop() + logInfo("Stopping context") + + // verify data has been received + assert(outputBuffer.size > 0) + assert(blockRdds.size > 0) + assert(persistentRddIds.size > 0) + + import Time._ + + val latestPersistedRddId = persistentRddIds(persistentRddIds.keySet.max) + val earliestPersistedRddId = persistentRddIds(persistentRddIds.keySet.min) + val latestBlockRdd = blockRdds(blockRdds.keySet.max) + val earliestBlockRdd = blockRdds(blockRdds.keySet.min) + // verify that the latest mapped RDD is persisted but the earliest one has been unpersisted + assert(ssc.sparkContext.persistentRdds.contains(latestPersistedRddId)) + assert(!ssc.sparkContext.persistentRdds.contains(earliestPersistedRddId)) + + // verify that the latest input blocks are present but the earliest blocks have been removed + assert(latestBlockRdd.isValid) + assert(latestBlockRdd.collect != null) + assert(!earliestBlockRdd.isValid) + earliestBlockRdd.blockIds.foreach { blockId => + assert(!ssc.sparkContext.env.blockManager.master.contains(blockId)) + } + ssc.stop() + } + /** Test cleanup of RDDs in DStream metadata */ def runCleanupTest[T: ClassTag]( conf2: SparkConf, diff --git a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala index 3bad871b5c580..b55b7834c90c1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/InputStreamsSuite.scala @@ -42,8 +42,6 @@ import org.apache.spark.streaming.receiver.{ActorHelper, Receiver} class InputStreamsSuite extends TestSuiteBase with BeforeAndAfter { - val testPort = 9999 - test("socket input stream") { // Start the server val testServer = new TestServer() @@ -288,17 +286,6 @@ class TestServer(portToBind: Int = 0) extends Logging { def port = serverSocket.getLocalPort } -object TestServer { - def main(args: Array[String]) { - val s = new TestServer() - s.start() - while(true) { - Thread.sleep(1000) - s.send("hello") - } - } -} - /** This is an actor for testing actor input stream */ class TestActor(port: Int) extends Actor with ActorHelper { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index 45304c76b0928..ff3619a59042d 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -29,6 +29,7 @@ import org.scalatest.FunSuite import org.scalatest.concurrent.Timeouts import org.scalatest.concurrent.Eventually._ import org.scalatest.time.SpanSugar._ +import scala.language.postfixOps /** Testsuite for testing the network receiver behavior */ class NetworkReceiverSuite extends FunSuite with Timeouts { From 4660991e679eda158a3ae8039b686eae197a71d1 Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 24 Apr 2014 18:21:00 -0700 Subject: [PATCH 370/397] [SQL] Add support for parsing indexing into arrays in SQL. Author: Michael Armbrust Closes #518 from marmbrus/parseArrayIndex and squashes the following commits: afd2d6b [Michael Armbrust] 100 chars c3d6026 [Michael Armbrust] Add support for parsing indexing into arrays in SQL. --- .../apache/spark/sql/catalyst/SqlParser.scala | 10 +++++++--- .../org/apache/spark/sql/SQLQuerySuite.scala | 16 ++++++++++++++++ .../scala/org/apache/spark/sql/TestData.scala | 7 +++++++ 3 files changed, 30 insertions(+), 3 deletions(-) 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 index 13a19d0adf5e6..8c76a3aa96546 100644 --- 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 @@ -20,6 +20,7 @@ package org.apache.spark.sql.catalyst import scala.language.implicitConversions import scala.util.parsing.combinator.lexical.StdLexical import scala.util.parsing.combinator.syntactical.StandardTokenParsers +import scala.util.parsing.combinator.PackratParsers import scala.util.parsing.input.CharArrayReader.EofCh import org.apache.spark.sql.catalyst.analysis._ @@ -39,7 +40,7 @@ import org.apache.spark.sql.catalyst.types._ * 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 sub-project. */ -class SqlParser extends StandardTokenParsers { +class SqlParser extends StandardTokenParsers with PackratParsers { def apply(input: String): LogicalPlan = { phrase(query)(new lexical.Scanner(input)) match { case Success(r, x) => r @@ -152,7 +153,7 @@ class SqlParser extends StandardTokenParsers { lexical.delimiters += ( "@", "*", "+", "-", "<", "=", "<>", "!=", "<=", ">=", ">", "/", "(", ")", - ",", ";", "%", "{", "}", ":" + ",", ";", "%", "{", "}", ":", "[", "]" ) protected def assignAliases(exprs: Seq[Expression]): Seq[NamedExpression] = { @@ -339,7 +340,10 @@ class SqlParser extends StandardTokenParsers { protected lazy val floatLit: Parser[String] = elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars) - protected lazy val baseExpression: Parser[Expression] = + protected lazy val baseExpression: PackratParser[Expression] = + expression ~ "[" ~ expression <~ "]" ^^ { + case base ~ _ ~ ordinal => GetItem(base, ordinal) + } | TRUE ^^^ Literal(true, BooleanType) | FALSE ^^^ Literal(false, BooleanType) | cast | diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 4c4fd6dbbedb4..dde957d715a28 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -28,6 +28,22 @@ class SQLQuerySuite extends QueryTest { // Make sure the tables are loaded. TestData + test("index into array") { + checkAnswer( + sql("SELECT data, data[0], data[0] + data[1], data[0 + 1] FROM arrayData"), + arrayData.map(d => (d.data, d.data(0), d.data(0) + d.data(1), d.data(1))).collect().toSeq) + } + + test("index into array of arrays") { + checkAnswer( + sql( + "SELECT nestedData, nestedData[0][0], nestedData[0][0] + nestedData[0][1] FROM arrayData"), + arrayData.map(d => + (d.nestedData, + d.nestedData(0)(0), + d.nestedData(0)(0) + d.nestedData(0)(1))).collect().toSeq) + } + test("agg") { checkAnswer( sql("SELECT a, SUM(b) FROM testData2 GROUP BY a"), diff --git a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala index 271b1d9fcacf8..002b7f0adafab 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/TestData.scala @@ -66,4 +66,11 @@ object TestData { LowerCaseData(3, "c") :: LowerCaseData(4, "d") :: Nil) lowerCaseData.registerAsTable("lowerCaseData") + + case class ArrayData(data: Seq[Int], nestedData: Seq[Seq[Int]]) + val arrayData = + TestSQLContext.sparkContext.parallelize( + ArrayData(Seq(1,2,3), Seq(Seq(1,2,3))) :: + ArrayData(Seq(2,3,4), Seq(Seq(2,3,4))) :: Nil) + arrayData.registerAsTable("arrayData") } From 44da5ab2dea6dcf1e13d624784741141883870bb Mon Sep 17 00:00:00 2001 From: Thomas Graves Date: Thu, 24 Apr 2014 18:38:10 -0700 Subject: [PATCH 371/397] Spark 1489 Fix the HistoryServer view acls This allows the view acls set by the user to be enforced by the history server. It also fixes filters being applied properly. Author: Thomas Graves Closes #509 from tgravescs/SPARK-1489 and squashes the following commits: 869c186 [Thomas Graves] change to either acls enabled or disabled 0d8333c [Thomas Graves] Add history ui policy to allow acls to either use application set, history server force acls on, or off 65148b5 [Thomas Graves] SPARK-1489 Fix the HistoryServer view acls --- .../org/apache/spark/SecurityManager.scala | 26 ++++++-- .../spark/deploy/history/HistoryServer.scala | 14 ++++- .../scheduler/ApplicationEventListener.scala | 11 ++++ .../org/apache/spark/ui/JettyUtils.scala | 4 +- .../scala/org/apache/spark/ui/SparkUI.scala | 8 +++ .../apache/spark/SecurityManagerSuite.scala | 63 +++++++++++++++++++ docs/monitoring.md | 13 ++++ 7 files changed, 128 insertions(+), 11 deletions(-) create mode 100644 core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala diff --git a/core/src/main/scala/org/apache/spark/SecurityManager.scala b/core/src/main/scala/org/apache/spark/SecurityManager.scala index b52f2d4f416b2..b4b0067801259 100644 --- a/core/src/main/scala/org/apache/spark/SecurityManager.scala +++ b/core/src/main/scala/org/apache/spark/SecurityManager.scala @@ -19,8 +19,6 @@ package org.apache.spark import java.net.{Authenticator, PasswordAuthentication} -import scala.collection.mutable.ArrayBuffer - import org.apache.hadoop.io.Text import org.apache.spark.deploy.SparkHadoopUtil @@ -139,13 +137,13 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { private val sparkSecretLookupKey = "sparkCookie" private val authOn = sparkConf.getBoolean("spark.authenticate", false) - private val uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + private var uiAclsOn = sparkConf.getBoolean("spark.ui.acls.enable", false) + private var viewAcls: Set[String] = _ // always add the current user and SPARK_USER to the viewAcls - private val aclUsers = ArrayBuffer[String](System.getProperty("user.name", ""), + private val defaultAclUsers = Seq[String](System.getProperty("user.name", ""), Option(System.getenv("SPARK_USER")).getOrElse("")) - aclUsers ++= sparkConf.get("spark.ui.view.acls", "").split(',') - private val viewAcls = aclUsers.map(_.trim()).filter(!_.isEmpty).toSet + setViewAcls(defaultAclUsers, sparkConf.get("spark.ui.view.acls", "")) private val secretKey = generateSecretKey() logInfo("SecurityManager, is authentication enabled: " + authOn + @@ -170,6 +168,20 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { ) } + private[spark] def setViewAcls(defaultUsers: Seq[String], allowedUsers: String) { + viewAcls = (defaultUsers ++ allowedUsers.split(',')).map(_.trim()).filter(!_.isEmpty).toSet + logInfo("Changing view acls to: " + viewAcls.mkString(",")) + } + + private[spark] def setViewAcls(defaultUser: String, allowedUsers: String) { + setViewAcls(Seq[String](defaultUser), allowedUsers) + } + + private[spark] def setUIAcls(aclSetting: Boolean) { + uiAclsOn = aclSetting + logInfo("Changing acls enabled to: " + uiAclsOn) + } + /** * Generates or looks up the secret key. * @@ -222,6 +234,8 @@ private[spark] class SecurityManager(sparkConf: SparkConf) extends Logging { * @return true is the user has permission, otherwise false */ def checkUIViewPermissions(user: String): Boolean = { + logDebug("user=" + user + " uiAclsEnabled=" + uiAclsEnabled() + " viewAcls=" + + viewAcls.mkString(",")) if (uiAclsEnabled() && (user != null) && (!viewAcls.contains(user))) false else true } diff --git a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala index d7a3246bcfb47..1238bbf9da2fd 100644 --- a/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala +++ b/core/src/main/scala/org/apache/spark/deploy/history/HistoryServer.scala @@ -168,17 +168,21 @@ class HistoryServer( * directory. If this file exists, the associated application is regarded to be completed, in * which case the server proceeds to render the SparkUI. Otherwise, the server does nothing. */ - private def renderSparkUI(logDir: FileStatus, logInfo: EventLoggingInfo) { + private def renderSparkUI(logDir: FileStatus, elogInfo: EventLoggingInfo) { val path = logDir.getPath val appId = path.getName - val replayBus = new ReplayListenerBus(logInfo.logPaths, fileSystem, logInfo.compressionCodec) + val replayBus = new ReplayListenerBus(elogInfo.logPaths, fileSystem, elogInfo.compressionCodec) val appListener = new ApplicationEventListener replayBus.addListener(appListener) - val ui = new SparkUI(conf, replayBus, appId, "/history/" + appId) + val appConf = conf.clone() + val appSecManager = new SecurityManager(appConf) + val ui = new SparkUI(conf, appSecManager, replayBus, appId, "/history/" + appId) // Do not call ui.bind() to avoid creating a new server for each application replayBus.replay() if (appListener.applicationStarted) { + appSecManager.setUIAcls(HISTORY_UI_ACLS_ENABLED) + appSecManager.setViewAcls(appListener.sparkUser, appListener.viewAcls) attachSparkUI(ui) val appName = appListener.appName val sparkUser = appListener.sparkUser @@ -202,6 +206,7 @@ class HistoryServer( private def attachSparkUI(ui: SparkUI) { assert(serverInfo.isDefined, "HistoryServer must be bound before attaching SparkUIs") ui.getHandlers.foreach(attachHandler) + addFilters(ui.getHandlers, conf) } /** Detach a reconstructed UI from this server. Only valid after bind(). */ @@ -255,6 +260,9 @@ object HistoryServer { // The port to which the web UI is bound val WEB_UI_PORT = conf.getInt("spark.history.ui.port", 18080) + // set whether to enable or disable view acls for all applications + val HISTORY_UI_ACLS_ENABLED = conf.getBoolean("spark.history.ui.acls.enable", false) + val STATIC_RESOURCE_DIR = SparkUI.STATIC_RESOURCE_DIR def main(argStrings: Array[String]) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala index c1001227151a5..cd5d44ad4a7e6 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/ApplicationEventListener.scala @@ -28,6 +28,8 @@ private[spark] class ApplicationEventListener extends SparkListener { var sparkUser = "" var startTime = -1L var endTime = -1L + var viewAcls = "" + var enableViewAcls = false def applicationStarted = startTime != -1 @@ -47,4 +49,13 @@ private[spark] class ApplicationEventListener extends SparkListener { override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd) { endTime = applicationEnd.time } + + override def onEnvironmentUpdate(environmentUpdate: SparkListenerEnvironmentUpdate) { + synchronized { + val environmentDetails = environmentUpdate.environmentDetails + val allProperties = environmentDetails("Spark Properties").toMap + viewAcls = allProperties.getOrElse("spark.ui.view.acls", "") + enableViewAcls = allProperties.getOrElse("spark.ui.acls.enable", "false").toBoolean + } + } } diff --git a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala index fdeb15b5d058a..b3ac2320f3431 100644 --- a/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/JettyUtils.scala @@ -137,8 +137,8 @@ private[spark] object JettyUtils extends Logging { contextHandler } - /** Add security filters, if any, do the given list of ServletContextHandlers */ - private def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { + /** Add filters, if any, to the given list of ServletContextHandlers */ + def addFilters(handlers: Seq[ServletContextHandler], conf: SparkConf) { val filters: Array[String] = conf.get("spark.ui.filters", "").split(',').map(_.trim()) filters.foreach { case filter : String => diff --git a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala index 2fef1a635427c..097a1b81e1dd1 100644 --- a/core/src/main/scala/org/apache/spark/ui/SparkUI.scala +++ b/core/src/main/scala/org/apache/spark/ui/SparkUI.scala @@ -43,6 +43,14 @@ private[spark] class SparkUI( def this(conf: SparkConf, listenerBus: SparkListenerBus, appName: String, basePath: String) = this(null, conf, new SecurityManager(conf), listenerBus, appName, basePath) + def this( + conf: SparkConf, + securityManager: SecurityManager, + listenerBus: SparkListenerBus, + appName: String, + basePath: String) = + this(null, conf, securityManager, listenerBus, appName, basePath) + // If SparkContext is not provided, assume the associated application is not live val live = sc != null diff --git a/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala new file mode 100644 index 0000000000000..e39093e24d68a --- /dev/null +++ b/core/src/test/scala/org/apache/spark/SecurityManagerSuite.scala @@ -0,0 +1,63 @@ +/* + * 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 + +import scala.collection.mutable.ArrayBuffer + +import org.scalatest.FunSuite + +class SecurityManagerSuite extends FunSuite { + + test("set security with conf") { + val conf = new SparkConf + conf.set("spark.authenticate", "true") + conf.set("spark.authenticate.secret", "good") + conf.set("spark.ui.acls.enable", "true") + conf.set("spark.ui.view.acls", "user1,user2") + val securityManager = new SecurityManager(conf); + assert(securityManager.isAuthenticationEnabled() === true) + assert(securityManager.uiAclsEnabled() === true) + assert(securityManager.checkUIViewPermissions("user1") === true) + assert(securityManager.checkUIViewPermissions("user2") === true) + assert(securityManager.checkUIViewPermissions("user3") === false) + } + + test("set security with api") { + val conf = new SparkConf + conf.set("spark.ui.view.acls", "user1,user2") + val securityManager = new SecurityManager(conf); + securityManager.setUIAcls(true) + assert(securityManager.uiAclsEnabled() === true) + securityManager.setUIAcls(false) + assert(securityManager.uiAclsEnabled() === false) + + // acls are off so doesn't matter what view acls set to + assert(securityManager.checkUIViewPermissions("user4") === true) + + securityManager.setUIAcls(true) + assert(securityManager.uiAclsEnabled() === true) + securityManager.setViewAcls(ArrayBuffer[String]("user5"), "user6,user7") + assert(securityManager.checkUIViewPermissions("user1") === false) + assert(securityManager.checkUIViewPermissions("user5") === true) + assert(securityManager.checkUIViewPermissions("user6") === true) + assert(securityManager.checkUIViewPermissions("user7") === true) + assert(securityManager.checkUIViewPermissions("user8") === false) + assert(securityManager.checkUIViewPermissions(null) === true) + } +} + diff --git a/docs/monitoring.md b/docs/monitoring.md index 347a9b1f1a329..6f35fc37c4075 100644 --- a/docs/monitoring.md +++ b/docs/monitoring.md @@ -115,6 +115,19 @@ represents an application's event logs. This creates a web interface at Location of the kerberos keytab file for the History Server. + + spark.history.ui.acls.enable + false + + Specifies whether acls should be checked to authorize users viewing the applications. + If enabled, access control checks are made regardless of what the individual application had + set for spark.ui.acls.enable when the application was run. The application owner + will always have authorization to view their own application and any users specified via + spark.ui.view.acls when the application was run will also have authorization + to view that application. + If disabled, no access control checks are made. + + Note that in all of these UIs, the tables are sortable by clicking their headers, From 80429f3e2ab786d103297652922c3d8da3cf5a01 Mon Sep 17 00:00:00 2001 From: jerryshao Date: Thu, 24 Apr 2014 18:56:57 -0700 Subject: [PATCH 372/397] [SPARK-1510] Spark Streaming metrics source for metrics system This pulls in changes made by @jerryshao in https://github.com/apache/spark/pull/424 and merges with the master. Author: jerryshao Author: Tathagata Das Closes #545 from tdas/streaming-metrics and squashes the following commits: 034b443 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-metrics fb3b0a5 [jerryshao] Modify according master update 21939f5 [jerryshao] Style changes according to style check error 976116b [jerryshao] Add StreamSource in StreamingContext for better monitoring through metrics system --- .../spark/streaming/StreamingContext.scala | 4 + .../spark/streaming/StreamingSource.scala | 73 +++++++++++++++++++ .../ui/StreamingJobProgressListener.scala | 3 +- 3 files changed, 79 insertions(+), 1 deletion(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala index 1c89543058211..e0677b795cb94 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingContext.scala @@ -154,6 +154,10 @@ class StreamingContext private[streaming] ( private[streaming] val uiTab = new StreamingTab(this) + /** Register streaming source to metrics system */ + private val streamingSource = new StreamingSource(this) + SparkEnv.get.metricsSystem.registerSource(streamingSource) + /** Enumeration to identify current state of the StreamingContext */ private[streaming] object StreamingContextState extends Enumeration { type CheckpointState = Value diff --git a/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala new file mode 100644 index 0000000000000..774adc3c23c21 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/StreamingSource.scala @@ -0,0 +1,73 @@ +/* + * 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.streaming + +import com.codahale.metrics.{Gauge, MetricRegistry} + +import org.apache.spark.metrics.source.Source +import org.apache.spark.streaming.ui.StreamingJobProgressListener + +private[streaming] class StreamingSource(ssc: StreamingContext) extends Source { + val metricRegistry = new MetricRegistry + val sourceName = "%s.StreamingMetrics".format(ssc.sparkContext.appName) + + val streamingListener = ssc.uiTab.listener + + private def registerGauge[T](name: String, f: StreamingJobProgressListener => T, + defaultValue: T) { + metricRegistry.register(MetricRegistry.name("streaming", name), new Gauge[T] { + override def getValue: T = Option(f(streamingListener)).getOrElse(defaultValue) + }) + } + + // Gauge for number of network receivers + registerGauge("receivers", _.numReceivers, 0) + + // Gauge for number of total completed batches + registerGauge("totalCompletedBatches", _.numTotalCompletedBatches, 0L) + + // Gauge for number of unprocessed batches + registerGauge("unprocessedBatches", _.numUnprocessedBatches, 0L) + + // Gauge for number of waiting batches + registerGauge("waitingBatches", _.waitingBatches.size, 0L) + + // Gauge for number of running batches + registerGauge("runningBatches", _.runningBatches.size, 0L) + + // Gauge for number of retained completed batches + registerGauge("retainedCompletedBatches", _.retainedCompletedBatches.size, 0L) + + // Gauge for last completed batch, useful for monitoring the streaming job's running status, + // displayed data -1 for any abnormal condition. + registerGauge("lastCompletedBatch_submissionTime", + _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L) + registerGauge("lastCompletedBatch_processStartTime", + _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L) + registerGauge("lastCompletedBatch_processEndTime", + _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L) + + // Gauge for last received batch, useful for monitoring the streaming job's running status, + // displayed data -1 for any abnormal condition. + registerGauge("lastReceivedBatch_submissionTime", + _.lastCompletedBatch.map(_.submissionTime).getOrElse(-1L), -1L) + registerGauge("lastReceivedBatch_processStartTime", + _.lastCompletedBatch.flatMap(_.processingStartTime).getOrElse(-1L), -1L) + registerGauge("lastReceivedBatch_processEndTime", + _.lastCompletedBatch.flatMap(_.processingEndTime).getOrElse(-1L), -1L) +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index bf637c1446314..14c33c728bfe1 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -28,7 +28,8 @@ import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted import org.apache.spark.util.Distribution -private[ui] class StreamingJobProgressListener(ssc: StreamingContext) extends StreamingListener { +private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) + extends StreamingListener { private val waitingBatchInfos = new HashMap[Time, BatchInfo] private val runningBatchInfos = new HashMap[Time, BatchInfo] From ee6f7e22a449837864072e3cd2b6696005f134f1 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Apr 2014 20:18:15 -0700 Subject: [PATCH 373/397] [SPARK-1615] Synchronize accesses to the LiveListenerBus' event queue Original poster is @zsxwing, who reported this bug in #516. Much of SparkListenerSuite relies on LiveListenerBus's `waitUntilEmpty()` method. As the name suggests, this waits until the event queue is empty. However, the following race condition could happen: (1) We dequeue an event (2) The queue is empty, we return true (even though the event has not been processed) (3) The test asserts something assuming that all listeners have finished executing (and fails) (4) The listeners receive and process the event This PR makes (1) and (4) atomic by synchronizing around it. To do that, however, we must avoid using `eventQueue.take`, which is blocking and will cause a deadlock if we synchronize around it. As a workaround, we use the non-blocking `eventQueue.poll` + a semaphore to provide the same semantics. This has been a possible race condition for a long time, but for some reason we've never run into it. Author: Andrew Or Closes #544 from andrewor14/stage-info-test-fix and squashes the following commits: 3cbe40c [Andrew Or] Merge github.com:apache/spark into stage-info-test-fix 56dbbcb [Andrew Or] Check if event is actually added before releasing semaphore eb486ae [Andrew Or] Synchronize accesses to the LiveListenerBus' event queue --- .../spark/scheduler/LiveListenerBus.scala | 38 ++++++++++++++----- .../spark/scheduler/SparkListenerSuite.scala | 5 ++- 2 files changed, 31 insertions(+), 12 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala index cbac4c13ca6fe..dec3316bf7745 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/LiveListenerBus.scala @@ -17,7 +17,7 @@ package org.apache.spark.scheduler -import java.util.concurrent.LinkedBlockingQueue +import java.util.concurrent.{LinkedBlockingQueue, Semaphore} import org.apache.spark.Logging @@ -36,16 +36,24 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { private val eventQueue = new LinkedBlockingQueue[SparkListenerEvent](EVENT_QUEUE_CAPACITY) private var queueFullErrorMessageLogged = false private var started = false + + // A counter that represents the number of events produced and consumed in the queue + private val eventLock = new Semaphore(0) + private val listenerThread = new Thread("SparkListenerBus") { setDaemon(true) override def run() { while (true) { - val event = eventQueue.take - if (event == SparkListenerShutdown) { - // Get out of the while loop and shutdown the daemon thread - return + eventLock.acquire() + // Atomically remove and process this event + LiveListenerBus.this.synchronized { + val event = eventQueue.poll + if (event == SparkListenerShutdown) { + // Get out of the while loop and shutdown the daemon thread + return + } + Option(event).foreach(postToAll) } - postToAll(event) } } } @@ -67,7 +75,9 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { def post(event: SparkListenerEvent) { val eventAdded = eventQueue.offer(event) - if (!eventAdded && !queueFullErrorMessageLogged) { + if (eventAdded) { + eventLock.release() + } else if (!queueFullErrorMessageLogged) { logError("Dropping SparkListenerEvent because no remaining room in event queue. " + "This likely means one of the SparkListeners is too slow and cannot keep up with the " + "rate at which tasks are being started by the scheduler.") @@ -76,13 +86,13 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { } /** - * Waits until there are no more events in the queue, or until the specified time has elapsed. - * Used for testing only. Returns true if the queue has emptied and false is the specified time + * For testing only. Wait until there are no more events in the queue, or until the specified + * time has elapsed. Return true if the queue has emptied and false is the specified time * elapsed before the queue emptied. */ def waitUntilEmpty(timeoutMillis: Int): Boolean = { val finishTime = System.currentTimeMillis + timeoutMillis - while (!eventQueue.isEmpty) { + while (!queueIsEmpty) { if (System.currentTimeMillis > finishTime) { return false } @@ -93,6 +103,14 @@ private[spark] class LiveListenerBus extends SparkListenerBus with Logging { true } + /** + * Return whether the event queue is empty. + * + * The use of synchronized here guarantees that all events that once belonged to this queue + * have already been processed by all attached listeners, if this returns true. + */ + def queueIsEmpty: Boolean = synchronized { eventQueue.isEmpty } + def stop() { if (!started) { throw new IllegalStateException("Attempted to stop a listener bus that has not yet started!") diff --git a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala index ba048ced32a93..4e9fd07e68a21 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/SparkListenerSuite.scala @@ -29,7 +29,8 @@ import org.apache.spark.SparkContext._ import org.apache.spark.executor.TaskMetrics class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatchers - with BeforeAndAfter with BeforeAndAfterAll { + with BeforeAndAfter with BeforeAndAfterAll { + /** Length of time to wait while draining listener events. */ val WAIT_TIMEOUT_MILLIS = 10000 @@ -37,7 +38,7 @@ class SparkListenerSuite extends FunSuite with LocalSparkContext with ShouldMatc sc = new SparkContext("local", "SparkListenerSuite") } - override def afterAll { + override def afterAll() { System.clearProperty("spark.akka.frameSize") } From e53eb4f0159ebd4d72c4bbc3586fdfc66ccacab7 Mon Sep 17 00:00:00 2001 From: Ahir Reddy Date: Thu, 24 Apr 2014 20:21:10 -0700 Subject: [PATCH 374/397] [SPARK-986]: Job cancelation for PySpark * Additions to the PySpark API to cancel jobs * Monitor Thread in PythonRDD to kill Python workers if a task is interrupted Author: Ahir Reddy Closes #541 from ahirreddy/python-cancel and squashes the following commits: dfdf447 [Ahir Reddy] Changed success -> completed and made logging message clearer 6c860ab [Ahir Reddy] PR Comments 4b4100a [Ahir Reddy] Success flag adba6ed [Ahir Reddy] Destroy python workers 27a2f8f [Ahir Reddy] Start the writer thread... d422f7b [Ahir Reddy] Remove unnecesssary vals adda337 [Ahir Reddy] Busy wait on the ocntext.interrupted flag, and then kill the python worker d9e472f [Ahir Reddy] Revert "removed unnecessary vals" 5b9cae5 [Ahir Reddy] removed unnecessary vals 07b54d9 [Ahir Reddy] Fix canceling unit test 8ae9681 [Ahir Reddy] Don't interrupt worker 7722342 [Ahir Reddy] Monitor Thread for python workers db04e16 [Ahir Reddy] Added canceling api to PySpark --- .../scala/org/apache/spark/SparkEnv.scala | 8 +++ .../apache/spark/api/python/PythonRDD.scala | 30 ++++++++++- python/pyspark/context.py | 52 +++++++++++++++++-- 3 files changed, 86 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkEnv.scala b/core/src/main/scala/org/apache/spark/SparkEnv.scala index 915315ed74436..bea435ec34ce9 100644 --- a/core/src/main/scala/org/apache/spark/SparkEnv.scala +++ b/core/src/main/scala/org/apache/spark/SparkEnv.scala @@ -97,6 +97,14 @@ class SparkEnv ( pythonWorkers.getOrElseUpdate(key, new PythonWorkerFactory(pythonExec, envVars)).create() } } + + private[spark] + def destroyPythonWorker(pythonExec: String, envVars: Map[String, String]) { + synchronized { + val key = (pythonExec, envVars) + pythonWorkers(key).stop() + } + } } object SparkEnv extends Logging { diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 0d71fdbb03ec6..1498b017a793d 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -110,13 +110,41 @@ private[spark] class PythonRDD[T: ClassTag]( } }.start() + // Necessary to distinguish between a task that has failed and a task that is finished + @volatile var complete: Boolean = false + + // It is necessary to have a monitor thread for python workers if the user cancels with + // interrupts disabled. In that case we will need to explicitly kill the worker, otherwise the + // threads can block indefinitely. + new Thread(s"Worker Monitor for $pythonExec") { + override def run() { + // Kill the worker if it is interrupted or completed + // When a python task completes, the context is always set to interupted + while (!context.interrupted) { + Thread.sleep(2000) + } + if (!complete) { + try { + logWarning("Incomplete task interrupted: Attempting to kill Python Worker") + env.destroyPythonWorker(pythonExec, envVars.toMap) + } catch { + case e: Exception => + logError("Exception when trying to kill worker", e) + } + } + } + }.start() + /* * Partial fix for SPARK-1019: Attempts to stop reading the input stream since * other completion callbacks might invalidate the input. Because interruption * is not synchronous this still leaves a potential race where the interruption is * processed only after the stream becomes invalid. */ - context.addOnCompleteCallback(() => context.interrupted = true) + context.addOnCompleteCallback{ () => + complete = true // Indicate that the task has completed successfully + context.interrupted = true + } // Return an iterator that read lines from the process's stdout val stream = new DataInputStream(new BufferedInputStream(worker.getInputStream, bufferSize)) diff --git a/python/pyspark/context.py b/python/pyspark/context.py index f63cc4a55fb98..c74dc5fd4f854 100644 --- a/python/pyspark/context.py +++ b/python/pyspark/context.py @@ -429,7 +429,7 @@ def _getJavaStorageLevel(self, storageLevel): storageLevel.deserialized, storageLevel.replication) - def setJobGroup(self, groupId, description): + def setJobGroup(self, groupId, description, interruptOnCancel=False): """ Assigns a group ID to all the jobs started by this thread until the group ID is set to a different value or cleared. @@ -437,8 +437,41 @@ def setJobGroup(self, groupId, description): Often, a unit of execution in an application consists of multiple Spark actions or jobs. Application programmers can use this method to group all those jobs together and give a group description. Once set, the Spark web UI will associate such jobs with this group. - """ - self._jsc.setJobGroup(groupId, description) + + The application can use L{SparkContext.cancelJobGroup} to cancel all + running jobs in this group. + + >>> import thread, threading + >>> from time import sleep + >>> result = "Not Set" + >>> lock = threading.Lock() + >>> def map_func(x): + ... sleep(100) + ... return x * x + >>> def start_job(x): + ... global result + ... try: + ... sc.setJobGroup("job_to_cancel", "some description") + ... result = sc.parallelize(range(x)).map(map_func).collect() + ... except Exception as e: + ... result = "Cancelled" + ... lock.release() + >>> def stop_job(): + ... sleep(5) + ... sc.cancelJobGroup("job_to_cancel") + >>> supress = lock.acquire() + >>> supress = thread.start_new_thread(start_job, (10,)) + >>> supress = thread.start_new_thread(stop_job, tuple()) + >>> supress = lock.acquire() + >>> print result + Cancelled + + If interruptOnCancel is set to true for the job group, then job cancellation will result + in Thread.interrupt() being called on the job's executor threads. This is useful to help ensure + that the tasks are actually stopped in a timely manner, but is off by default due to HDFS-1208, + where HDFS may respond to Thread.interrupt() by marking nodes as dead. + """ + self._jsc.setJobGroup(groupId, description, interruptOnCancel) def setLocalProperty(self, key, value): """ @@ -460,6 +493,19 @@ def sparkUser(self): """ return self._jsc.sc().sparkUser() + def cancelJobGroup(self, groupId): + """ + Cancel active jobs for the specified group. See L{SparkContext.setJobGroup} + for more information. + """ + self._jsc.sc().cancelJobGroup(groupId) + + def cancelAllJobs(self): + """ + Cancel all jobs that have been scheduled or are running. + """ + self._jsc.sc().cancelAllJobs() + def _test(): import atexit import doctest From d5c6ae6cc3305b9aa3185486b5b6ba0a6e5aca90 Mon Sep 17 00:00:00 2001 From: tmalaska Date: Thu, 24 Apr 2014 20:31:17 -0700 Subject: [PATCH 375/397] SPARK-1584: Upgrade Flume dependency to 1.4.0 Updated the Flume dependency in the maven pom file and the scala build file. Author: tmalaska Closes #507 from tmalaska/master and squashes the following commits: 79492c8 [tmalaska] excluded all thrift 159c3f1 [tmalaska] fixed the flume pom file issues 5bf56a7 [tmalaska] Upgrade flume version --- external/flume/pom.xml | 6 +++++- project/SparkBuild.scala | 3 ++- 2 files changed, 7 insertions(+), 2 deletions(-) diff --git a/external/flume/pom.xml b/external/flume/pom.xml index f21963531574b..b84ca0c5e1dd0 100644 --- a/external/flume/pom.xml +++ b/external/flume/pom.xml @@ -61,12 +61,16 @@ org.apache.flume flume-ng-sdk - 1.2.0 + 1.4.0 org.jboss.netty netty + + org.apache.thrift + libthrift + diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index b8af2bbd2ef6a..9cbc188c4d485 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -313,6 +313,7 @@ object SparkBuild extends Build { val excludeCurator = ExclusionRule(organization = "org.apache.curator") val excludePowermock = ExclusionRule(organization = "org.powermock") val excludeFastutil = ExclusionRule(organization = "it.unimi.dsi") + val excludeThrift = ExclusionRule(organization = "org.apache.thrift") def sparkPreviousArtifact(id: String, organization: String = "org.apache.spark", version: String = "0.9.0-incubating", crossVersion: String = "2.10"): Option[sbt.ModuleID] = { @@ -607,7 +608,7 @@ object SparkBuild extends Build { name := "spark-streaming-flume", previousArtifact := sparkPreviousArtifact("spark-streaming-flume"), libraryDependencies ++= Seq( - "org.apache.flume" % "flume-ng-sdk" % "1.2.0" % "compile" excludeAll(excludeNetty) + "org.apache.flume" % "flume-ng-sdk" % "1.4.0" % "compile" excludeAll(excludeNetty, excludeThrift) ) ) From 968c0187a12f5ae4a696c02c1ff088e998ed7edd Mon Sep 17 00:00:00 2001 From: Mridul Muralidharan Date: Thu, 24 Apr 2014 20:48:33 -0700 Subject: [PATCH 376/397] SPARK-1586 Windows build fixes Unfortunately, this is not exhaustive - particularly hive tests still fail due to path issues. Author: Mridul Muralidharan This patch had conflicts when merged, resolved by Committer: Matei Zaharia Closes #505 from mridulm/windows_fixes and squashes the following commits: ef12283 [Mridul Muralidharan] Move to org.apache.commons.lang3 for StringEscapeUtils. Earlier version was buggy appparently cdae406 [Mridul Muralidharan] Remove leaked changes from > 2G fix branch 3267f4b [Mridul Muralidharan] Fix build failures 35b277a [Mridul Muralidharan] Fix Scalastyle failures bc69d14 [Mridul Muralidharan] Change from hardcoded path separator 10c4d78 [Mridul Muralidharan] Use explicit encoding while using getBytes 1337abd [Mridul Muralidharan] fix classpath while running in windows --- bin/compute-classpath.cmd | 157 ++++++++++-------- .../org/apache/spark/SparkSaslClient.scala | 6 +- .../org/apache/spark/SparkSaslServer.scala | 8 +- .../apache/spark/api/python/PythonRDD.scala | 2 +- .../apache/spark/network/ReceiverTest.scala | 2 +- .../org/apache/spark/network/SenderTest.scala | 2 +- .../scala/org/apache/spark/rdd/PipedRDD.scala | 8 +- .../scala/org/apache/spark/util/Utils.scala | 24 ++- .../java/org/apache/spark/JavaAPISuite.java | 4 +- .../streaming/examples/MQTTWordCount.scala | 2 +- .../streaming/flume/FlumeStreamSuite.scala | 2 +- .../streaming/mqtt/MQTTInputDStream.scala | 2 +- .../org/apache/spark/repl/ReplSuite.scala | 6 +- .../spark/sql/columnar/ColumnType.scala | 6 +- .../spark/sql/columnar/ColumnTypeSuite.scala | 44 ++++- .../spark/sql/hive/ScriptTransformation.scala | 2 +- .../org/apache/spark/sql/hive/TestHive.scala | 9 +- .../execution/BigDataBenchmarkSuite.scala | 2 +- .../hive/execution/HiveComparisonTest.scala | 5 +- .../execution/HiveCompatibilitySuite.scala | 6 +- .../deploy/yarn/YarnSparkHadoopUtil.scala | 2 +- 21 files changed, 185 insertions(+), 116 deletions(-) diff --git a/bin/compute-classpath.cmd b/bin/compute-classpath.cmd index 4f60bff19cb93..065553eb31939 100644 --- a/bin/compute-classpath.cmd +++ b/bin/compute-classpath.cmd @@ -1,69 +1,88 @@ -@echo off - -rem -rem Licensed to the Apache Software Foundation (ASF) under one or more -rem contributor license agreements. See the NOTICE file distributed with -rem this work for additional information regarding copyright ownership. -rem The ASF licenses this file to You under the Apache License, Version 2.0 -rem (the "License"); you may not use this file except in compliance with -rem the License. You may obtain a copy of the License at -rem -rem http://www.apache.org/licenses/LICENSE-2.0 -rem -rem Unless required by applicable law or agreed to in writing, software -rem distributed under the License is distributed on an "AS IS" BASIS, -rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -rem See the License for the specific language governing permissions and -rem limitations under the License. -rem - -rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" -rem script and the ExecutorRunner in standalone cluster mode. - -set SCALA_VERSION=2.10 - -rem Figure out where the Spark framework is installed -set FWDIR=%~dp0..\ - -rem Load environment variables from conf\spark-env.cmd, if it exists -if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" - -rem Build up classpath -set CLASSPATH=%FWDIR%conf -if exist "%FWDIR%RELEASE" ( - for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) else ( - for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( - set ASSEMBLY_JAR=%%d - ) -) -set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% - -if "x%SPARK_TESTING%"=="x1" ( - rem Add test clases to path - set CLASSPATH=%CLASSPATH%;%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes - set CLASSPATH=%CLASSPATH%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes -) - -rem Add hadoop conf dir - else FileSystem.*, etc fail -rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts -rem the configurtion files. -if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir - set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% -:no_hadoop_conf_dir - -if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir - set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% -:no_yarn_conf_dir - -rem A bit of a hack to allow calling this script within run2.cmd without seeing output -if "%DONT_PRINT_CLASSPATH%"=="1" goto exit - -echo %CLASSPATH% - -:exit +@echo off + +rem +rem Licensed to the Apache Software Foundation (ASF) under one or more +rem contributor license agreements. See the NOTICE file distributed with +rem this work for additional information regarding copyright ownership. +rem The ASF licenses this file to You under the Apache License, Version 2.0 +rem (the "License"); you may not use this file except in compliance with +rem the License. You may obtain a copy of the License at +rem +rem http://www.apache.org/licenses/LICENSE-2.0 +rem +rem Unless required by applicable law or agreed to in writing, software +rem distributed under the License is distributed on an "AS IS" BASIS, +rem WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +rem See the License for the specific language governing permissions and +rem limitations under the License. +rem + +rem This script computes Spark's classpath and prints it to stdout; it's used by both the "run" +rem script and the ExecutorRunner in standalone cluster mode. + +set SCALA_VERSION=2.10 + +rem Figure out where the Spark framework is installed +set FWDIR=%~dp0..\ + +rem Load environment variables from conf\spark-env.cmd, if it exists +if exist "%FWDIR%conf\spark-env.cmd" call "%FWDIR%conf\spark-env.cmd" + +rem Build up classpath +set CLASSPATH=%FWDIR%conf +if exist "%FWDIR%RELEASE" ( + for %%d in ("%FWDIR%jars\spark-assembly*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) else ( + for %%d in ("%FWDIR%assembly\target\scala-%SCALA_VERSION%\spark-assembly*hadoop*.jar") do ( + set ASSEMBLY_JAR=%%d + ) +) + +set CLASSPATH=%CLASSPATH%;%ASSEMBLY_JAR% + +set SPARK_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%tools\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\classes +set SPARK_CLASSES=%SPARK_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\classes + +set SPARK_TEST_CLASSES=%FWDIR%core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%repl\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%mllib\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%bagel\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%graphx\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%streaming\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\catalyst\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\core\target\scala-%SCALA_VERSION%\test-classes +set SPARK_TEST_CLASSES=%SPARK_TEST_CLASSES%;%FWDIR%sql\hive\target\scala-%SCALA_VERSION%\test-classes + +if "x%SPARK_TESTING%"=="x1" ( + rem Add test clases to path - note, add SPARK_CLASSES and SPARK_TEST_CLASSES before CLASSPATH + rem so that local compilation takes precedence over assembled jar + set CLASSPATH=%SPARK_CLASSES%;%SPARK_TEST_CLASSES%;%CLASSPATH% +) + +rem Add hadoop conf dir - else FileSystem.*, etc fail +rem Note, this assumes that there is either a HADOOP_CONF_DIR or YARN_CONF_DIR which hosts +rem the configurtion files. +if "x%HADOOP_CONF_DIR%"=="x" goto no_hadoop_conf_dir + set CLASSPATH=%CLASSPATH%;%HADOOP_CONF_DIR% +:no_hadoop_conf_dir + +if "x%YARN_CONF_DIR%"=="x" goto no_yarn_conf_dir + set CLASSPATH=%CLASSPATH%;%YARN_CONF_DIR% +:no_yarn_conf_dir + +rem A bit of a hack to allow calling this script within run2.cmd without seeing output +if "%DONT_PRINT_CLASSPATH%"=="1" goto exit + +echo %CLASSPATH% + +:exit diff --git a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala index 5b14c4291d91a..65003b6ac6a0a 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslClient.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslClient.scala @@ -111,10 +111,10 @@ private[spark] class SparkSaslClient(securityMgr: SecurityManager) extends Logg CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) private val secretKey = securityMgr.getSecretKey() - private val userPassword: Array[Char] = - SparkSaslServer.encodePassword(if (secretKey != null) secretKey.getBytes() else "".getBytes()) + private val userPassword: Array[Char] = SparkSaslServer.encodePassword( + if (secretKey != null) secretKey.getBytes("utf-8") else "".getBytes("utf-8")) /** * Implementation used to respond to SASL request from the server. diff --git a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala index 6161a6fb7ae85..f6b0a9132aca4 100644 --- a/core/src/main/scala/org/apache/spark/SparkSaslServer.scala +++ b/core/src/main/scala/org/apache/spark/SparkSaslServer.scala @@ -89,7 +89,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi extends CallbackHandler { private val userName: String = - SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes()) + SparkSaslServer.encodeIdentifier(securityMgr.getSaslUser().getBytes("utf-8")) override def handle(callbacks: Array[Callback]) { logDebug("In the sasl server callback handler") @@ -101,7 +101,7 @@ private[spark] class SparkSaslServer(securityMgr: SecurityManager) extends Loggi case pc: PasswordCallback => { logDebug("handle: SASL server callback: setting userPassword") val password: Array[Char] = - SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes()) + SparkSaslServer.encodePassword(securityMgr.getSecretKey().getBytes("utf-8")) pc.setPassword(password) } case rc: RealmCallback => { @@ -159,7 +159,7 @@ private[spark] object SparkSaslServer { * @return Base64-encoded string */ def encodeIdentifier(identifier: Array[Byte]): String = { - new String(Base64.encodeBase64(identifier)) + new String(Base64.encodeBase64(identifier), "utf-8") } /** @@ -168,7 +168,7 @@ private[spark] object SparkSaslServer { * @return password as a char array. */ def encodePassword(password: Array[Byte]): Array[Char] = { - new String(Base64.encodeBase64(password)).toCharArray() + new String(Base64.encodeBase64(password), "utf-8").toCharArray() } } diff --git a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala index 1498b017a793d..672c344a56597 100644 --- a/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala +++ b/core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala @@ -187,7 +187,7 @@ private[spark] class PythonRDD[T: ClassTag]( val exLength = stream.readInt() val obj = new Array[Byte](exLength) stream.readFully(obj) - throw new PythonException(new String(obj), readerException) + throw new PythonException(new String(obj, "utf-8"), readerException) case SpecialLengths.END_OF_DATA_SECTION => // We've finished the data section of the output, but we can still // read some accumulator updates: diff --git a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala index 9dc51e0d401f8..53a6038a9b59e 100644 --- a/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala +++ b/core/src/main/scala/org/apache/spark/network/ReceiverTest.scala @@ -28,7 +28,7 @@ private[spark] object ReceiverTest { manager.onReceiveMessage((msg: Message, id: ConnectionManagerId) => { /* println("Received [" + msg + "] from [" + id + "] at " + System.currentTimeMillis) */ - val buffer = ByteBuffer.wrap("response".getBytes) + val buffer = ByteBuffer.wrap("response".getBytes("utf-8")) Some(Message.createBufferMessage(buffer, msg.id)) }) Thread.currentThread.join() diff --git a/core/src/main/scala/org/apache/spark/network/SenderTest.scala b/core/src/main/scala/org/apache/spark/network/SenderTest.scala index 14c094c6177d5..b8ea7c2cff9a2 100644 --- a/core/src/main/scala/org/apache/spark/network/SenderTest.scala +++ b/core/src/main/scala/org/apache/spark/network/SenderTest.scala @@ -54,7 +54,7 @@ private[spark] object SenderTest { val responseStr = manager.sendMessageReliablySync(targetConnectionManagerId, dataMessage) .map { response => val buffer = response.asInstanceOf[BufferMessage].buffers(0) - new String(buffer.array) + new String(buffer.array, "utf-8") }.getOrElse("none") val finishTime = System.currentTimeMillis diff --git a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala index e441d4a40ccd2..5d77d37378458 100644 --- a/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/PipedRDD.scala @@ -87,10 +87,10 @@ private[spark] class PipedRDD[T: ClassTag]( // When spark.worker.separated.working.directory option is turned on, each // task will be run in separate directory. This should be resolve file // access conflict issue - val taskDirectory = "./tasks/" + java.util.UUID.randomUUID.toString + val taskDirectory = "tasks" + File.separator + java.util.UUID.randomUUID.toString var workInTaskDirectory = false logDebug("taskDirectory = " + taskDirectory) - if (separateWorkingDir == true) { + if (separateWorkingDir) { val currentDir = new File(".") logDebug("currentDir = " + currentDir.getAbsolutePath()) val taskDirFile = new File(taskDirectory) @@ -106,13 +106,13 @@ private[spark] class PipedRDD[T: ClassTag]( for (file <- currentDir.list(tasksDirFilter)) { val fileWithDir = new File(currentDir, file) Utils.symlink(new File(fileWithDir.getAbsolutePath()), - new File(taskDirectory + "/" + fileWithDir.getName())) + new File(taskDirectory + File.separator + fileWithDir.getName())) } pb.directory(taskDirFile) workInTaskDirectory = true } catch { case e: Exception => logError("Unable to setup task working directory: " + e.getMessage + - " (" + taskDirectory + ")") + " (" + taskDirectory + ")", e) } } diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 084a71c4caebd..8351f7156a5e4 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -30,6 +30,7 @@ import scala.io.Source import scala.reflect.ClassTag import com.google.common.io.Files +import org.apache.commons.lang.SystemUtils import com.google.common.util.concurrent.ThreadFactoryBuilder import org.apache.hadoop.fs.{FileSystem, FileUtil, Path} import org.json4s._ @@ -45,10 +46,13 @@ import org.apache.spark.serializer.{DeserializationStream, SerializationStream, */ private[spark] object Utils extends Logging { - val osName = System.getProperty("os.name") - val random = new Random() + def sparkBin(sparkHome: String, which: String): File = { + val suffix = if (SystemUtils.IS_OS_WINDOWS) ".cmd" else "" + new File(sparkHome + File.separator + "bin", which + suffix) + } + /** Serialize an object using Java serialization */ def serialize[T](o: T): Array[Byte] = { val bos = new ByteArrayOutputStream() @@ -605,7 +609,7 @@ private[spark] object Utils extends Logging { */ def isSymlink(file: File): Boolean = { if (file == null) throw new NullPointerException("File must not be null") - if (osName.startsWith("Windows")) return false + if (SystemUtils.IS_OS_WINDOWS) return false val fileInCanonicalDir = if (file.getParent() == null) { file } else { @@ -1008,10 +1012,18 @@ private[spark] object Utils extends Logging { if (dst.isAbsolute()) { throw new IOException("Destination must be relative") } - val linkCmd = if (osName.startsWith("Windows")) "copy" else "ln -sf" + var cmdSuffix = "" + val linkCmd = if (SystemUtils.IS_OS_WINDOWS) { + // refer to http://technet.microsoft.com/en-us/library/cc771254.aspx + cmdSuffix = " /s /e /k /h /y /i" + "cmd /c xcopy " + } else { + cmdSuffix = "" + "ln -sf " + } import scala.sys.process._ - (linkCmd + " " + src.getAbsolutePath() + " " + dst.getPath()) lines_! ProcessLogger(line => - (logInfo(line))) + (linkCmd + src.getAbsolutePath() + " " + dst.getPath() + cmdSuffix) lines_! + ProcessLogger(line => (logInfo(line))) } diff --git a/core/src/test/java/org/apache/spark/JavaAPISuite.java b/core/src/test/java/org/apache/spark/JavaAPISuite.java index 8d2e9f1846343..76c6f5af82a5e 100644 --- a/core/src/test/java/org/apache/spark/JavaAPISuite.java +++ b/core/src/test/java/org/apache/spark/JavaAPISuite.java @@ -610,8 +610,8 @@ public void textFiles() throws IOException { @Test public void wholeTextFiles() throws IOException { - byte[] content1 = "spark is easy to use.\n".getBytes(); - byte[] content2 = "spark is also easy to use.\n".getBytes(); + byte[] content1 = "spark is easy to use.\n".getBytes("utf-8"); + byte[] content2 = "spark is also easy to use.\n".getBytes("utf-8"); File tempDir = Files.createTempDir(); String tempDirName = tempDir.getAbsolutePath(); diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala index 3d3c27ed78204..62aef0fb47107 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/MQTTWordCount.scala @@ -56,7 +56,7 @@ object MQTTPublisher { val msg: String = "hello mqtt demo for spark streaming" while (true) { - val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes()) + val message: MqttMessage = new MqttMessage(String.valueOf(msg).getBytes("utf-8")) msgtopic.publish(message) println("Published data. topic: " + msgtopic.getName() + " Message: " + message) } diff --git a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala index 78603200d2d8a..dd287d0ef90a0 100644 --- a/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala +++ b/external/flume/src/test/scala/org/apache/spark/streaming/flume/FlumeStreamSuite.scala @@ -57,7 +57,7 @@ class FlumeStreamSuite extends TestSuiteBase { for (i <- 0 until input.size) { val event = new AvroFlumeEvent - event.setBody(ByteBuffer.wrap(input(i).toString.getBytes())) + event.setBody(ByteBuffer.wrap(input(i).toString.getBytes("utf-8"))) event.setHeaders(Map[CharSequence, CharSequence]("test" -> "header")) client.append(event) Thread.sleep(500) diff --git a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala index 0beee8b4157ae..77661f71ada21 100644 --- a/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala +++ b/external/mqtt/src/main/scala/org/apache/spark/streaming/mqtt/MQTTInputDStream.scala @@ -92,7 +92,7 @@ class MQTTReceiver( // Handles Mqtt message override def messageArrived(arg0: String, arg1: MqttMessage) { - store(new String(arg1.getPayload())) + store(new String(arg1.getPayload(),"utf-8")) } override def deliveryComplete(arg0: IMqttDeliveryToken) { diff --git a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala index 4155007c6d337..e33f4f9803054 100644 --- a/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala +++ b/repl/src/test/scala/org/apache/spark/repl/ReplSuite.scala @@ -25,6 +25,7 @@ import scala.collection.mutable.ArrayBuffer import com.google.common.io.Files import org.scalatest.FunSuite import org.apache.spark.SparkContext +import org.apache.commons.lang3.StringEscapeUtils class ReplSuite extends FunSuite { @@ -185,11 +186,12 @@ class ReplSuite extends FunSuite { out.close() val output = runInterpreter("local", """ - |var file = sc.textFile("%s/input").cache() + |var file = sc.textFile("%s").cache() |file.count() |file.count() |file.count() - """.stripMargin.format(tempDir.getAbsolutePath)) + """.stripMargin.format(StringEscapeUtils.escapeJava( + tempDir.getAbsolutePath + File.separator + "input"))) assertDoesNotContain("error:", output) assertDoesNotContain("Exception", output) assertContains("res0: Long = 3", output) diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala index 5be76890afe31..4cd52d8288137 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala @@ -200,10 +200,10 @@ private[sql] object SHORT extends NativeColumnType(ShortType, 6, 2) { } private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { - override def actualSize(v: String): Int = v.getBytes.length + 4 + override def actualSize(v: String): Int = v.getBytes("utf-8").length + 4 override def append(v: String, buffer: ByteBuffer) { - val stringBytes = v.getBytes() + val stringBytes = v.getBytes("utf-8") buffer.putInt(stringBytes.length).put(stringBytes, 0, stringBytes.length) } @@ -211,7 +211,7 @@ private[sql] object STRING extends NativeColumnType(StringType, 7, 8) { val length = buffer.getInt() val stringBytes = new Array[Byte](length) buffer.get(stringBytes, 0, length) - new String(stringBytes) + new String(stringBytes, "utf-8") } override def setField(row: MutableRow, ordinal: Int, value: String) { diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala index 1d3608ed2d9ab..325173cf95fdf 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala @@ -58,7 +58,7 @@ class ColumnTypeSuite extends FunSuite { checkActualSize(DOUBLE, Double.MaxValue, 8) checkActualSize(FLOAT, Float.MaxValue, 4) checkActualSize(BOOLEAN, true, 1) - checkActualSize(STRING, "hello", 4 + 5) + checkActualSize(STRING, "hello", 4 + "hello".getBytes("utf-8").length) val binary = Array.fill[Byte](4)(0: Byte) checkActualSize(BINARY, binary, 4 + 4) @@ -91,14 +91,16 @@ class ColumnTypeSuite extends FunSuite { testNativeColumnType[StringType.type]( STRING, (buffer: ByteBuffer, string: String) => { - val bytes = string.getBytes() - buffer.putInt(bytes.length).put(string.getBytes) + + val bytes = string.getBytes("utf-8") + buffer.putInt(bytes.length) + buffer.put(bytes) }, (buffer: ByteBuffer) => { val length = buffer.getInt() val bytes = new Array[Byte](length) - buffer.get(bytes, 0, length) - new String(bytes) + buffer.get(bytes) + new String(bytes, "utf-8") }) testColumnType[BinaryType.type, Array[Byte]]( @@ -161,9 +163,13 @@ class ColumnTypeSuite extends FunSuite { buffer.rewind() seq.foreach { expected => + println("buffer = " + buffer + ", expected = " + expected) + val extracted = columnType.extract(buffer) assert( - expected === columnType.extract(buffer), - "Extracted value didn't equal to the original one") + expected === extracted, + "Extracted value didn't equal to the original one. " + + hexDump(expected) + " != " + hexDump(extracted) + + ", buffer = " + dumpBuffer(buffer.duplicate().rewind().asInstanceOf[ByteBuffer])) } } @@ -179,4 +185,28 @@ class ColumnTypeSuite extends FunSuite { } } } + + private def hexDump(value: Any): String = { + if (value.isInstanceOf[String]) { + val sb = new StringBuilder() + for (ch <- value.asInstanceOf[String].toCharArray) { + sb.append(Integer.toHexString(ch & 0xffff)).append(' ') + } + if (! sb.isEmpty) sb.setLength(sb.length - 1) + sb.toString() + } else { + // for now .. + hexDump(value.toString) + } + } + + private def dumpBuffer(buff: ByteBuffer): Any = { + val sb = new StringBuilder() + while (buff.hasRemaining) { + val b = buff.get() + sb.append(Integer.toHexString(b & 0xff)).append(' ') + } + if (! sb.isEmpty) sb.setLength(sb.length - 1) + sb.toString() + } } 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 610fa9cb843c0..8258ee5fef0eb 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 @@ -71,7 +71,7 @@ case class ScriptTransformation( iter .map(outputProjection) // TODO: Use SerDe - .map(_.mkString("", "\t", "\n").getBytes).foreach(outputStream.write) + .map(_.mkString("", "\t", "\n").getBytes("utf-8")).foreach(outputStream.write) outputStream.close() readerThread.join() outputLines.toIterator diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala index 74110ee27b76c..3ad66a3d7f45f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala @@ -100,14 +100,15 @@ class TestHiveContext(sc: SparkContext) extends LocalHiveContext(sc) { hiveFilesTemp.delete() hiveFilesTemp.mkdir() - val inRepoTests = if (System.getProperty("user.dir").endsWith("sql/hive")) { - new File("src/test/resources/") + val inRepoTests = if (System.getProperty("user.dir").endsWith("sql" + File.separator + "hive")) { + new File("src" + File.separator + "test" + File.separator + "resources" + File.separator) } else { - new File("sql/hive/src/test/resources") + new File("sql" + File.separator + "hive" + File.separator + "src" + File.separator + "test" + + File.separator + "resources") } def getHiveFile(path: String): File = { - val stripped = path.replaceAll("""\.\.\/""", "") + val stripped = path.replaceAll("""\.\.\/""", "").replace('/', File.separatorChar) hiveDevHome .map(new File(_, stripped)) .filter(_.exists) diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala index 9b9a823b6e95b..42a82c1fbf5c7 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala @@ -26,7 +26,7 @@ import org.apache.spark.sql.hive.test.TestHive._ * https://amplab.cs.berkeley.edu/benchmark/ */ class BigDataBenchmarkSuite extends HiveComparisonTest { - val testDataDirectory = new File("target/big-data-benchmark-testdata") + val testDataDirectory = new File("target" + File.separator + "big-data-benchmark-testdata") val testTables = Seq( TestTable( 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 ea17e6e93b41c..edff38b901073 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 @@ -78,7 +78,8 @@ abstract class HiveComparisonTest .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/resources/golden") + protected val answerCache = new File("src" + File.separator + "test" + + File.separator + "resources" + File.separator + "golden") if (!answerCache.exists) { answerCache.mkdir() } @@ -120,7 +121,7 @@ abstract class HiveComparisonTest protected val cacheDigest = java.security.MessageDigest.getInstance("MD5") protected def getMd5(str: String): String = { val digest = java.security.MessageDigest.getInstance("MD5") - digest.update(str.getBytes) + digest.update(str.getBytes("utf-8")) new java.math.BigInteger(1, digest.digest).toString(16) } diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala index dfe88b960b639..0bb76f31c373d 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala @@ -17,6 +17,8 @@ package org.apache.spark.sql.hive.execution +import java.io.File + import org.scalatest.BeforeAndAfter import org.apache.spark.sql.hive.test.TestHive @@ -26,7 +28,9 @@ import org.apache.spark.sql.hive.test.TestHive */ class HiveCompatibilitySuite extends HiveQueryFileTest with BeforeAndAfter { // TODO: bundle in jar files... get from classpath - lazy val hiveQueryDir = TestHive.getHiveFile("ql/src/test/queries/clientpositive") + lazy val hiveQueryDir = TestHive.getHiveFile("ql" + File.separator + "src" + + File.separator + "test" + File.separator + "queries" + File.separator + "clientpositive") + def testCases = hiveQueryDir.listFiles.map(f => f.getName.stripSuffix(".q") -> f) override def beforeAll() { diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala index 832d45b3ad10e..718cb19f57261 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/YarnSparkHadoopUtil.scala @@ -64,7 +64,7 @@ class YarnSparkHadoopUtil extends SparkHadoopUtil { override def addSecretKeyToUserCredentials(key: String, secret: String) { val creds = new Credentials() - creds.addSecretKey(new Text(key), secret.getBytes()) + creds.addSecretKey(new Text(key), secret.getBytes("utf-8")) addCurrentUserCredentials(creds) } From cd12dd9bde91303d0341180e5f70d2a03d6b65b6 Mon Sep 17 00:00:00 2001 From: Tathagata Das Date: Thu, 24 Apr 2014 21:34:37 -0700 Subject: [PATCH 377/397] [SPARK-1617] and [SPARK-1618] Improvements to streaming ui and bug fix to socket receiver 1617: These changes expose the receiver state (active or inactive) and last error in the UI 1618: If the socket receiver cannot connect in the first attempt, it should try to restart after a delay. That was broken, as the thread that restarts (hence, stops) the receiver waited on Thread.join on itself! Author: Tathagata Das Closes #540 from tdas/streaming-ui-fix and squashes the following commits: e469434 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into streaming-ui-fix dbddf75 [Tathagata Das] Style fix. 66df1a5 [Tathagata Das] Merge remote-tracking branch 'apache/master' into streaming-ui-fix ad98bc9 [Tathagata Das] Refactored streaming listener to use ReceiverInfo. d7f849c [Tathagata Das] Revert "Moved BatchInfo from streaming.scheduler to streaming.ui" 5c80919 [Tathagata Das] Moved BatchInfo from streaming.scheduler to streaming.ui da244f6 [Tathagata Das] Fixed socket receiver as well as made receiver state and error visible in the streamign UI. --- .../scala/org/apache/spark/ui/UIUtils.scala | 2 +- .../dstream/SocketInputDStream.scala | 49 +++++++--------- .../streaming/receiver/ActorReceiver.scala | 12 +++- .../spark/streaming/receiver/Receiver.scala | 5 +- .../streaming/receiver/ReceiverMessage.scala | 4 +- .../receiver/ReceiverSupervisor.scala | 58 +++++++++++-------- .../receiver/ReceiverSupervisorImpl.scala | 24 +++++--- .../spark/streaming/scheduler/BatchInfo.scala | 3 + .../streaming/scheduler/ReceiverInfo.scala | 37 ++++++++++++ .../streaming/scheduler/ReceiverTracker.scala | 40 ++++++++----- .../scheduler/StreamingListener.scala | 25 +++++++- .../ui/StreamingJobProgressListener.scala | 18 +++++- .../spark/streaming/ui/StreamingPage.scala | 20 +++++-- .../streaming/NetworkReceiverSuite.scala | 8 ++- .../streaming/StreamingListenerSuite.scala | 15 ++--- 15 files changed, 217 insertions(+), 103 deletions(-) create mode 100644 streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala diff --git a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala index cf987a1ab02c3..a3d6a1821245b 100644 --- a/core/src/main/scala/org/apache/spark/ui/UIUtils.scala +++ b/core/src/main/scala/org/apache/spark/ui/UIUtils.scala @@ -122,7 +122,7 @@ private[spark] object UIUtils extends Logging { } } if (unit.isEmpty) { - "%d".formatLocal(Locale.US, value) + "%d".formatLocal(Locale.US, value.toInt) } else { "%.1f%s".formatLocal(Locale.US, value, unit) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala index 1e32727eacfa3..8b72bcf20653d 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/SocketInputDStream.scala @@ -50,49 +50,42 @@ class SocketReceiver[T: ClassTag]( storageLevel: StorageLevel ) extends Receiver[T](storageLevel) with Logging { - var socket: Socket = null - var receivingThread: Thread = null - def onStart() { - receivingThread = new Thread("Socket Receiver") { - override def run() { - connect() - receive() - } - } - receivingThread.start() + // Start the thread that receives data over a connection + new Thread("Socket Receiver") { + setDaemon(true) + override def run() { receive() } + }.start() } def onStop() { - if (socket != null) { - socket.close() - } - socket = null - if (receivingThread != null) { - receivingThread.join() - } + // There is nothing much to do as the thread calling receive() + // is designed to stop by itself isStopped() returns false } - def connect() { + /** Create a socket connection and receive data until receiver is stopped */ + def receive() { + var socket: Socket = null try { logInfo("Connecting to " + host + ":" + port) socket = new Socket(host, port) - } catch { - case e: Exception => - restart("Could not connect to " + host + ":" + port, e) - } - } - - def receive() { - try { logInfo("Connected to " + host + ":" + port) val iterator = bytesToObjects(socket.getInputStream()) while(!isStopped && iterator.hasNext) { store(iterator.next) } + logInfo("Stopped receiving") + restart("Retrying connecting to " + host + ":" + port) } catch { - case e: Exception => - restart("Error receiving data from socket", e) + case e: java.net.ConnectException => + restart("Error connecting to " + host + ":" + port, e) + case t: Throwable => + restart("Error receiving data", t) + } finally { + if (socket != null) { + socket.close() + logInfo("Closed socket to " + host + ":" + port) + } } } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala index 821cf19481d44..743be58950c09 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ActorReceiver.scala @@ -28,8 +28,13 @@ import akka.actor.SupervisorStrategy.{Escalate, Restart} import org.apache.spark.{Logging, SparkEnv} import org.apache.spark.storage.StorageLevel import java.nio.ByteBuffer +import org.apache.spark.annotation.DeveloperApi -/** A helper with set of defaults for supervisor strategy */ +/** + * :: DeveloperApi :: + * A helper with set of defaults for supervisor strategy + */ +@DeveloperApi object ActorSupervisorStrategy { val defaultStrategy = OneForOneStrategy(maxNrOfRetries = 10, withinTimeRange = @@ -40,6 +45,7 @@ object ActorSupervisorStrategy { } /** + * :: DeveloperApi :: * A receiver trait to be mixed in with your Actor to gain access to * the API for pushing received data into Spark Streaming for being processed. * @@ -61,6 +67,7 @@ object ActorSupervisorStrategy { * to ensure the type safety, i.e parametrized type of push block and InputDStream * should be same. */ +@DeveloperApi trait ActorHelper { self: Actor => // to ensure that this can be added to Actor classes only @@ -92,10 +99,12 @@ trait ActorHelper { } /** + * :: DeveloperApi :: * Statistics for querying the supervisor about state of workers. Used in * conjunction with `StreamingContext.actorStream` and * [[org.apache.spark.streaming.receiver.ActorHelper]]. */ +@DeveloperApi case class Statistics(numberOfMsgs: Int, numberOfWorkers: Int, numberOfHiccups: Int, @@ -188,4 +197,3 @@ private[streaming] class ActorReceiver[T: ClassTag]( supervisor ! PoisonPill } } - diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala index 44eecf1dd2567..524c1b8d8ce46 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/Receiver.scala @@ -23,8 +23,10 @@ import scala.collection.mutable.ArrayBuffer import scala.collection.JavaConversions._ import org.apache.spark.storage.StorageLevel +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * Abstract class of a receiver that can be run on worker nodes to receive external data. A * custom receiver can be defined by defining the functions onStart() and onStop(). onStart() * should define the setup steps necessary to start receiving data, @@ -51,6 +53,7 @@ import org.apache.spark.storage.StorageLevel * } * }}} */ +@DeveloperApi abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable { /** @@ -198,7 +201,7 @@ abstract class Receiver[T](val storageLevel: StorageLevel) extends Serializable /** Check if receiver has been marked for stopping. */ def isStopped(): Boolean = { - !executor.isReceiverStarted() + executor.isReceiverStopped() } /** Get unique identifier of this receiver. */ diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala index 6ab3ca6ea5fa6..bf39d1e891cae 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverMessage.scala @@ -18,6 +18,6 @@ package org.apache.spark.streaming.receiver /** Messages sent to the NetworkReceiver. */ -private[streaming] sealed trait NetworkReceiverMessage -private[streaming] object StopReceiver extends NetworkReceiverMessage +private[streaming] sealed trait ReceiverMessage +private[streaming] object StopReceiver extends ReceiverMessage diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala index 256b3335e49aa..09be3a50d2dfa 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisor.scala @@ -88,15 +88,29 @@ private[streaming] abstract class ReceiverSupervisor( /** Report errors. */ def reportError(message: String, throwable: Throwable) - /** Start the executor */ + /** Called when supervisor is started */ + protected def onStart() { } + + /** Called when supervisor is stopped */ + protected def onStop(message: String, error: Option[Throwable]) { } + + /** Called when receiver is started */ + protected def onReceiverStart() { } + + /** Called when receiver is stopped */ + protected def onReceiverStop(message: String, error: Option[Throwable]) { } + + /** Start the supervisor */ def start() { + onStart() startReceiver() } - /** Mark the executor and the receiver for stopping */ + /** Mark the supervisor and the receiver for stopping */ def stop(message: String, error: Option[Throwable]) { stoppingError = error.orNull stopReceiver(message, error) + onStop(message, error) stopLatch.countDown() } @@ -104,6 +118,8 @@ private[streaming] abstract class ReceiverSupervisor( def startReceiver(): Unit = synchronized { try { logInfo("Starting receiver") + receiver.onStart() + logInfo("Called receiver onStart") onReceiverStart() receiverState = Started } catch { @@ -115,7 +131,10 @@ private[streaming] abstract class ReceiverSupervisor( /** Stop receiver */ def stopReceiver(message: String, error: Option[Throwable]): Unit = synchronized { try { + logInfo("Stopping receiver with message: " + message + ": " + error.getOrElse("")) receiverState = Stopped + receiver.onStop() + logInfo("Called receiver onStop") onReceiverStop(message, error) } catch { case t: Throwable => @@ -130,41 +149,32 @@ private[streaming] abstract class ReceiverSupervisor( /** Restart receiver with delay */ def restartReceiver(message: String, error: Option[Throwable], delay: Int) { - logWarning("Restarting receiver with delay " + delay + " ms: " + message, - error.getOrElse(null)) - stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) - future { + Future { + logWarning("Restarting receiver with delay " + delay + " ms: " + message, + error.getOrElse(null)) + stopReceiver("Restarting receiver with delay " + delay + "ms: " + message, error) logDebug("Sleeping for " + delay) Thread.sleep(delay) - logDebug("Starting receiver again") + logInfo("Starting receiver again") startReceiver() logInfo("Receiver started again") } } - /** Called when the receiver needs to be started */ - protected def onReceiverStart(): Unit = synchronized { - // Call user-defined onStart() - logInfo("Calling receiver onStart") - receiver.onStart() - logInfo("Called receiver onStart") - } - - /** Called when the receiver needs to be stopped */ - protected def onReceiverStop(message: String, error: Option[Throwable]): Unit = synchronized { - // Call user-defined onStop() - logInfo("Calling receiver onStop") - receiver.onStop() - logInfo("Called receiver onStop") - } - /** Check if receiver has been marked for stopping */ def isReceiverStarted() = { logDebug("state = " + receiverState) receiverState == Started } - /** Wait the thread until the executor is stopped */ + /** Check if receiver has been marked for stopping */ + def isReceiverStopped() = { + logDebug("state = " + receiverState) + receiverState == Stopped + } + + + /** Wait the thread until the supervisor is stopped */ def awaitTermination() { stopLatch.await() logInfo("Waiting for executor stop is over") diff --git a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala index 2a3521bd46ae7..ce8316bb14891 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/receiver/ReceiverSupervisorImpl.scala @@ -79,6 +79,8 @@ private[streaming] class ReceiverSupervisorImpl( logInfo("Received stop signal") stop("Stopped by driver", None) } + + def ref = self }), "Receiver-" + streamId + "-" + System.currentTimeMillis()) /** Unique block ids if one wants to add blocks directly */ @@ -154,14 +156,23 @@ private[streaming] class ReceiverSupervisorImpl( logWarning("Reported error " + message + " - " + error) } - override def onReceiverStart() { + override protected def onStart() { blockGenerator.start() - super.onReceiverStart() } - override def onReceiverStop(message: String, error: Option[Throwable]) { - super.onReceiverStop(message, error) + override protected def onStop(message: String, error: Option[Throwable]) { blockGenerator.stop() + env.actorSystem.stop(actor) + } + + override protected def onReceiverStart() { + val msg = RegisterReceiver( + streamId, receiver.getClass.getSimpleName, Utils.localHostName(), actor) + val future = trackerActor.ask(msg)(askTimeout) + Await.result(future, askTimeout) + } + + override protected def onReceiverStop(message: String, error: Option[Throwable]) { logInfo("Deregistering receiver " + streamId) val errorString = error.map(Throwables.getStackTraceAsString).getOrElse("") val future = trackerActor.ask( @@ -170,11 +181,6 @@ private[streaming] class ReceiverSupervisorImpl( logInfo("Stopped receiver " + streamId) } - override def stop(message: String, error: Option[Throwable]) { - super.stop(message, error) - env.actorSystem.stop(actor) - } - /** Generate new block ID */ private def nextBlockId = StreamBlockId(streamId, newBlockId.getAndIncrement) } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala index 9c69a2a4e21f5..a68aecb881117 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/BatchInfo.scala @@ -18,8 +18,10 @@ package org.apache.spark.streaming.scheduler import org.apache.spark.streaming.Time +import org.apache.spark.annotation.DeveloperApi /** + * :: DeveloperApi :: * Class having information on completed batches. * @param batchTime Time of the batch * @param submissionTime Clock time of when jobs of this batch was submitted to @@ -27,6 +29,7 @@ import org.apache.spark.streaming.Time * @param processingStartTime Clock time of when the first job of this batch started processing * @param processingEndTime Clock time of when the last job of this batch finished processing */ +@DeveloperApi case class BatchInfo( batchTime: Time, receivedBlockInfo: Map[Int, Array[ReceivedBlockInfo]], diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.scala new file mode 100644 index 0000000000000..d7e39c528c519 --- /dev/null +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverInfo.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.streaming.scheduler + +import akka.actor.ActorRef +import org.apache.spark.annotation.DeveloperApi + +/** + * :: DeveloperApi :: + * Class having information about a receiver + */ +@DeveloperApi +case class ReceiverInfo( + streamId: Int, + name: String, + private[streaming] val actor: ActorRef, + active: Boolean, + location: String, + lastErrorMessage: String = "", + lastError: String = "" + ) { +} diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala index 557e0961d5944..5307fe189d717 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/ReceiverTracker.scala @@ -28,13 +28,8 @@ import org.apache.spark.streaming.{StreamingContext, Time} import org.apache.spark.streaming.receiver.{Receiver, ReceiverSupervisorImpl, StopReceiver} import org.apache.spark.util.AkkaUtils -/** Information about receiver */ -case class ReceiverInfo(streamId: Int, typ: String, location: String) { - override def toString = s"$typ-$streamId" -} - /** Information about blocks received by the receiver */ -case class ReceivedBlockInfo( +private[streaming] case class ReceivedBlockInfo( streamId: Int, blockId: StreamBlockId, numRecords: Long, @@ -69,7 +64,7 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { val receiverInputStreams = ssc.graph.getReceiverInputStreams() val receiverInputStreamMap = Map(receiverInputStreams.map(x => (x.id, x)): _*) val receiverExecutor = new ReceiverLauncher() - val receiverInfo = new HashMap[Int, ActorRef] with SynchronizedMap[Int, ActorRef] + val receiverInfo = new HashMap[Int, ReceiverInfo] with SynchronizedMap[Int, ReceiverInfo] val receivedBlockInfo = new HashMap[Int, SynchronizedQueue[ReceivedBlockInfo]] with SynchronizedMap[Int, SynchronizedQueue[ReceivedBlockInfo]] val timeout = AkkaUtils.askTimeout(ssc.conf) @@ -129,17 +124,23 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { if (!receiverInputStreamMap.contains(streamId)) { throw new Exception("Register received for unexpected id " + streamId) } - receiverInfo += ((streamId, receiverActor)) - ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted( - ReceiverInfo(streamId, typ, host) - )) + receiverInfo(streamId) = ReceiverInfo( + streamId, s"${typ}-${streamId}", receiverActor, true, host) + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStarted(receiverInfo(streamId))) logInfo("Registered receiver for stream " + streamId + " from " + sender.path.address) } /** Deregister a receiver */ def deregisterReceiver(streamId: Int, message: String, error: String) { - receiverInfo -= streamId - ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(streamId, message, error)) + val newReceiverInfo = receiverInfo.get(streamId) match { + case Some(oldInfo) => + oldInfo.copy(actor = null, active = false, lastErrorMessage = message, lastError = error) + case None => + logWarning("No prior receiver info") + ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) + } + receiverInfo(streamId) = newReceiverInfo + ssc.scheduler.listenerBus.post(StreamingListenerReceiverStopped(receiverInfo(streamId))) val messageWithError = if (error != null && !error.isEmpty) { s"$message - $error" } else { @@ -157,7 +158,15 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { /** Report error sent by a receiver */ def reportError(streamId: Int, message: String, error: String) { - ssc.scheduler.listenerBus.post(StreamingListenerReceiverError(streamId, message, error)) + val newReceiverInfo = receiverInfo.get(streamId) match { + case Some(oldInfo) => + oldInfo.copy(lastErrorMessage = message, lastError = error) + case None => + logWarning("No prior receiver info") + ReceiverInfo(streamId, "", null, false, "", lastErrorMessage = message, lastError = error) + } + receiverInfo(streamId) = newReceiverInfo + ssc.scheduler.listenerBus.post(StreamingListenerReceiverError(receiverInfo(streamId))) val messageWithError = if (error != null && !error.isEmpty) { s"$message - $error" } else { @@ -270,7 +279,8 @@ class ReceiverTracker(ssc: StreamingContext) extends Logging { /** Stops the receivers. */ private def stopReceivers() { // Signal the receivers to stop - receiverInfo.values.foreach(_ ! StopReceiver) + receiverInfo.values.flatMap { info => Option(info.actor)} + .foreach { _ ! StopReceiver } logInfo("Sent stop signal to all " + receiverInfo.size + " receivers") } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala index 9d6ec1fa33549..ed1aa114e19d9 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/scheduler/StreamingListener.scala @@ -20,28 +20,45 @@ package org.apache.spark.streaming.scheduler import scala.collection.mutable.Queue import org.apache.spark.util.Distribution +import org.apache.spark.annotation.DeveloperApi -/** Base trait for events related to StreamingListener */ +/** + * :: DeveloperApi :: + * Base trait for events related to StreamingListener + */ +@DeveloperApi sealed trait StreamingListenerEvent +@DeveloperApi case class StreamingListenerBatchSubmitted(batchInfo: BatchInfo) extends StreamingListenerEvent + +@DeveloperApi case class StreamingListenerBatchCompleted(batchInfo: BatchInfo) extends StreamingListenerEvent + +@DeveloperApi case class StreamingListenerBatchStarted(batchInfo: BatchInfo) extends StreamingListenerEvent +@DeveloperApi case class StreamingListenerReceiverStarted(receiverInfo: ReceiverInfo) extends StreamingListenerEvent -case class StreamingListenerReceiverError(streamId: Int, message: String, error: String) + +@DeveloperApi +case class StreamingListenerReceiverError(receiverInfo: ReceiverInfo) extends StreamingListenerEvent -case class StreamingListenerReceiverStopped(streamId: Int, message: String, error: String) + +@DeveloperApi +case class StreamingListenerReceiverStopped(receiverInfo: ReceiverInfo) extends StreamingListenerEvent /** An event used in the listener to shutdown the listener daemon thread. */ private[scheduler] case object StreamingListenerShutdown extends StreamingListenerEvent /** + * :: DeveloperApi :: * A listener interface for receiving information about an ongoing streaming * computation. */ +@DeveloperApi trait StreamingListener { /** Called when a receiver has been started */ @@ -65,9 +82,11 @@ trait StreamingListener { /** + * :: DeveloperApi :: * A simple StreamingListener that logs summary statistics across Spark Streaming batches * @param numBatchInfos Number of last batches to consider for generating statistics (default: 10) */ +@DeveloperApi class StatsReportListener(numBatchInfos: Int = 10) extends StreamingListener { // Queue containing latest completed batches val batchInfos = new Queue[BatchInfo]() diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala index 14c33c728bfe1..f61069b56db5e 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingJobProgressListener.scala @@ -23,9 +23,9 @@ import scala.collection.mutable.{Queue, HashMap} import org.apache.spark.streaming.scheduler.StreamingListenerReceiverStarted import org.apache.spark.streaming.scheduler.StreamingListenerBatchStarted import org.apache.spark.streaming.scheduler.BatchInfo -import org.apache.spark.streaming.scheduler.ReceiverInfo import org.apache.spark.streaming.scheduler.StreamingListenerBatchSubmitted import org.apache.spark.util.Distribution +import org.apache.spark.Logging private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) @@ -40,9 +40,21 @@ private[streaming] class StreamingJobProgressListener(ssc: StreamingContext) val batchDuration = ssc.graph.batchDuration.milliseconds - override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) = { + override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { synchronized { - receiverInfos.put(receiverStarted.receiverInfo.streamId, receiverStarted.receiverInfo) + receiverInfos(receiverStarted.receiverInfo.streamId) = receiverStarted.receiverInfo + } + } + + override def onReceiverError(receiverError: StreamingListenerReceiverError) { + synchronized { + receiverInfos(receiverError.receiverInfo.streamId) = receiverError.receiverInfo + } + } + + override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { + synchronized { + receiverInfos(receiverStopped.receiverInfo.streamId) = receiverStopped.receiverInfo } } diff --git a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala index 8fe1219356cdc..451b23e01c995 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/ui/StreamingPage.scala @@ -78,25 +78,33 @@ private[ui] class StreamingPage(parent: StreamingTab) val table = if (receivedRecordDistributions.size > 0) { val headerRow = Seq( "Receiver", + "Status", "Location", "Records in last batch\n[" + formatDate(Calendar.getInstance().getTime()) + "]", "Minimum rate\n[records/sec]", - "25th percentile rate\n[records/sec]", "Median rate\n[records/sec]", - "75th percentile rate\n[records/sec]", - "Maximum rate\n[records/sec]" + "Maximum rate\n[records/sec]", + "Last Error" ) val dataRows = (0 until listener.numReceivers).map { receiverId => val receiverInfo = listener.receiverInfo(receiverId) - val receiverName = receiverInfo.map(_.toString).getOrElse(s"Receiver-$receiverId") + val receiverName = receiverInfo.map(_.name).getOrElse(s"Receiver-$receiverId") + val receiverActive = receiverInfo.map { info => + if (info.active) "ACTIVE" else "INACTIVE" + }.getOrElse(emptyCell) val receiverLocation = receiverInfo.map(_.location).getOrElse(emptyCell) val receiverLastBatchRecords = formatNumber(lastBatchReceivedRecord(receiverId)) val receivedRecordStats = receivedRecordDistributions(receiverId).map { d => - d.getQuantiles().map(r => formatNumber(r.toLong)) + d.getQuantiles(Seq(0.0, 0.5, 1.0)).map(r => formatNumber(r.toLong)) }.getOrElse { Seq(emptyCell, emptyCell, emptyCell, emptyCell, emptyCell) } - Seq(receiverName, receiverLocation, receiverLastBatchRecords) ++ receivedRecordStats + val receiverLastError = listener.receiverInfo(receiverId).map { info => + val msg = s"${info.lastErrorMessage} - ${info.lastError}" + if (msg.size > 100) msg.take(97) + "..." else msg + }.getOrElse(emptyCell) + Seq(receiverName, receiverActive, receiverLocation, receiverLastBatchRecords) ++ + receivedRecordStats ++ Seq(receiverLastError) } Some(listingTable(headerRow, dataRows)) } else { diff --git a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala index ff3619a59042d..303d149d285e1 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/NetworkReceiverSuite.scala @@ -94,9 +94,13 @@ class NetworkReceiverSuite extends FunSuite with Timeouts { // Verify restarting actually stops and starts the receiver receiver.restart("restarting", null, 100) - assert(receiver.isStopped) - assert(receiver.onStopCalled) + eventually(timeout(50 millis), interval(10 millis)) { + // receiver will be stopped async + assert(receiver.isStopped) + assert(receiver.onStopCalled) + } eventually(timeout(1000 millis), interval(100 millis)) { + // receiver will be started async assert(receiver.onStartCalled) assert(executor.isReceiverStarted) assert(receiver.isStarted) diff --git a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala index 458dd3a2b13d8..ef0efa552ceaf 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/StreamingListenerSuite.scala @@ -66,7 +66,7 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { test("receiver info reporting") { val ssc = new StreamingContext("local[2]", "test", Milliseconds(1000)) - val inputStream = ssc.networkStream(new StreamingListenerSuiteReceiver) + val inputStream = ssc.receiverStream(new StreamingListenerSuiteReceiver) inputStream.foreachRDD(_.count) val collector = new ReceiverInfoCollector @@ -75,8 +75,8 @@ class StreamingListenerSuite extends TestSuiteBase with ShouldMatchers { ssc.start() try { eventually(timeout(1000 millis), interval(20 millis)) { - collector.startedReceiverInfo should have size 1 - collector.startedReceiverInfo(0).streamId should equal (0) + collector.startedReceiverStreamIds.size should be >= 1 + collector.startedReceiverStreamIds(0) should equal (0) collector.stoppedReceiverStreamIds should have size 1 collector.stoppedReceiverStreamIds(0) should equal (0) collector.receiverErrors should have size 1 @@ -108,20 +108,21 @@ class BatchInfoCollector extends StreamingListener { /** Listener that collects information on processed batches */ class ReceiverInfoCollector extends StreamingListener { - val startedReceiverInfo = new ArrayBuffer[ReceiverInfo] + val startedReceiverStreamIds = new ArrayBuffer[Int] val stoppedReceiverStreamIds = new ArrayBuffer[Int]() val receiverErrors = new ArrayBuffer[(Int, String, String)]() override def onReceiverStarted(receiverStarted: StreamingListenerReceiverStarted) { - startedReceiverInfo += receiverStarted.receiverInfo + startedReceiverStreamIds += receiverStarted.receiverInfo.streamId } override def onReceiverStopped(receiverStopped: StreamingListenerReceiverStopped) { - stoppedReceiverStreamIds += receiverStopped.streamId + stoppedReceiverStreamIds += receiverStopped.receiverInfo.streamId } override def onReceiverError(receiverError: StreamingListenerReceiverError) { - receiverErrors += ((receiverError.streamId, receiverError.message, receiverError.error)) + receiverErrors += ((receiverError.receiverInfo.streamId, + receiverError.receiverInfo.lastErrorMessage, receiverError.receiverInfo.lastError)) } } From 86ff8b10270bbe2579cdb1dc2297a9f4e145973e Mon Sep 17 00:00:00 2001 From: Michael Armbrust Date: Thu, 24 Apr 2014 21:42:33 -0700 Subject: [PATCH 378/397] Generalize pattern for planning hash joins. This will be helpful for [SPARK-1495](https://issues.apache.org/jira/browse/SPARK-1495) and other cases where we want to have custom hash join implementations but don't want to repeat the logic for finding the join keys. Author: Michael Armbrust Closes #418 from marmbrus/hashFilter and squashes the following commits: d5cc79b [Michael Armbrust] Address @rxin 's comments. 366b6d9 [Michael Armbrust] style fixes 14560eb [Michael Armbrust] Generalize pattern for planning hash joins. f4809c1 [Michael Armbrust] Move common functions to PredicateHelper. --- .../sql/catalyst/expressions/predicates.scala | 29 ++++++++--- .../sql/catalyst/planning/patterns.scala | 52 +++++++++++++++++++ .../spark/sql/execution/SparkStrategies.scala | 49 +++-------------- 3 files changed, 82 insertions(+), 48 deletions(-) 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 da5b2cf5b0362..82c7af684459f 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 @@ -17,10 +17,11 @@ package org.apache.spark.sql.catalyst.expressions -import org.apache.spark.sql.catalyst.trees -import org.apache.spark.sql.catalyst.errors.TreeNodeException import org.apache.spark.sql.catalyst.analysis.UnresolvedException -import org.apache.spark.sql.catalyst.types.{BooleanType, StringType, TimestampType} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.trees +import org.apache.spark.sql.catalyst.types.BooleanType + object InterpretedPredicate { def apply(expression: Expression): (Row => Boolean) = { @@ -37,10 +38,26 @@ trait Predicate extends Expression { } trait PredicateHelper { - def splitConjunctivePredicates(condition: Expression): Seq[Expression] = condition match { - case And(cond1, cond2) => splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) - case other => other :: Nil + protected def splitConjunctivePredicates(condition: Expression): Seq[Expression] = { + condition match { + case And(cond1, cond2) => + splitConjunctivePredicates(cond1) ++ splitConjunctivePredicates(cond2) + case other => other :: Nil + } } + + /** + * Returns true if `expr` can be evaluated using only the output of `plan`. This method + * can be used to determine when is is acceptable to move expression evaluation within a query + * plan. + * + * For example consider a join between two relations R(a, b) and S(c, d). + * + * `canEvaluate(Equals(a,b), R)` returns `true` where as `canEvaluate(Equals(a,c), R)` returns + * `false`. + */ + protected def canEvaluate(expr: Expression, plan: LogicalPlan): Boolean = + expr.references.subsetOf(plan.outputSet) } abstract class BinaryPredicate extends BinaryExpression with Predicate { diff --git a/sql/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 index 6dd816aa91dd1..0e3a8a6bd30a8 100644 --- a/sql/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 @@ -19,7 +19,10 @@ package org.apache.spark.sql.catalyst.planning import scala.annotation.tailrec +import org.apache.spark.sql.Logging + import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ /** @@ -101,6 +104,55 @@ object PhysicalOperation extends PredicateHelper { } } +/** + * A pattern that finds joins with equality conditions that can be evaluated using hashing + * techniques. For inner joins, any filters on top of the join operator are also matched. + */ +object HashFilteredJoin extends Logging with PredicateHelper { + /** (joinType, rightKeys, leftKeys, condition, leftChild, rightChild) */ + type ReturnType = + (JoinType, Seq[Expression], Seq[Expression], Option[Expression], LogicalPlan, LogicalPlan) + + def unapply(plan: LogicalPlan): Option[ReturnType] = plan match { + // All predicates can be evaluated for inner join (i.e., those that are in the ON + // clause and WHERE clause.) + case FilteredOperation(predicates, join @ Join(left, right, Inner, condition)) => + logger.debug(s"Considering hash inner join on: ${predicates ++ condition}") + splitPredicates(predicates ++ condition, join) + case join @ Join(left, right, joinType, condition) => + logger.debug(s"Considering hash join on: $condition") + splitPredicates(condition.toSeq, join) + case _ => None + } + + // Find equi-join predicates that can be evaluated before the join, and thus can be used + // as join keys. + def splitPredicates(allPredicates: Seq[Expression], join: Join): Option[ReturnType] = { + val Join(left, right, joinType, _) = join + val (joinPredicates, otherPredicates) = allPredicates.partition { + case Equals(l, r) if (canEvaluate(l, left) && canEvaluate(r, right)) || + (canEvaluate(l, right) && canEvaluate(r, left)) => true + 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) + } + + // Do not consider this strategy if there are no join keys. + if (joinKeys.nonEmpty) { + val leftKeys = joinKeys.map(_._1) + val rightKeys = joinKeys.map(_._2) + + Some((joinType, leftKeys, rightKeys, otherPredicates.reduceOption(And), left, right)) + } else { + logger.debug(s"Avoiding hash join with no join keys.") + None + } + } +} + /** * A pattern that collects all adjacent unions and returns their children as a Seq. */ 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 500fde19719af..f763106da4e0e 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 @@ -28,51 +28,16 @@ import org.apache.spark.sql.parquet._ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { self: SQLContext#SparkPlanner => - object HashJoin extends Strategy { + object HashJoin extends Strategy with PredicateHelper { 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 - // 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 _ => 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) - } - - // 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 joinOp = execution.HashJoin( - leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) - - // Make sure other conditions are met if present. - if (otherPredicates.nonEmpty) { - execution.Filter(combineConjunctivePredicates(otherPredicates), joinOp) :: Nil - } else { - joinOp :: Nil - } - } else { - logger.debug(s"Avoiding spark join with no join keys.") - Nil - } + // Find inner joins where at least some predicates can be evaluated by matching hash keys + // using the HashFilteredJoin pattern. + case HashFilteredJoin(Inner, leftKeys, rightKeys, condition, left, right) => + val hashJoin = + execution.HashJoin(leftKeys, rightKeys, BuildRight, planLater(left), planLater(right)) + condition.map(Filter(_, hashJoin)).getOrElse(hashJoin) :: Nil 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 } object PartialAggregation extends Strategy { From 06e82d94b6eba0b174b54c58280b3daca2f26dd8 Mon Sep 17 00:00:00 2001 From: Andrew Or Date: Thu, 24 Apr 2014 21:51:17 -0700 Subject: [PATCH 379/397] [Typo] In the maven docs: chd -> cdh Author: Andrew Or Closes #548 from andrewor14/doc-typo and squashes the following commits: 3eaf4c4 [Andrew Or] chd -> cdh --- docs/building-with-maven.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/building-with-maven.md b/docs/building-with-maven.md index 9cebaf12283fc..771d1933a13ea 100644 --- a/docs/building-with-maven.md +++ b/docs/building-with-maven.md @@ -45,7 +45,7 @@ For Apache Hadoop 2.x, 0.23.x, Cloudera CDH MRv2, and other Hadoop versions with $ mvn -Pyarn-alpha -Dhadoop.version=2.0.5-alpha -Dyarn.version=2.0.5-alpha -DskipTests clean package # Cloudera CDH 4.2.0 with MapReduce v2 - $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-chd4.2.0 -DskipTests clean package + $ mvn -Pyarn-alpha -Dhadoop.version=2.0.0-cdh4.2.0 -Dyarn.version=2.0.0-cdh4.2.0 -DskipTests clean package # Apache Hadoop 2.2.X ( e.g. 2.2.0 as below ) and newer $ mvn -Pyarn -Dhadoop.version=2.2.0 -Dyarn.version=2.2.0 -DskipTests clean package From 095b5182536a43e2ae738be93294ee5215d86581 Mon Sep 17 00:00:00 2001 From: Sandeep Date: Thu, 24 Apr 2014 21:51:52 -0700 Subject: [PATCH 380/397] Fix [SPARK-1078]: Remove the Unnecessary lift-json dependency Remove the Unnecessary lift-json dependency from pom.xml Author: Sandeep Closes #536 from techaddict/FIX-SPARK-1078 and squashes the following commits: bd0fd1d [Sandeep] Fix [SPARK-1078]: Replace lift-json with json4s-jackson. Remove the Unnecessary lift-json dependency from pom.xml --- pom.xml | 18 ++---------------- 1 file changed, 2 insertions(+), 16 deletions(-) diff --git a/pom.xml b/pom.xml index 4ff18afa227be..e292775c6267e 100644 --- a/pom.xml +++ b/pom.xml @@ -380,20 +380,6 @@ derby 10.4.2.0 - - net.liftweb - lift-json_${scala.binary.version} - 2.5.1 - - - - org.scala-lang - scalap - - - com.codahale.metrics metrics-core @@ -851,7 +837,7 @@ - + hadoop-provided @@ -896,6 +882,6 @@ - + From e03bc379ee03fde0ee4fa578d3c39aae35c63f01 Mon Sep 17 00:00:00 2001 From: Holden Karau Date: Thu, 24 Apr 2014 23:07:54 -0700 Subject: [PATCH 381/397] SPARK-1242 Add aggregate to python rdd Author: Holden Karau Closes #139 from holdenk/add_aggregate_to_python_api and squashes the following commits: 0f39ae3 [Holden Karau] Merge in master 4879c75 [Holden Karau] CR feedback, fix issue with empty RDDs in aggregate 70b4724 [Holden Karau] Style fixes from code review 96b047b [Holden Karau] Add aggregate to python rdd --- python/pyspark/rdd.py | 31 +++++++++++++++++++++++++++++-- 1 file changed, 29 insertions(+), 2 deletions(-) diff --git a/python/pyspark/rdd.py b/python/pyspark/rdd.py index d73ab7006e9c7..a59778c72130e 100644 --- a/python/pyspark/rdd.py +++ b/python/pyspark/rdd.py @@ -599,7 +599,7 @@ def _collect_iterator_through_file(self, iterator): def reduce(self, f): """ Reduces the elements of this RDD using the specified commutative and - associative binary operator. + associative binary operator. Currently reduces partitions locally. >>> from operator import add >>> sc.parallelize([1, 2, 3, 4, 5]).reduce(add) @@ -641,7 +641,34 @@ def func(iterator): vals = self.mapPartitions(func).collect() return reduce(op, vals, zeroValue) - # TODO: aggregate + def aggregate(self, zeroValue, seqOp, combOp): + """ + Aggregate the elements of each partition, and then the results for all + the partitions, using a given combine functions and a neutral "zero + value." + + The functions C{op(t1, t2)} is allowed to modify C{t1} and return it + as its result value to avoid object allocation; however, it should not + modify C{t2}. + + The first function (seqOp) can return a different result type, U, than + the type of this RDD. Thus, we need one operation for merging a T into an U + and one operation for merging two U + + >>> seqOp = (lambda x, y: (x[0] + y, x[1] + 1)) + >>> combOp = (lambda x, y: (x[0] + y[0], x[1] + y[1])) + >>> sc.parallelize([1, 2, 3, 4]).aggregate((0, 0), seqOp, combOp) + (10, 4) + >>> sc.parallelize([]).aggregate((0, 0), seqOp, combOp) + (0, 0) + """ + def func(iterator): + acc = zeroValue + for obj in iterator: + acc = seqOp(acc, obj) + yield acc + + return self.mapPartitions(func).fold(zeroValue, combOp) def max(self): From 45ad7f0ca7be4a89c066b246b0fee0c0c2e7c759 Mon Sep 17 00:00:00 2001 From: Aaron Davidson Date: Thu, 24 Apr 2014 23:22:03 -0700 Subject: [PATCH 382/397] Call correct stop(). Oopsie in #504. Author: Aaron Davidson Closes #527 from aarondav/stop and squashes the following commits: 8d1446a [Aaron Davidson] Call correct stop(). --- .../main/scala/org/apache/spark/storage/DiskBlockManager.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala index 054f66a8b7260..cf6ef0029a861 100644 --- a/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala +++ b/core/src/main/scala/org/apache/spark/storage/DiskBlockManager.scala @@ -150,11 +150,12 @@ private[spark] class DiskBlockManager(shuffleManager: ShuffleBlockManager, rootD Runtime.getRuntime.addShutdownHook(new Thread("delete Spark local dirs") { override def run() { logDebug("Shutdown hook called") - stop() + DiskBlockManager.this.stop() } }) } + /** Cleanup local dirs and stop shuffle sender. */ private[spark] def stop() { localDirs.foreach { localDir => if (localDir.isDirectory() && localDir.exists()) { From 6e101f1183f92769779bc8ac14813c063bf1ff3f Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Thu, 24 Apr 2014 23:34:00 -0700 Subject: [PATCH 383/397] SPARK-1607. Replace octal literals, removed in Scala 2.11, with hex literals Octal literals like "0700" are deprecated in Scala 2.10, generating a warning. They have been removed entirely in 2.11. See https://issues.scala-lang.org/browse/SI-7618 This change simply replaces two uses of octals with hex literals, which seemed the next-best representation since they express a bit mask (file permission in particular) Author: Sean Owen Closes #529 from srowen/SPARK-1607 and squashes the following commits: 1ee0e67 [Sean Owen] Use Integer.parseInt(...,8) for octal literal instead of hex equivalent 0102f3d [Sean Owen] Replace octal literals, removed in Scala 2.11, with hex literals --- .../scala/org/apache/spark/deploy/yarn/ClientBase.scala | 6 ++++-- 1 file changed, 4 insertions(+), 2 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index b403292d9cb2e..eb95d784318ea 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -59,9 +59,11 @@ trait ClientBase extends Logging { private val distCacheMgr = new ClientDistributedCacheManager() // Staging directory is private! -> rwx-------- - val STAGING_DIR_PERMISSION: FsPermission = FsPermission.createImmutable(0700: Short) + val STAGING_DIR_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("700", 8): Short) // App files are world-wide readable and owner writable -> rw-r--r-- - val APP_FILE_PERMISSION: FsPermission = FsPermission.createImmutable(0644: Short) + val APP_FILE_PERMISSION: FsPermission = + FsPermission.createImmutable(Integer.parseInt("644", 8): Short) // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { From dc3b640a0ab3501b678b591be3e99fbcf3badbec Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Thu, 24 Apr 2014 23:59:16 -0700 Subject: [PATCH 384/397] SPARK-1619 Launch spark-shell with spark-submit This simplifies the shell a bunch and passes all arguments through to spark-submit. There is a tiny incompatibility from 0.9.1 which is that you can't put `-c` _or_ `--cores`, only `--cores`. However, spark-submit will give a good error message in this case, I don't think many people used this, and it's a trivial change for users. Author: Patrick Wendell Closes #542 from pwendell/spark-shell and squashes the following commits: 9eb3e6f [Patrick Wendell] Updating Spark docs b552459 [Patrick Wendell] Andrew's feedback 97720fa [Patrick Wendell] Review feedback aa2900b [Patrick Wendell] SPARK-1619 Launch spark-shell with spark-submit --- .gitignore | 1 + bin/spark-shell | 177 +----------------- bin/spark-submit | 10 +- .../org/apache/spark/deploy/SparkSubmit.scala | 10 +- .../spark/deploy/SparkSubmitArguments.scala | 2 +- docs/scala-programming-guide.md | 9 +- docs/spark-debugger.md | 2 +- docs/spark-standalone.md | 4 +- docs/streaming-programming-guide.md | 6 +- make-distribution.sh | 2 +- .../org/apache/spark/repl/SparkILoop.scala | 5 +- 11 files changed, 39 insertions(+), 189 deletions(-) diff --git a/.gitignore b/.gitignore index cd9f90d55932c..857e9feb953bd 100644 --- a/.gitignore +++ b/.gitignore @@ -18,6 +18,7 @@ conf/java-opts conf/spark-env.sh conf/streaming-env.sh conf/log4j.properties +conf/spark-defaults.conf docs/_site docs/api target/ diff --git a/bin/spark-shell b/bin/spark-shell index ea12d256b23a1..f1f3c18877ed4 100755 --- a/bin/spark-shell +++ b/bin/spark-shell @@ -19,9 +19,8 @@ # # Shell script for starting the Spark Shell REPL -# Note that it will set MASTER to spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT} -# if those two env vars are set in spark-env.sh but MASTER is not. +args="$@" cygwin=false case "`uname`" in CYGWIN*) cygwin=true;; @@ -30,133 +29,16 @@ esac # Enter posix mode for bash set -o posix +if [[ "$@" == *--help* ]]; then + echo "Usage: ./bin/spark-shell [options]" + ./bin/spark-submit --help 2>&1 | grep -v Usage 1>&2 + exit 0 +fi + ## Global script variables FWDIR="$(cd `dirname $0`/..; pwd)" -SPARK_REPL_OPTS="${SPARK_REPL_OPTS:-""}" -DEFAULT_MASTER="local[*]" -MASTER=${MASTER:-""} - -info_log=0 - -#CLI Color Templates -txtund=$(tput sgr 0 1) # Underline -txtbld=$(tput bold) # Bold -bldred=${txtbld}$(tput setaf 1) # red -bldyel=${txtbld}$(tput setaf 3) # yellow -bldblu=${txtbld}$(tput setaf 4) # blue -bldwht=${txtbld}$(tput setaf 7) # white -txtrst=$(tput sgr0) # Reset -info=${bldwht}*${txtrst} # Feedback -pass=${bldblu}*${txtrst} -warn=${bldred}*${txtrst} -ques=${bldblu}?${txtrst} - -# Helper function to describe the script usage -function usage() { - cat << EOF -${txtbld}Usage${txtrst}: spark-shell [OPTIONS] - -${txtbld}OPTIONS${txtrst}: - -h --help : Print this help information. - -c --cores : The maximum number of cores to be used by the Spark Shell. - -em --executor-memory : The memory used by each executor of the Spark Shell, the number - is followed by m for megabytes or g for gigabytes, e.g. "1g". - -dm --driver-memory : The memory used by the Spark Shell, the number is followed - by m for megabytes or g for gigabytes, e.g. "1g". - -m --master : A full string that describes the Spark Master, defaults to "local[*]" - e.g. "spark://localhost:7077". - --log-conf : Enables logging of the supplied SparkConf as INFO at start of the - Spark Context. - -e.g. - spark-shell -m spark://localhost:7077 -c 4 -dm 512m -em 2g - -EOF -} - -function out_error(){ - echo -e "${txtund}${bldred}ERROR${txtrst}: $1" - usage - exit 1 -} - -function log_info(){ - [ $info_log -eq 1 ] && echo -e "${bldyel}INFO${txtrst}: $1" -} - -function log_warn(){ - echo -e "${txtund}${bldyel}WARN${txtrst}: $1" -} - -# PATTERNS used to validate more than one optional arg. -ARG_FLAG_PATTERN="^-" -MEM_PATTERN="^[0-9]+[m|g|M|G]$" -NUM_PATTERN="^[0-9]+$" -PORT_PATTERN="^[0-9]+$" - -# Setters for optional args. -function set_cores(){ - CORE_PATTERN="^[0-9]+$" - if [[ "$1" =~ $CORE_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.cores.max=$1" - else - out_error "wrong format for $2" - fi -} - -function set_em(){ - if [[ $1 =~ $MEM_PATTERN ]]; then - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.executor.memory=$1" - else - out_error "wrong format for $2" - fi -} - -function set_dm(){ - if [[ $1 =~ $MEM_PATTERN ]]; then - export SPARK_DRIVER_MEMORY=$1 - else - out_error "wrong format for $2" - fi -} - -function set_spark_log_conf(){ - SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Dspark.logConf=$1" -} - -function set_spark_master(){ - if ! [[ "$1" =~ $ARG_FLAG_PATTERN ]]; then - export MASTER="$1" - else - out_error "wrong format for $2" - fi -} - -function resolve_spark_master(){ - # Set MASTER from spark-env if possible - DEFAULT_SPARK_MASTER_PORT=7077 - if [ -z "$MASTER" ]; then - . $FWDIR/bin/load-spark-env.sh - if [ -n "$SPARK_MASTER_IP" ]; then - SPARK_MASTER_PORT="${SPARK_MASTER_PORT:-"$DEFAULT_SPARK_MASTER_PORT"}" - export MASTER="spark://${SPARK_MASTER_IP}:${SPARK_MASTER_PORT}" - fi - fi - - if [ -z "$MASTER" ]; then - export MASTER="$DEFAULT_MASTER" - fi - -} - function main(){ - log_info "Base Directory set to $FWDIR" - - resolve_spark_master - log_info "Spark Master is $MASTER" - - log_info "Spark REPL options $SPARK_REPL_OPTS" if $cygwin; then # Workaround for issue involving JLine and Cygwin # (see http://sourceforge.net/p/jline/bugs/40/). @@ -165,55 +47,14 @@ function main(){ # (see https://github.com/sbt/sbt/issues/562). stty -icanon min 1 -echo > /dev/null 2>&1 export SPARK_REPL_OPTS="$SPARK_REPL_OPTS -Djline.terminal=unix" - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-submit spark-internal "$args" --class org.apache.spark.repl.Main stty icanon echo > /dev/null 2>&1 else export SPARK_REPL_OPTS - $FWDIR/bin/spark-class org.apache.spark.repl.Main "$@" + $FWDIR/bin/spark-submit spark-internal "$args" --class org.apache.spark.repl.Main fi } -for option in "$@" -do - case $option in - -h | --help ) - usage - exit 1 - ;; - -c | --cores) - shift - _1=$1 - shift - set_cores $_1 "-c/--cores" - ;; - -em | --executor-memory) - shift - _1=$1 - shift - set_em $_1 "-em/--executor-memory" - ;; - -dm | --driver-memory) - shift - _1=$1 - shift - set_dm $_1 "-dm/--driver-memory" - ;; - -m | --master) - shift - _1=$1 - shift - set_spark_master $_1 "-m/--master" - ;; - --log-conf) - shift - set_spark_log_conf "true" - info_log=1 - ;; - ?) - ;; - esac -done - # Copy restore-TTY-on-exit functions from Scala script so spark-shell exits properly even in # binary distribution of Spark where Scala is not installed exit_status=127 diff --git a/bin/spark-submit b/bin/spark-submit index 498d0b27bacdf..b2a1dca721dff 100755 --- a/bin/spark-submit +++ b/bin/spark-submit @@ -21,15 +21,15 @@ export SPARK_HOME="$(cd `dirname $0`/..; pwd)" ORIG_ARGS=$@ while (($#)); do - if [ $1 = "--deploy-mode" ]; then + if [ "$1" = "--deploy-mode" ]; then DEPLOY_MODE=$2 - elif [ $1 = "--driver-memory" ]; then + elif [ "$1" = "--driver-memory" ]; then DRIVER_MEMORY=$2 - elif [ $1 = "--driver-library-path" ]; then + elif [ "$1" = "--driver-library-path" ]; then export _SPARK_LIBRARY_PATH=$2 - elif [ $1 = "--driver-class-path" ]; then + elif [ "$1" = "--driver-class-path" ]; then export SPARK_CLASSPATH="$SPARK_CLASSPATH:$2" - elif [ $1 = "--driver-java-options" ]; then + elif [ "$1" = "--driver-java-options" ]; then export SPARK_JAVA_OPTS="$SPARK_JAVA_OPTS $2" fi shift diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index a2efddbfe3f69..58aa6d951a204 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -38,6 +38,12 @@ object SparkSubmit { private var clusterManager: Int = LOCAL + /** + * A special jar name that indicates the class being run is inside of Spark itself, + * and therefore no user jar is needed. + */ + private val RESERVED_JAR_NAME = "spark-internal" + def main(args: Array[String]) { val appArgs = new SparkSubmitArguments(args) if (appArgs.verbose) { @@ -113,7 +119,9 @@ object SparkSubmit { if (!deployOnCluster) { childMainClass = appArgs.mainClass - childClasspath += appArgs.primaryResource + if (appArgs.primaryResource != RESERVED_JAR_NAME) { + childClasspath += appArgs.primaryResource + } } else if (clusterManager == YARN) { childMainClass = "org.apache.spark.deploy.yarn.Client" childArgs += ("--jar", appArgs.primaryResource) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index c3e8c6b8c65a9..c545b093ac82e 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -107,7 +107,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { deployMode = Option(deployMode).getOrElse(System.getenv("DEPLOY_MODE")) // Global defaults. These should be keep to minimum to avoid confusing behavior. - master = Option(master).getOrElse("local") + master = Option(master).getOrElse("local[*]") } /** Ensure that required fields exists. Call this only once all defaults are loaded. */ diff --git a/docs/scala-programming-guide.md b/docs/scala-programming-guide.md index a3171709ff05b..b8d89cf00ffbd 100644 --- a/docs/scala-programming-guide.md +++ b/docs/scala-programming-guide.md @@ -60,17 +60,18 @@ which avoids hard-coding the master name in your application. In the Spark shell, a special interpreter-aware SparkContext is already created for you, in the variable called `sc`. Making your own SparkContext will not work. You can set which master the -context connects to using the `MASTER` environment variable, and you can add JARs to the classpath -with the `ADD_JARS` variable. For example, to run `bin/spark-shell` on exactly four cores, use +context connects to using the `--master` argument, and you can add JARs to the classpath +by passing a comma separated list to the `--jars` argument. For example, to run +`bin/spark-shell` on exactly four cores, use {% highlight bash %} -$ MASTER=local[4] ./bin/spark-shell +$ ./bin/spark-shell --master local[4] {% endhighlight %} Or, to also add `code.jar` to its classpath, use: {% highlight bash %} -$ MASTER=local[4] ADD_JARS=code.jar ./bin/spark-shell +$ ./bin/spark-shell --master local[4] --jars code.jar {% endhighlight %} ### Master URLs diff --git a/docs/spark-debugger.md b/docs/spark-debugger.md index 891c2bfa8943d..35d06c51aaf0e 100644 --- a/docs/spark-debugger.md +++ b/docs/spark-debugger.md @@ -39,7 +39,7 @@ where `path/to/event-log` is where you want the event log to go relative to `$SP ### Loading the event log into the debugger -1. Run a Spark shell with `MASTER=host ./bin/spark-shell`. +1. Run a Spark shell with `./bin/spark-shell --master hist`. 2. Use `EventLogReader` to load the event log as follows: {% highlight scala %} spark> val r = new spark.EventLogReader(sc, Some("path/to/event-log")) diff --git a/docs/spark-standalone.md b/docs/spark-standalone.md index 7e4eea323aa63..dc7f206e03996 100644 --- a/docs/spark-standalone.md +++ b/docs/spark-standalone.md @@ -139,12 +139,12 @@ constructor](scala-programming-guide.html#initializing-spark). To run an interactive Spark shell against the cluster, run the following command: - MASTER=spark://IP:PORT ./bin/spark-shell + ./bin/spark-shell --master spark://IP:PORT Note that if you are running spark-shell from one of the spark cluster machines, the `bin/spark-shell` script will automatically set MASTER from the `SPARK_MASTER_IP` and `SPARK_MASTER_PORT` variables in `conf/spark-env.sh`. -You can also pass an option `-c ` to control the number of cores that spark-shell uses on the cluster. +You can also pass an option `--cores ` to control the number of cores that spark-shell uses on the cluster. # Launching Compiled Spark Applications diff --git a/docs/streaming-programming-guide.md b/docs/streaming-programming-guide.md index 946d6c4879cf3..7ad06427cac5c 100644 --- a/docs/streaming-programming-guide.md +++ b/docs/streaming-programming-guide.md @@ -272,12 +272,10 @@ Time: 1357008430000 ms -If you plan to run the Scala code for Spark Streaming-based use cases in the Spark -shell, you should start the shell with the SparkConfiguration pre-configured to -discard old batches periodically: +You can also use Spark Streaming directly from the Spark shell: {% highlight bash %} -$ SPARK_JAVA_OPTS=-Dspark.cleaner.ttl=10000 bin/spark-shell +$ bin/spark-shell {% endhighlight %} ... and create your StreamingContext by wrapping the existing interactive shell diff --git a/make-distribution.sh b/make-distribution.sh index 62a28217740cd..55fe6c09d0d66 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -36,7 +36,7 @@ # 2) cd to deploy dir; ./sbin/start-master.sh # 3) Verify master is up by visiting web page, ie http://master-ip:8080. Note the spark:// URL. # 4) ./sbin/start-slave.sh 1 <> -# 5) MASTER="spark://my-master-ip:7077" ./bin/spark-shell +# 5) ./bin/spark-shell --master spark://my-master-ip:7077 # # Figure out where the Spark framework is installed diff --git a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala index beb40e87024bd..296da740687ec 100644 --- a/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -963,8 +963,9 @@ class SparkILoop(in0: Option[BufferedReader], protected val out: JPrintWriter, val master = this.master match { case Some(m) => m case None => { - val prop = System.getenv("MASTER") - if (prop != null) prop else "local[*]" + val envMaster = sys.env.get("MASTER") + val propMaster = sys.props.get("spark.master") + envMaster.orElse(propMaster).getOrElse("local[*]") } } master From a24d918c71f6ac4adbe3ae363ef69f4658118938 Mon Sep 17 00:00:00 2001 From: Matei Zaharia Date: Fri, 25 Apr 2014 11:12:41 -0700 Subject: [PATCH 385/397] SPARK-1621 Upgrade Chill to 0.3.6 It registers more Scala classes, including things like Ranges that we had to register manually before. See https://github.com/twitter/chill/releases for Chill's change log. Author: Matei Zaharia Closes #543 from mateiz/chill-0.3.6 and squashes the following commits: a1dc5e0 [Matei Zaharia] Upgrade Chill to 0.3.6 and remove our special registration of Ranges --- .../org/apache/spark/serializer/KryoSerializer.scala | 12 +++++------- pom.xml | 2 +- project/SparkBuild.scala | 6 +++--- 3 files changed, 9 insertions(+), 11 deletions(-) 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 d1e8c3ef63622..c4daec7875d26 100644 --- a/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala +++ b/core/src/main/scala/org/apache/spark/serializer/KryoSerializer.scala @@ -58,7 +58,9 @@ class KryoSerializer(conf: SparkConf) // Do this before we invoke the user registrator so the user registrator can override this. kryo.setReferences(referenceTracking) - for (cls <- KryoSerializer.toRegister) kryo.register(cls) + for (cls <- KryoSerializer.toRegister) { + kryo.register(cls) + } // Allow sending SerializableWritable kryo.register(classOf[SerializableWritable[_]], new KryoJavaSerializer()) @@ -77,7 +79,7 @@ class KryoSerializer(conf: SparkConf) } // Register Chill's classes; we do this after our ranges and the user's own classes to let - // our code override the generic serialziers in Chill for things like Seq + // our code override the generic serializers in Chill for things like Seq new AllScalaRegistrar().apply(kryo) kryo.setClassLoader(classLoader) @@ -176,10 +178,6 @@ private[serializer] object KryoSerializer { classOf[GetBlock], classOf[MapStatus], classOf[BlockManagerId], - classOf[Array[Byte]], - (1 to 10).getClass, - (1 until 10).getClass, - (1L to 10L).getClass, - (1L until 10L).getClass + classOf[Array[Byte]] ) } diff --git a/pom.xml b/pom.xml index e292775c6267e..e9117998252fc 100644 --- a/pom.xml +++ b/pom.xml @@ -125,7 +125,7 @@ 1.3.2 1.2.3 8.1.14.v20131031 - 0.3.1 + 0.3.6 3.0.0 64m diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 9cbc188c4d485..48f234ce87a44 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -294,7 +294,7 @@ object SparkBuild extends Build { ) ++ net.virtualvoid.sbt.graph.Plugin.graphSettings ++ ScalaStyleSettings ++ genjavadocSettings val akkaVersion = "2.2.3-shaded-protobuf" - val chillVersion = "0.3.1" + val chillVersion = "0.3.6" val codahaleMetricsVersion = "3.0.0" val jblasVersion = "1.2.3" val jettyVersion = "8.1.14.v20131031" @@ -412,8 +412,8 @@ object SparkBuild extends Build { ) def examplesSettings = sharedSettings ++ Seq( - name := "spark-examples", - jarName in assembly <<= version map { + name := "spark-examples", + jarName in assembly <<= version map { v => "spark-examples-" + v + "-hadoop" + hadoopVersion + ".jar" }, libraryDependencies ++= Seq( "com.twitter" %% "algebird-core" % "0.1.11", From 25a276dd21457fa0db76b6be93765682a50b784a Mon Sep 17 00:00:00 2001 From: WangTao Date: Fri, 25 Apr 2014 11:47:01 -0700 Subject: [PATCH 386/397] Delete the val that never used It seems that the val "startTime" and "endTime" is never used, so delete them. Author: WangTao Closes #553 from WangTaoTheTonic/master and squashes the following commits: 4fcb639 [WangTao] Delete the val that never used --- .../main/scala/org/apache/spark/storage/BlockMessage.scala | 4 ---- 1 file changed, 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala index 337b45b727dec..a2bfce7b4a0fa 100644 --- a/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala +++ b/core/src/main/scala/org/apache/spark/storage/BlockMessage.scala @@ -57,7 +57,6 @@ private[spark] class BlockMessage() { } def set(buffer: ByteBuffer) { - val startTime = System.currentTimeMillis /* println() println("BlockMessage: ") @@ -100,7 +99,6 @@ private[spark] class BlockMessage() { data.flip() } - val finishTime = System.currentTimeMillis } def set(bufferMsg: BufferMessage) { @@ -115,7 +113,6 @@ private[spark] class BlockMessage() { def getLevel: StorageLevel = level def toBufferMessage: BufferMessage = { - val startTime = System.currentTimeMillis val buffers = new ArrayBuffer[ByteBuffer]() var buffer = ByteBuffer.allocate(4 + 4 + id.name.length * 2) buffer.putInt(typ).putInt(id.name.length) @@ -153,7 +150,6 @@ private[spark] class BlockMessage() { println() println() */ - val finishTime = System.currentTimeMillis Message.createBufferMessage(buffers) } From 8aaef5c75624b48806a05943454f7b72f3ca1bdc Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?baishuo=28=E7=99=BD=E7=A1=95=29?= Date: Fri, 25 Apr 2014 13:18:49 -0700 Subject: [PATCH 387/397] Update KafkaWordCount.scala MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit modify the required args number Author: baishuo(白硕) Closes #523 from baishuo/master and squashes the following commits: 0368ba9 [baishuo(白硕)] Update KafkaWordCount.scala --- .../org/apache/spark/streaming/examples/KafkaWordCount.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala index 3a18bd02b89ac..2aa4f1474a59e 100644 --- a/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala +++ b/examples/src/main/scala/org/apache/spark/streaming/examples/KafkaWordCount.scala @@ -71,7 +71,7 @@ object KafkaWordCount { object KafkaWordCountProducer { def main(args: Array[String]) { - if (args.length < 2) { + if (args.length < 4) { System.err.println("Usage: KafkaWordCountProducer " + " ") System.exit(1) From df6d81425bf3b8830988288069f6863de873aee2 Mon Sep 17 00:00:00 2001 From: Sean Owen Date: Fri, 25 Apr 2014 14:17:38 -0700 Subject: [PATCH 388/397] SPARK-1607. HOTFIX: Fix syntax adapting Int result to Short Sorry folks. This should make the change for SPARK-1607 compile again. Verified this time with the yarn build enabled. Author: Sean Owen Closes #556 from srowen/SPARK-1607.2 and squashes the following commits: e3fe7a3 [Sean Owen] Fix syntax adapting Int result to Short --- .../main/scala/org/apache/spark/deploy/yarn/ClientBase.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala index eb95d784318ea..f2be8217a2f8a 100644 --- a/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala +++ b/yarn/common/src/main/scala/org/apache/spark/deploy/yarn/ClientBase.scala @@ -60,10 +60,10 @@ trait ClientBase extends Logging { // Staging directory is private! -> rwx-------- val STAGING_DIR_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("700", 8): Short) + FsPermission.createImmutable(Integer.parseInt("700", 8).toShort) // App files are world-wide readable and owner writable -> rw-r--r-- val APP_FILE_PERMISSION: FsPermission = - FsPermission.createImmutable(Integer.parseInt("644", 8): Short) + FsPermission.createImmutable(Integer.parseInt("644", 8).toShort) // TODO(harvey): This could just go in ClientArguments. def validateArgs() = { From 027f1b85f961ce16ee069afe3d90a36dce009994 Mon Sep 17 00:00:00 2001 From: CodingCat Date: Fri, 25 Apr 2014 16:04:48 -0700 Subject: [PATCH 389/397] SPARK-1235: manage the DAGScheduler EventProcessActor with supervisor and refactor the DAGScheduler with Akka https://spark-project.atlassian.net/browse/SPARK-1235 In the current implementation, the running job will hang if the DAGScheduler crashes for some reason (eventProcessActor throws exception in receive() ) The reason is that the actor will automatically restart when the exception is thrown during the running but is not captured properly (Akka behaviour), and the JobWaiters are still waiting there for the completion of the tasks In this patch, I refactored the DAGScheduler with Akka and manage the eventProcessActor with supervisor, so that upon the failure of a eventProcessActor, the supervisor will terminate the EventProcessActor and close the SparkContext thanks for @kayousterhout and @markhamstra to give the hints in JIRA Author: CodingCat Author: Xiangrui Meng Author: Nan Zhu Closes #186 from CodingCat/SPARK-1235 and squashes the following commits: a7fb0ee [CodingCat] throw Exception on failure of creating DAG 124d82d [CodingCat] blocking the constructor until event actor is ready baf2d38 [CodingCat] fix the issue brought by non-blocking actorOf 35c886a [CodingCat] fix bug 82d08b3 [CodingCat] calling actorOf on system to ensure it is blocking 310a579 [CodingCat] style fix cd02d9a [Nan Zhu] small fix 561cfbc [CodingCat] recover doCheckpoint c048d0e [CodingCat] call submitWaitingStages for every event a9eea039 [CodingCat] address Matei's comments ac878ab [CodingCat] typo fix 5d1636a [CodingCat] re-trigger the test..... 9dfb033 [CodingCat] remove unnecessary changes a7a2a97 [CodingCat] add StageCancelled message fdf3b17 [CodingCat] just to retrigger the test...... 089bc2f [CodingCat] address andrew's comments 228f4b0 [CodingCat] address comments from Mark b68c1c7 [CodingCat] refactor DAGScheduler with Akka 810efd8 [Xiangrui Meng] akka solution --- .../scala/org/apache/spark/SparkContext.scala | 20 +- .../main/scala/org/apache/spark/rdd/RDD.scala | 6 +- .../apache/spark/scheduler/DAGScheduler.scala | 419 ++++++++++-------- .../spark/scheduler/DAGSchedulerEvent.scala | 4 +- .../spark/scheduler/TaskSetManager.scala | 2 +- .../spark/scheduler/DAGSchedulerSuite.scala | 58 ++- .../scheduler/TaskSchedulerImplSuite.scala | 2 +- 7 files changed, 290 insertions(+), 221 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/SparkContext.scala b/core/src/main/scala/org/apache/spark/SparkContext.scala index e9d2f5757963d..eb14d87467af7 100644 --- a/core/src/main/scala/org/apache/spark/SparkContext.scala +++ b/core/src/main/scala/org/apache/spark/SparkContext.scala @@ -300,10 +300,17 @@ class SparkContext(config: SparkConf) extends Logging { // Create and start the scheduler private[spark] var taskScheduler = SparkContext.createTaskScheduler(this, master) - taskScheduler.start() + @volatile private[spark] var dagScheduler: DAGScheduler = _ + try { + dagScheduler = new DAGScheduler(this) + } catch { + case e: Exception => throw + new SparkException("DAGScheduler cannot be initialized due to %s".format(e.getMessage)) + } - @volatile private[spark] var dagScheduler = new DAGScheduler(this) - dagScheduler.start() + // start TaskScheduler after taskScheduler sets DAGScheduler reference in DAGScheduler's + // constructor + taskScheduler.start() private[spark] val cleaner: Option[ContextCleaner] = { if (conf.getBoolean("spark.cleaner.referenceTracking", true)) { @@ -1022,8 +1029,8 @@ class SparkContext(config: SparkConf) extends Logging { partitions: Seq[Int], allowLocal: Boolean, resultHandler: (Int, U) => Unit) { - partitions.foreach{ p => - require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") + if (dagScheduler == null) { + throw new SparkException("SparkContext has been shutdown") } val callSite = getCallSite val cleanedFunc = clean(func) @@ -1132,9 +1139,6 @@ class SparkContext(config: SparkConf) extends Logging { resultHandler: (Int, U) => Unit, resultFunc: => R): SimpleFutureAction[R] = { - partitions.foreach{ p => - require(p >= 0 && p < rdd.partitions.size, s"Invalid partition requested: $p") - } val cleanF = clean(processPartition) val callSite = getCallSite val waiter = dagScheduler.submitJob( diff --git a/core/src/main/scala/org/apache/spark/rdd/RDD.scala b/core/src/main/scala/org/apache/spark/rdd/RDD.scala index e8bbfbf01679c..3b3524f33e811 100644 --- a/core/src/main/scala/org/apache/spark/rdd/RDD.scala +++ b/core/src/main/scala/org/apache/spark/rdd/RDD.scala @@ -1142,9 +1142,9 @@ abstract class RDD[T: ClassTag]( @transient private var doCheckpointCalled = false /** - * Performs the checkpointing of this RDD by saving this. It is called by the DAGScheduler - * after a job using this RDD has completed (therefore the RDD has been materialized and - * potentially stored in memory). doCheckpoint() is called recursively on the parent RDDs. + * Performs the checkpointing of this RDD by saving this. It is called after a job using this RDD + * has completed (therefore the RDD has been materialized and potentially stored in memory). + * doCheckpoint() is called recursively on the parent RDDs. */ private[spark] def doCheckpoint() { if (!doCheckpointCalled) { diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala index dbde9b591dccc..ff411e24a3d85 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGScheduler.scala @@ -22,10 +22,16 @@ import java.util.Properties import java.util.concurrent.atomic.AtomicInteger import scala.collection.mutable.{ArrayBuffer, HashMap, HashSet, Map} +import scala.concurrent.Await import scala.concurrent.duration._ +import scala.language.postfixOps import scala.reflect.ClassTag import akka.actor._ +import akka.actor.OneForOneStrategy +import akka.actor.SupervisorStrategy.Stop +import akka.pattern.ask +import akka.util.Timeout import org.apache.spark._ import org.apache.spark.executor.TaskMetrics @@ -47,14 +53,11 @@ import org.apache.spark.util.Utils * not caused by shuffle file loss are handled by the TaskScheduler, which will retry each task * a small number of times before cancelling the whole stage. * - * THREADING: This class runs all its logic in a single thread executing the run() method, to which - * events are submitted using a synchronized queue (eventQueue). The public API methods, such as - * runJob, taskEnded and executorLost, post events asynchronously to this queue. All other methods - * should be private. */ private[spark] class DAGScheduler( - taskScheduler: TaskScheduler, + private[scheduler] val sc: SparkContext, + private[scheduler] val taskScheduler: TaskScheduler, listenerBus: LiveListenerBus, mapOutputTracker: MapOutputTrackerMaster, blockManagerMaster: BlockManagerMaster, @@ -65,6 +68,7 @@ class DAGScheduler( def this(sc: SparkContext, taskScheduler: TaskScheduler) = { this( + sc, taskScheduler, sc.listenerBus, sc.env.mapOutputTracker.asInstanceOf[MapOutputTrackerMaster], @@ -74,8 +78,6 @@ class DAGScheduler( def this(sc: SparkContext) = this(sc, sc.taskScheduler) - private var eventProcessActor: ActorRef = _ - private[scheduler] val nextJobId = new AtomicInteger(0) private[scheduler] def numTotalJobs: Int = nextJobId.get() private val nextStageId = new AtomicInteger(0) @@ -113,50 +115,31 @@ class DAGScheduler( // stray messages to detect. private val failedEpoch = new HashMap[String, Long] - taskScheduler.setDAGScheduler(this) + private val dagSchedulerActorSupervisor = + env.actorSystem.actorOf(Props(new DAGSchedulerActorSupervisor(this))) - /** - * Starts the event processing actor. The actor has two responsibilities: - * - * 1. Waits for events like job submission, task finished, task failure etc., and calls - * [[org.apache.spark.scheduler.DAGScheduler.processEvent()]] to process them. - * 2. Schedules a periodical task to resubmit failed stages. - * - * NOTE: the actor cannot be started in the constructor, because the periodical task references - * some internal states of the enclosing [[org.apache.spark.scheduler.DAGScheduler]] object, thus - * cannot be scheduled until the [[org.apache.spark.scheduler.DAGScheduler]] is fully constructed. - */ - def start() { - eventProcessActor = env.actorSystem.actorOf(Props(new Actor { - /** - * The main event loop of the DAG scheduler. - */ - def receive = { - case event: DAGSchedulerEvent => - logTrace("Got event of type " + event.getClass.getName) - - /** - * All events are forwarded to `processEvent()`, so that the event processing logic can - * easily tested without starting a dedicated actor. Please refer to `DAGSchedulerSuite` - * for details. - */ - if (!processEvent(event)) { - submitWaitingStages() - } else { - context.stop(self) - } - } - })) + private[scheduler] var eventProcessActor: ActorRef = _ + + private def initializeEventProcessActor() { + // blocking the thread until supervisor is started, which ensures eventProcessActor is + // not null before any job is submitted + implicit val timeout = Timeout(30 seconds) + val initEventActorReply = + dagSchedulerActorSupervisor ? Props(new DAGSchedulerEventProcessActor(this)) + eventProcessActor = Await.result(initEventActorReply, timeout.duration). + asInstanceOf[ActorRef] } + initializeEventProcessActor() + // Called by TaskScheduler to report task's starting. def taskStarted(task: Task[_], taskInfo: TaskInfo) { eventProcessActor ! BeginEvent(task, taskInfo) } // Called to report that a task has completed and results are being fetched remotely. - def taskGettingResult(task: Task[_], taskInfo: TaskInfo) { - eventProcessActor ! GettingResultEvent(task, taskInfo) + def taskGettingResult(taskInfo: TaskInfo) { + eventProcessActor ! GettingResultEvent(taskInfo) } // Called by TaskScheduler to report task completions or failures. @@ -436,7 +419,7 @@ class DAGScheduler( { // Check to make sure we are not launching a task on a partition that does not exist. val maxPartitions = rdd.partitions.length - partitions.find(p => p >= maxPartitions).foreach { p => + partitions.find(p => p >= maxPartitions || p < 0).foreach { p => throw new IllegalArgumentException( "Attempting to access a non-existent partition: " + p + ". " + "Total number of partitions: " + maxPartitions) @@ -511,6 +494,15 @@ class DAGScheduler( eventProcessActor ! AllJobsCancelled } + private[scheduler] def doCancelAllJobs() { + // Cancel all running jobs. + runningStages.map(_.jobId).foreach(handleJobCancellation(_, + reason = "as part of cancellation of all jobs")) + activeJobs.clear() // These should already be empty by this point, + jobIdToActiveJob.clear() // but just in case we lost track of some jobs... + submitWaitingStages() + } + /** * Cancel all jobs associated with a running or scheduled stage. */ @@ -518,148 +510,30 @@ class DAGScheduler( eventProcessActor ! StageCancelled(stageId) } - /** - * Process one event retrieved from the event processing actor. - * - * @param event The event to be processed. - * @return `true` if we should stop the event loop. - */ - private[scheduler] def processEvent(event: DAGSchedulerEvent): Boolean = { - event match { - case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => - var finalStage: Stage = null - try { - // New stage creation may throw an exception if, for example, jobs are run on a HadoopRDD - // whose underlying HDFS files have been deleted. - finalStage = newStage(rdd, partitions.size, None, jobId, Some(callSite)) - } catch { - case e: Exception => - logWarning("Creating new stage failed due to exception - job: " + jobId, e) - listener.jobFailed(e) - return false - } - val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) - clearCacheLocs() - logInfo("Got job " + job.jobId + " (" + callSite + ") with " + partitions.length + - " output partitions (allowLocal=" + allowLocal + ")") - logInfo("Final stage: " + finalStage + " (" + finalStage.name + ")") - logInfo("Parents of final stage: " + finalStage.parents) - logInfo("Missing parents: " + getMissingParentStages(finalStage)) - if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { - // Compute very short actions like first() or take() with no parent stages locally. - listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) - runLocally(job) - } else { - jobIdToActiveJob(jobId) = job - activeJobs += job - resultStageToJob(finalStage) = job - listenerBus.post( - SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, properties)) - submitStage(finalStage) - } - - case StageCancelled(stageId) => - handleStageCancellation(stageId) - - case JobCancelled(jobId) => - handleJobCancellation(jobId) - - case JobGroupCancelled(groupId) => - // Cancel all jobs belonging to this job group. - // First finds all active jobs with this group id, and then kill stages for them. - val activeInGroup = activeJobs.filter(activeJob => - groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) - val jobIds = activeInGroup.map(_.jobId) - jobIds.foreach(jobId => handleJobCancellation(jobId, - "as part of cancelled job group %s".format(groupId))) - - case AllJobsCancelled => - // Cancel all running jobs. - runningStages.map(_.jobId).foreach(jobId => handleJobCancellation(jobId, - "as part of cancellation of all jobs")) - activeJobs.clear() // These should already be empty by this point, - jobIdToActiveJob.clear() // but just in case we lost track of some jobs... - - case ExecutorAdded(execId, host) => - handleExecutorAdded(execId, host) - - case ExecutorLost(execId) => - handleExecutorLost(execId) - - case BeginEvent(task, taskInfo) => - for ( - stage <- stageIdToStage.get(task.stageId); - stageInfo <- stageToInfos.get(stage) - ) { - if (taskInfo.serializedSize > TASK_SIZE_TO_WARN * 1024 && - !stageInfo.emittedTaskSizeWarning) { - stageInfo.emittedTaskSizeWarning = true - logWarning(("Stage %d (%s) contains a task of very large " + - "size (%d KB). The maximum recommended task size is %d KB.").format( - task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, TASK_SIZE_TO_WARN)) - } - } - listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) - - case GettingResultEvent(task, taskInfo) => - listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) - - case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => - val stageId = task.stageId - val taskType = Utils.getFormattedClassName(task) - listenerBus.post(SparkListenerTaskEnd(stageId, taskType, reason, taskInfo, taskMetrics)) - handleTaskCompletion(completion) - - case TaskSetFailed(taskSet, reason) => - stageIdToStage.get(taskSet.stageId).foreach { abortStage(_, reason) } - - case ResubmitFailedStages => - if (failedStages.size > 0) { - // Failed stages may be removed by job cancellation, so failed might be empty even if - // the ResubmitFailedStages event has been scheduled. - resubmitFailedStages() - } - - case StopDAGScheduler => - // Cancel any active jobs - for (job <- activeJobs) { - val error = new SparkException("Job cancelled because SparkContext was shut down") - job.listener.jobFailed(error) - // Tell the listeners that all of the running stages have ended. Don't bother - // cancelling the stages because if the DAG scheduler is stopped, the entire application - // is in the process of getting stopped. - val stageFailedMessage = "Stage cancelled because SparkContext was shut down" - runningStages.foreach { stage => - val info = stageToInfos(stage) - info.stageFailed(stageFailedMessage) - listenerBus.post(SparkListenerStageCompleted(info)) - } - listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) - } - return true - } - false - } - /** * Resubmit any failed stages. Ordinarily called after a small amount of time has passed since * the last fetch failure. */ private[scheduler] def resubmitFailedStages() { - logInfo("Resubmitting failed stages") - clearCacheLocs() - val failedStagesCopy = failedStages.toArray - failedStages.clear() - for (stage <- failedStagesCopy.sortBy(_.jobId)) { - submitStage(stage) + if (failedStages.size > 0) { + // Failed stages may be removed by job cancellation, so failed might be empty even if + // the ResubmitFailedStages event has been scheduled. + logInfo("Resubmitting failed stages") + clearCacheLocs() + val failedStagesCopy = failedStages.toArray + failedStages.clear() + for (stage <- failedStagesCopy.sortBy(_.jobId)) { + submitStage(stage) + } } + submitWaitingStages() } /** * Check for waiting or failed stages which are now eligible for resubmission. * Ordinarily run on every iteration of the event loop. */ - private[scheduler] def submitWaitingStages() { + private def submitWaitingStages() { // TODO: We might want to run this less often, when we are sure that something has become // runnable that wasn't before. logTrace("Checking for newly runnable parent stages") @@ -730,6 +604,102 @@ class DAGScheduler( } } + private[scheduler] def handleJobGroupCancelled(groupId: String) { + // Cancel all jobs belonging to this job group. + // First finds all active jobs with this group id, and then kill stages for them. + val activeInGroup = activeJobs.filter(activeJob => + groupId == activeJob.properties.get(SparkContext.SPARK_JOB_GROUP_ID)) + val jobIds = activeInGroup.map(_.jobId) + jobIds.foreach(handleJobCancellation(_, "part of cancelled job group %s".format(groupId))) + submitWaitingStages() + } + + private[scheduler] def handleBeginEvent(task: Task[_], taskInfo: TaskInfo) { + for (stage <- stageIdToStage.get(task.stageId); stageInfo <- stageToInfos.get(stage)) { + if (taskInfo.serializedSize > DAGScheduler.TASK_SIZE_TO_WARN * 1024 && + !stageInfo.emittedTaskSizeWarning) { + stageInfo.emittedTaskSizeWarning = true + logWarning(("Stage %d (%s) contains a task of very large " + + "size (%d KB). The maximum recommended task size is %d KB.").format( + task.stageId, stageInfo.name, taskInfo.serializedSize / 1024, + DAGScheduler.TASK_SIZE_TO_WARN)) + } + } + listenerBus.post(SparkListenerTaskStart(task.stageId, taskInfo)) + submitWaitingStages() + } + + private[scheduler] def handleTaskSetFailed(taskSet: TaskSet, reason: String) { + stageIdToStage.get(taskSet.stageId).foreach {abortStage(_, reason) } + submitWaitingStages() + } + + private[scheduler] def cleanUpAfterSchedulerStop() { + for (job <- activeJobs) { + val error = new SparkException("Job cancelled because SparkContext was shut down") + job.listener.jobFailed(error) + // Tell the listeners that all of the running stages have ended. Don't bother + // cancelling the stages because if the DAG scheduler is stopped, the entire application + // is in the process of getting stopped. + val stageFailedMessage = "Stage cancelled because SparkContext was shut down" + runningStages.foreach { stage => + val info = stageToInfos(stage) + info.stageFailed(stageFailedMessage) + listenerBus.post(SparkListenerStageCompleted(info)) + } + listenerBus.post(SparkListenerJobEnd(job.jobId, JobFailed(error))) + } + } + + private[scheduler] def handleGetTaskResult(taskInfo: TaskInfo) { + listenerBus.post(SparkListenerTaskGettingResult(taskInfo)) + submitWaitingStages() + } + + private[scheduler] def handleJobSubmitted(jobId: Int, + finalRDD: RDD[_], + func: (TaskContext, Iterator[_]) => _, + partitions: Array[Int], + allowLocal: Boolean, + callSite: String, + listener: JobListener, + properties: Properties = null) + { + var finalStage: Stage = null + try { + // New stage creation may throw an exception if, for example, jobs are run on a + // HadoopRDD whose underlying HDFS files have been deleted. + finalStage = newStage(finalRDD, partitions.size, None, jobId, Some(callSite)) + } catch { + case e: Exception => + logWarning("Creating new stage failed due to exception - job: " + jobId, e) + listener.jobFailed(e) + return + } + if (finalStage != null) { + val job = new ActiveJob(jobId, finalStage, func, partitions, callSite, listener, properties) + clearCacheLocs() + logInfo("Got job %s (%s) with %d output partitions (allowLocal=%s)".format( + job.jobId, callSite, partitions.length, allowLocal)) + logInfo("Final stage: " + finalStage + "(" + finalStage.name + ")") + logInfo("Parents of final stage: " + finalStage.parents) + logInfo("Missing parents: " + getMissingParentStages(finalStage)) + if (allowLocal && finalStage.parents.size == 0 && partitions.length == 1) { + // Compute very short actions like first() or take() with no parent stages locally. + listenerBus.post(SparkListenerJobStart(job.jobId, Array[Int](), properties)) + runLocally(job) + } else { + jobIdToActiveJob(jobId) = job + activeJobs += job + resultStageToJob(finalStage) = job + listenerBus.post(SparkListenerJobStart(job.jobId, jobIdToStageIds(jobId).toArray, + properties)) + submitStage(finalStage) + } + } + submitWaitingStages() + } + /** Submits stage, but first recursively submits any missing parents. */ private def submitStage(stage: Stage) { val jobId = activeJobForStage(stage) @@ -819,9 +789,12 @@ class DAGScheduler( * Responds to a task finishing. This is called inside the event loop so it assumes that it can * modify the scheduler's internal state. Use taskEnded() to post a task end event from outside. */ - private def handleTaskCompletion(event: CompletionEvent) { + private[scheduler] def handleTaskCompletion(event: CompletionEvent) { val task = event.task - + val stageId = task.stageId + val taskType = Utils.getFormattedClassName(task) + listenerBus.post(SparkListenerTaskEnd(stageId, taskType, event.reason, event.taskInfo, + event.taskMetrics)) if (!stageIdToStage.contains(task.stageId)) { // Skip all the actions if the stage has been cancelled. return @@ -964,6 +937,7 @@ class DAGScheduler( // Unrecognized failure - also do nothing. If the task fails repeatedly, the TaskScheduler // will abort the job. } + submitWaitingStages() } /** @@ -973,7 +947,7 @@ class DAGScheduler( * Optionally the epoch during which the failure was caught can be passed to avoid allowing * stray fetch failures from possibly retriggering the detection of a node as lost. */ - private def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { + private[scheduler] def handleExecutorLost(execId: String, maybeEpoch: Option[Long] = None) { val currentEpoch = maybeEpoch.getOrElse(mapOutputTracker.getEpoch) if (!failedEpoch.contains(execId) || failedEpoch(execId) < currentEpoch) { failedEpoch(execId) = currentEpoch @@ -993,17 +967,19 @@ class DAGScheduler( logDebug("Additional executor lost message for " + execId + "(epoch " + currentEpoch + ")") } + submitWaitingStages() } - private def handleExecutorAdded(execId: String, host: String) { + private[scheduler] def handleExecutorAdded(execId: String, host: String) { // remove from failedEpoch(execId) ? if (failedEpoch.contains(execId)) { logInfo("Host added was in lost list earlier: " + host) failedEpoch -= execId } + submitWaitingStages() } - private def handleStageCancellation(stageId: Int) { + private[scheduler] def handleStageCancellation(stageId: Int) { if (stageIdToJobIds.contains(stageId)) { val jobsThatUseStage: Array[Int] = stageIdToJobIds(stageId).toArray jobsThatUseStage.foreach(jobId => { @@ -1012,22 +988,24 @@ class DAGScheduler( } else { logInfo("No active jobs to kill for Stage " + stageId) } + submitWaitingStages() } - private def handleJobCancellation(jobId: Int, reason: String = "") { + private[scheduler] def handleJobCancellation(jobId: Int, reason: String = "") { if (!jobIdToStageIds.contains(jobId)) { logDebug("Trying to cancel unregistered job " + jobId) } else { failJobAndIndependentStages(jobIdToActiveJob(jobId), "Job %d cancelled %s".format(jobId, reason), None) } + submitWaitingStages() } /** * Aborts all jobs depending on a particular Stage. This is called in response to a task set * being canceled by the TaskScheduler. Use taskSetFailed() to inject this event from outside. */ - private def abortStage(failedStage: Stage, reason: String) { + private[scheduler] def abortStage(failedStage: Stage, reason: String) { if (!stageIdToStage.contains(failedStage.id)) { // Skip all the actions if the stage has been removed. return @@ -1156,13 +1134,88 @@ class DAGScheduler( } def stop() { - if (eventProcessActor != null) { - eventProcessActor ! StopDAGScheduler - } + logInfo("Stopping DAGScheduler") + dagSchedulerActorSupervisor ! PoisonPill taskScheduler.stop() } } +private[scheduler] class DAGSchedulerActorSupervisor(dagScheduler: DAGScheduler) + extends Actor with Logging { + + override val supervisorStrategy = + OneForOneStrategy() { + case x: Exception => + logError("eventProcesserActor failed due to the error %s; shutting down SparkContext" + .format(x.getMessage)) + dagScheduler.doCancelAllJobs() + dagScheduler.sc.stop() + Stop + } + + def receive = { + case p: Props => sender ! context.actorOf(p) + case _ => logWarning("received unknown message in DAGSchedulerActorSupervisor") + } +} + +private[scheduler] class DAGSchedulerEventProcessActor(dagScheduler: DAGScheduler) + extends Actor with Logging { + + override def preStart() { + // set DAGScheduler for taskScheduler to ensure eventProcessActor is always + // valid when the messages arrive + dagScheduler.taskScheduler.setDAGScheduler(dagScheduler) + } + + /** + * The main event loop of the DAG scheduler. + */ + def receive = { + case JobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, listener, properties) => + dagScheduler.handleJobSubmitted(jobId, rdd, func, partitions, allowLocal, callSite, + listener, properties) + + case StageCancelled(stageId) => + dagScheduler.handleStageCancellation(stageId) + + case JobCancelled(jobId) => + dagScheduler.handleJobCancellation(jobId) + + case JobGroupCancelled(groupId) => + dagScheduler.handleJobGroupCancelled(groupId) + + case AllJobsCancelled => + dagScheduler.doCancelAllJobs() + + case ExecutorAdded(execId, host) => + dagScheduler.handleExecutorAdded(execId, host) + + case ExecutorLost(execId) => + dagScheduler.handleExecutorLost(execId) + + case BeginEvent(task, taskInfo) => + dagScheduler.handleBeginEvent(task, taskInfo) + + case GettingResultEvent(taskInfo) => + dagScheduler.handleGetTaskResult(taskInfo) + + case completion @ CompletionEvent(task, reason, _, _, taskInfo, taskMetrics) => + dagScheduler.handleTaskCompletion(completion) + + case TaskSetFailed(taskSet, reason) => + dagScheduler.handleTaskSetFailed(taskSet, reason) + + case ResubmitFailedStages => + dagScheduler.resubmitFailedStages() + } + + override def postStop() { + // Cancel any active jobs in postStop hook + dagScheduler.cleanUpAfterSchedulerStop() + } +} + private[spark] object DAGScheduler { // The time, in millis, to wait for fetch failure events to stop coming in after one is detected; // this is a simplistic way to avoid resubmitting tasks in the non-fetchable map stage one by one diff --git a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala index 0800c5684c60f..23f57441b4b11 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/DAGSchedulerEvent.scala @@ -57,7 +57,7 @@ private[scheduler] case class BeginEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] -case class GettingResultEvent(task: Task[_], taskInfo: TaskInfo) extends DAGSchedulerEvent +case class GettingResultEvent(taskInfo: TaskInfo) extends DAGSchedulerEvent private[scheduler] case class CompletionEvent( task: Task[_], @@ -76,5 +76,3 @@ private[scheduler] case class TaskSetFailed(taskSet: TaskSet, reason: String) extends DAGSchedulerEvent private[scheduler] case object ResubmitFailedStages extends DAGSchedulerEvent - -private[scheduler] case object StopDAGScheduler extends DAGSchedulerEvent diff --git a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala index a81b834267f0d..f3bd0797aa035 100644 --- a/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala +++ b/core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala @@ -465,7 +465,7 @@ private[spark] class TaskSetManager( def handleTaskGettingResult(tid: Long) = { val info = taskInfos(tid) info.markGettingResult() - sched.dagScheduler.taskGettingResult(tasks(info.index), info) + sched.dagScheduler.taskGettingResult(info) } /** diff --git a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala index ff69eb7e53f8e..d172dd1ac8e1b 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/DAGSchedulerSuite.scala @@ -21,6 +21,8 @@ import scala.Tuple2 import scala.collection.mutable.{HashSet, HashMap, Map} import scala.language.reflectiveCalls +import akka.actor._ +import akka.testkit.{ImplicitSender, TestKit, TestActorRef} import org.scalatest.{BeforeAndAfter, FunSuite} import org.apache.spark._ @@ -28,19 +30,16 @@ import org.apache.spark.rdd.RDD import org.apache.spark.scheduler.SchedulingMode.SchedulingMode import org.apache.spark.storage.{BlockId, BlockManagerId, BlockManagerMaster} -/** - * Tests for DAGScheduler. These tests directly call the event processing functions in DAGScheduler - * rather than spawning an event loop thread as happens in the real code. They use EasyMock - * to mock out two classes that DAGScheduler interacts with: TaskScheduler (to which TaskSets are - * submitted) and BlockManagerMaster (from which cache locations are retrieved and to which dead - * host notifications are sent). In addition, tests may check for side effects on a non-mocked - * MapOutputTracker instance. - * - * Tests primarily consist of running DAGScheduler#processEvent and - * DAGScheduler#submitWaitingStages (via test utility functions like runEvent or respondToTaskSet) - * and capturing the resulting TaskSets from the mock TaskScheduler. - */ -class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkContext { +class BuggyDAGEventProcessActor extends Actor { + val state = 0 + def receive = { + case _ => throw new SparkException("error") + } +} + +class DAGSchedulerSuite extends TestKit(ActorSystem("DAGSchedulerSuite")) with FunSuite + with ImplicitSender with BeforeAndAfter with LocalSparkContext { + val conf = new SparkConf /** Set of TaskSets the DAGScheduler has requested executed. */ val taskSets = scala.collection.mutable.Buffer[TaskSet]() @@ -82,6 +81,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont var mapOutputTracker: MapOutputTrackerMaster = null var scheduler: DAGScheduler = null + var dagEventProcessTestActor: TestActorRef[DAGSchedulerEventProcessActor] = null /** * Set of cache locations to return from our mock BlockManagerMaster. @@ -121,6 +121,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont results.clear() mapOutputTracker = new MapOutputTrackerMaster(conf) scheduler = new DAGScheduler( + sc, taskScheduler, sc.listenerBus, mapOutputTracker, @@ -131,10 +132,13 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont runLocallyWithinThread(job) } } + dagEventProcessTestActor = TestActorRef[DAGSchedulerEventProcessActor]( + Props(classOf[DAGSchedulerEventProcessActor], scheduler))(system) } - after { - scheduler.stop() + override def afterAll() { + super.afterAll() + TestKit.shutdownActorSystem(system) } /** @@ -178,8 +182,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont * DAGScheduler event loop. */ private def runEvent(event: DAGSchedulerEvent) { - assert(!scheduler.processEvent(event)) - scheduler.submitWaitingStages() + dagEventProcessTestActor.receive(event) } /** @@ -209,7 +212,7 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont listener: JobListener = jobListener): Int = { val jobId = scheduler.nextJobId.getAndIncrement() runEvent(JobSubmitted(jobId, rdd, func, partitions, allowLocal, null, listener)) - return jobId + jobId } /** Sends TaskSetFailed to the scheduler. */ @@ -223,19 +226,17 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont } test("zero split job") { - val rdd = makeRdd(0, Nil) var numResults = 0 val fakeListener = new JobListener() { override def taskSucceeded(partition: Int, value: Any) = numResults += 1 override def jobFailed(exception: Exception) = throw exception } - submit(rdd, Array(), listener = fakeListener) + submit(makeRdd(0, Nil), Array(), listener = fakeListener) assert(numResults === 0) } test("run trivial job") { - val rdd = makeRdd(1, Nil) - submit(rdd, Array(0)) + submit(makeRdd(1, Nil), Array(0)) complete(taskSets(0), List((Success, 42))) assert(results === Map(0 -> 42)) assertDataStructuresEmpty @@ -529,6 +530,18 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assertDataStructuresEmpty } + test("DAGSchedulerActorSupervisor closes the SparkContext when EventProcessActor crashes") { + val actorSystem = ActorSystem("test") + val supervisor = actorSystem.actorOf( + Props(classOf[DAGSchedulerActorSupervisor], scheduler), "dagSupervisor") + supervisor ! Props[BuggyDAGEventProcessActor] + val child = expectMsgType[ActorRef] + watch(child) + child ! "hi" + expectMsgPF(){ case Terminated(child) => () } + assert(scheduler.sc.dagScheduler === null) + } + /** * Assert that the supplied TaskSet has exactly the given hosts as its preferred locations. * Note that this checks only the host and not the executor ID. @@ -561,3 +574,4 @@ class DAGSchedulerSuite extends FunSuite with BeforeAndAfter with LocalSparkCont assert(scheduler.waitingStages.isEmpty) } } + diff --git a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala index 2fb750d9ee378..a8b605c5b212a 100644 --- a/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala +++ b/core/src/test/scala/org/apache/spark/scheduler/TaskSchedulerImplSuite.scala @@ -305,7 +305,7 @@ class TaskSchedulerImplSuite extends FunSuite with LocalSparkContext with Loggin override def taskStarted(task: Task[_], taskInfo: TaskInfo) {} override def executorAdded(execId: String, host: String) {} } - + taskScheduler.setDAGScheduler(dagScheduler) // Give zero core offers. Should not generate any tasks val zeroCoreWorkerOffers = Seq(new WorkerOffer("executor0", "host0", 0), new WorkerOffer("executor1", "host1", 0)) From 87cf35c2d6acc9649b3fb05648b79b9862b3959b Mon Sep 17 00:00:00 2001 From: Sandy Ryza Date: Fri, 25 Apr 2014 17:55:04 -0700 Subject: [PATCH 390/397] SPARK-1632. Remove unnecessary boxing in compares in ExternalAppendOnlyM... ...ap Author: Sandy Ryza Closes #559 from sryza/sandy-spark-1632 and squashes the following commits: a6cd352 [Sandy Ryza] Only compute hashes once 04e3884 [Sandy Ryza] SPARK-1632. Remove unnecessary boxing in compares in ExternalAppendOnlyMap --- .../spark/util/collection/ExternalAppendOnlyMap.scala | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala index d615767284c0b..170f09be21534 100644 --- a/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala +++ b/core/src/main/scala/org/apache/spark/util/collection/ExternalAppendOnlyMap.scala @@ -337,8 +337,8 @@ class ExternalAppendOnlyMap[K, V, C]( } override def compareTo(other: StreamBuffer): Int = { - // minus sign because mutable.PriorityQueue dequeues the max, not the min - -minKeyHash.compareTo(other.minKeyHash) + // descending order because mutable.PriorityQueue dequeues the max, not the min + if (other.minKeyHash < minKeyHash) -1 else if (other.minKeyHash == minKeyHash) 0 else 1 } } } @@ -422,7 +422,9 @@ class ExternalAppendOnlyMap[K, V, C]( private[spark] object ExternalAppendOnlyMap { private class KCComparator[K, C] extends Comparator[(K, C)] { def compare(kc1: (K, C), kc2: (K, C)): Int = { - kc1._1.hashCode().compareTo(kc2._1.hashCode()) + val hash1 = kc1._1.hashCode() + val hash2 = kc2._1.hashCode() + if (hash1 < hash2) -1 else if (hash1 == hash2) 0 else 1 } } } From 058797c1722c9251f6bc6ad2672cb0e79146b04f Mon Sep 17 00:00:00 2001 From: zsxwing Date: Fri, 25 Apr 2014 19:04:34 -0700 Subject: [PATCH 391/397] [Spark-1382] Fix NPE in DStream.slice (updated version of #365) @zsxwing I cherry-picked your changes and merged the master. #365 had some conflicts once again! Author: zsxwing Author: Tathagata Das Closes #562 from tdas/SPARK-1382 and squashes the following commits: e2962c1 [Tathagata Das] Merge remote-tracking branch 'apache-github/master' into SPARK-1382 20968d9 [zsxwing] Replace Exception with SparkException in DStream e476651 [zsxwing] Merge remote-tracking branch 'origin/master' into SPARK-1382 35ba56a [zsxwing] SPARK-1382: Fix NPE in DStream.slice --- .../spark/streaming/dstream/DStream.scala | 22 ++++++++++--------- .../streaming/BasicOperationsSuite.scala | 12 +++++++++- 2 files changed, 23 insertions(+), 11 deletions(-) diff --git a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala index f69f69e0c44af..4709a62381647 100644 --- a/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala +++ b/streaming/src/main/scala/org/apache/spark/streaming/dstream/DStream.scala @@ -18,20 +18,19 @@ package org.apache.spark.streaming.dstream +import java.io.{IOException, ObjectInputStream, ObjectOutputStream} + import scala.deprecated import scala.collection.mutable.HashMap import scala.reflect.ClassTag -import java.io.{IOException, ObjectInputStream, ObjectOutputStream} - -import org.apache.spark.Logging +import org.apache.spark.{Logging, SparkException} import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.storage.StorageLevel -import org.apache.spark.util.MetadataCleaner import org.apache.spark.streaming._ import org.apache.spark.streaming.StreamingContext._ import org.apache.spark.streaming.scheduler.Job -import org.apache.spark.streaming.Duration +import org.apache.spark.util.MetadataCleaner /** * A Discretized Stream (DStream), the basic abstraction in Spark Streaming, is a continuous @@ -144,7 +143,7 @@ abstract class DStream[T: ClassTag] ( */ private[streaming] def initialize(time: Time) { if (zeroTime != null && zeroTime != time) { - throw new Exception("ZeroTime is already initialized to " + zeroTime + throw new SparkException("ZeroTime is already initialized to " + zeroTime + ", cannot initialize it again to " + time) } zeroTime = time @@ -220,7 +219,7 @@ abstract class DStream[T: ClassTag] ( "which requires " + this.getClass.getSimpleName + " to remember generated RDDs for more " + "than " + rememberDuration.milliseconds / 1000 + " seconds. But Spark's metadata cleanup" + "delay is set to " + metadataCleanerDelay + " seconds, which is not sufficient. Please " + - "set the Java property 'spark.cleaner.delay' to more than " + + "set the Java cleaner delay to more than " + math.ceil(rememberDuration.milliseconds / 1000.0).toInt + " seconds." ) @@ -235,7 +234,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def setContext(s: StreamingContext) { if (ssc != null && ssc != s) { - throw new Exception("Context is already set in " + this + ", cannot set it again") + throw new SparkException("Context is already set in " + this + ", cannot set it again") } ssc = s logInfo("Set context for " + this) @@ -244,7 +243,7 @@ abstract class DStream[T: ClassTag] ( private[streaming] def setGraph(g: DStreamGraph) { if (graph != null && graph != g) { - throw new Exception("Graph is already set in " + this + ", cannot set it again") + throw new SparkException("Graph is already set in " + this + ", cannot set it again") } graph = g dependencies.foreach(_.setGraph(graph)) @@ -261,7 +260,7 @@ abstract class DStream[T: ClassTag] ( /** Checks whether the 'time' is valid wrt slideDuration for generating RDD */ private[streaming] def isTimeValid(time: Time): Boolean = { if (!isInitialized) { - throw new Exception (this + " has not been initialized") + throw new SparkException (this + " has not been initialized") } else if (time <= zeroTime || ! (time - zeroTime).isMultipleOf(slideDuration)) { logInfo("Time " + time + " is invalid as zeroTime is " + zeroTime + " and slideDuration is " + slideDuration + " and difference is " + (time - zeroTime)) @@ -728,6 +727,9 @@ abstract class DStream[T: ClassTag] ( * Return all the RDDs between 'fromTime' to 'toTime' (both included) */ def slice(fromTime: Time, toTime: Time): Seq[RDD[T]] = { + if (!isInitialized) { + throw new SparkException(this + " has not been initialized") + } if (!(fromTime - zeroTime).isMultipleOf(slideDuration)) { logWarning("fromTime (" + fromTime + ") is not a multiple of slideDuration (" + slideDuration + ")") diff --git a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala index 4792ca1f8ae3e..04925886c39e4 100644 --- a/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala +++ b/streaming/src/test/scala/org/apache/spark/streaming/BasicOperationsSuite.scala @@ -23,7 +23,7 @@ import org.apache.spark.rdd.{BlockRDD, RDD} import org.apache.spark.SparkContext._ import util.ManualClock -import org.apache.spark.{SparkContext, SparkConf} +import org.apache.spark.{SparkException, SparkConf} import org.apache.spark.streaming.dstream.{WindowedDStream, DStream} import scala.collection.mutable.{SynchronizedBuffer, ArrayBuffer} import scala.reflect.ClassTag @@ -398,6 +398,16 @@ class BasicOperationsSuite extends TestSuiteBase { Thread.sleep(1000) } + test("slice - has not been initialized") { + val ssc = new StreamingContext(conf, Seconds(1)) + val input = Seq(Seq(1), Seq(2), Seq(3), Seq(4)) + val stream = new TestInputStream[Int](ssc, input, 2) + val thrown = intercept[SparkException] { + stream.slice(new Time(0), new Time(1000)) + } + assert(thrown.getMessage.contains("has not been initialized")) + } + val cleanupTestInput = (0 until 10).map(x => Seq(x, x + 1)).toSeq test("rdd cleanup - map and window") { From e6e44e46e345640fd89be9f10cdc448b44d30cc1 Mon Sep 17 00:00:00 2001 From: wangfei Date: Sat, 26 Apr 2014 14:38:42 -0700 Subject: [PATCH 392/397] add note of how to support table with more than 22 fields Author: wangfei Closes #564 from scwf/patch-6 and squashes the following commits: a331876 [wangfei] Update sql-programming-guide.md 685135b [wangfei] Update sql-programming-guide.md 10b3dc0 [wangfei] Update sql-programming-guide.md 1c40480 [wangfei] add note of how to support table with 22 fields --- docs/sql-programming-guide.md | 2 ++ 1 file changed, 2 insertions(+) diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 3ca6cc95bd2db..0c743c9d6010d 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -114,6 +114,8 @@ val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext._ // Define the schema using a case class. +// Note: Case classes in Scala 2.10 can support only up to 22 fields. To work around this limit, +// you can use custom classes that implement the Product interface. case class Person(name: String, age: Int) // Create an RDD of Person objects and register it as a table. From 8e37ed6eb81687140b6cdb00f4ec609ec7ba9be1 Mon Sep 17 00:00:00 2001 From: Takuya UESHIN Date: Sat, 26 Apr 2014 14:39:54 -0700 Subject: [PATCH 393/397] [SPARK-1608] [SQL] Fix Cast.nullable when cast from StringType to NumericType/TimestampType. `Cast.nullable` should be `true` when cast from `StringType` to `NumericType` or `TimestampType`. Because if `StringType` expression has an illegal number string or illegal timestamp string, the casted value becomes `null`. Author: Takuya UESHIN Closes #532 from ueshin/issues/SPARK-1608 and squashes the following commits: 065d37c [Takuya UESHIN] Add tests to check nullabilities of cast expressions. f278ed7 [Takuya UESHIN] Revert test to keep it readable and concise. 9fc9380 [Takuya UESHIN] Fix Cast.nullable when cast from StringType to NumericType/TimestampType. --- .../apache/spark/sql/catalyst/expressions/Cast.scala | 6 +++++- .../expressions/ExpressionEvaluationSuite.scala | 12 ++++++++++++ 2 files changed, 17 insertions(+), 1 deletion(-) 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 8b79b0cd65a84..40d2b42a0cda3 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 @@ -24,7 +24,11 @@ import org.apache.spark.sql.catalyst.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 + def nullable = (child.dataType, dataType) match { + case (StringType, _: NumericType) => true + case (StringType, TimestampType) => true + case _ => child.nullable + } override def toString = s"CAST($child, $dataType)" type EvaluatedType = Any diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala index 4ce0dff9e1586..d287ad73b9e9f 100644 --- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala +++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala @@ -245,6 +245,18 @@ class ExpressionEvaluationSuite extends FunSuite { checkEvaluation(Literal(23.toShort) + Cast(true, ShortType), 24) intercept[Exception] {evaluate(Literal(1) cast BinaryType, null)} + + assert(("abcdef" cast StringType).nullable === false) + assert(("abcdef" cast BinaryType).nullable === false) + assert(("abcdef" cast BooleanType).nullable === false) + assert(("abcdef" cast TimestampType).nullable === true) + assert(("abcdef" cast LongType).nullable === true) + assert(("abcdef" cast IntegerType).nullable === true) + assert(("abcdef" cast ShortType).nullable === true) + assert(("abcdef" cast ByteType).nullable === true) + assert(("abcdef" cast DecimalType).nullable === true) + assert(("abcdef" cast DoubleType).nullable === true) + assert(("abcdef" cast FloatType).nullable === true) } test("timestamp") { From 762af4e9c2837c8ca125838b2ca01e328ee6669f Mon Sep 17 00:00:00 2001 From: Sandeep Date: Sat, 26 Apr 2014 19:04:33 -0700 Subject: [PATCH 394/397] SPARK-1467: Make StorageLevel.apply() factory methods Developer APIs We may want to evolve these in the future to add things like SSDs, so let's mark them as experimental for now. Long-term the right solution might be some kind of builder. The stable API should be the existing StorageLevel constants. Author: Sandeep Closes #551 from techaddict/SPARK-1467 and squashes the following commits: 6bdda24 [Sandeep] SPARK-1467: Make StorageLevel.apply() factory methods as Developer Api's We may want to evolve these in the future to add things like SSDs, so let's mark them as experimental for now. Long-term the right solution might be some kind of builder. The stable API should be the existing StorageLevel constants. --- .../apache/spark/storage/StorageLevel.scala | 26 ++++++++++++++++--- 1 file changed, 22 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala index 95e71de2d3f1d..c9a52e0366d93 100644 --- a/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala +++ b/core/src/main/scala/org/apache/spark/storage/StorageLevel.scala @@ -19,6 +19,8 @@ package org.apache.spark.storage import java.io.{Externalizable, IOException, ObjectInput, ObjectOutput} +import org.apache.spark.annotation.DeveloperApi + /** * Flags for controlling the storage of an RDD. Each StorageLevel records whether to use memory, * or Tachyon, whether to drop the RDD to disk if it falls out of memory or Tachyon , whether to @@ -142,21 +144,37 @@ object StorageLevel { val MEMORY_AND_DISK_SER_2 = new StorageLevel(true, true, false, false, 2) val OFF_HEAP = new StorageLevel(false, false, true, false) - /** Create a new StorageLevel object without setting useOffHeap */ + /** + * :: DeveloperApi :: + * Create a new StorageLevel object without setting useOffHeap + */ + @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, useOffHeap: Boolean, deserialized: Boolean, replication: Int) = getCachedStorageLevel( new StorageLevel(useDisk, useMemory, useOffHeap, deserialized, replication)) - /** Create a new StorageLevel object */ + /** + * :: DeveloperApi :: + * Create a new StorageLevel object + */ + @DeveloperApi def apply(useDisk: Boolean, useMemory: Boolean, deserialized: Boolean, replication: Int = 1) = getCachedStorageLevel( new StorageLevel(useDisk, useMemory, false, deserialized, replication)) - /** Create a new StorageLevel object from its integer representation */ + /** + * :: DeveloperApi :: + * Create a new StorageLevel object from its integer representation + */ + @DeveloperApi def apply(flags: Int, replication: Int): StorageLevel = getCachedStorageLevel(new StorageLevel(flags, replication)) - /** Read StorageLevel object from ObjectInput stream */ + /** + * :: DeveloperApi :: + * Read StorageLevel object from ObjectInput stream + */ + @DeveloperApi def apply(in: ObjectInput): StorageLevel = { val obj = new StorageLevel() obj.readExternal(in) From aa9a7f5db7bffcdbcd7fd53694c606d3a8cdd21f Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sat, 26 Apr 2014 19:24:29 -0700 Subject: [PATCH 395/397] SPARK-1606: Infer user application arguments instead of requiring --arg. This modifies spark-submit to do something more like the Hadoop `jar` command. Now we have the following syntax: ./bin/spark-submit [options] user.jar [user options] Author: Patrick Wendell Closes #563 from pwendell/spark-submit and squashes the following commits: 32241fc [Patrick Wendell] Review feedback 3adfb69 [Patrick Wendell] Small fix bc48139 [Patrick Wendell] SPARK-1606: Infer user application arguments instead of requiring --arg. --- .../org/apache/spark/deploy/SparkSubmit.scala | 1 - .../spark/deploy/SparkSubmitArguments.scala | 226 +++++++++--------- .../scala/org/apache/spark/util/Utils.scala | 7 + .../spark/deploy/SparkSubmitSuite.scala | 77 +++--- docs/cluster-overview.md | 22 +- docs/quick-start.md | 10 +- 6 files changed, 181 insertions(+), 162 deletions(-) diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala index 58aa6d951a204..24edc60684376 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala @@ -185,7 +185,6 @@ object SparkSubmit { if (clusterManager == STANDALONE) { val existingJars = sysProps.get("spark.jars").map(x => x.split(",").toSeq).getOrElse(Seq()) sysProps.put("spark.jars", (existingJars ++ Seq(appArgs.primaryResource)).mkString(",")) - println("SPARK JARS" + sysProps.get("spark.jars")) } if (deployOnCluster && clusterManager == STANDALONE) { diff --git a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala index c545b093ac82e..58d9e9add764a 100644 --- a/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala +++ b/core/src/main/scala/org/apache/spark/deploy/SparkSubmitArguments.scala @@ -21,14 +21,15 @@ import java.io.{File, FileInputStream, IOException} import java.util.Properties import scala.collection.JavaConversions._ -import scala.collection.mutable.{HashMap, ArrayBuffer} +import scala.collection.mutable.{ArrayBuffer, HashMap} import org.apache.spark.SparkException +import org.apache.spark.util.Utils /** * Parses and encapsulates arguments from the spark-submit script. */ -private[spark] class SparkSubmitArguments(args: Array[String]) { +private[spark] class SparkSubmitArguments(args: Seq[String]) { var master: String = null var deployMode: String = null var executorMemory: String = null @@ -118,8 +119,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { if (master.startsWith("yarn")) { val hasHadoopEnv = sys.env.contains("HADOOP_CONF_DIR") || sys.env.contains("YARN_CONF_DIR") - val testing = sys.env.contains("SPARK_TESTING") - if (!hasHadoopEnv && !testing) { + if (!hasHadoopEnv && !Utils.isTesting) { throw new Exception(s"When running with master '$master' " + "either HADOOP_CONF_DIR or YARN_CONF_DIR must be set in the environment.") } @@ -156,119 +156,121 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { """.stripMargin } - private def parseOpts(opts: List[String]): Unit = opts match { - case ("--name") :: value :: tail => - name = value - parseOpts(tail) + /** Fill in values by parsing user options. */ + private def parseOpts(opts: Seq[String]): Unit = { + // Delineates parsing of Spark options from parsing of user options. + var inSparkOpts = true + parse(opts) - case ("--master") :: value :: tail => - master = value - parseOpts(tail) + def parse(opts: Seq[String]): Unit = opts match { + case ("--name") :: value :: tail => + name = value + parse(tail) - case ("--class") :: value :: tail => - mainClass = value - parseOpts(tail) + case ("--master") :: value :: tail => + master = value + parse(tail) - case ("--deploy-mode") :: value :: tail => - if (value != "client" && value != "cluster") { - SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") - } - deployMode = value - parseOpts(tail) - - case ("--num-executors") :: value :: tail => - numExecutors = value - parseOpts(tail) - - case ("--total-executor-cores") :: value :: tail => - totalExecutorCores = value - parseOpts(tail) - - case ("--executor-cores") :: value :: tail => - executorCores = value - parseOpts(tail) - - case ("--executor-memory") :: value :: tail => - executorMemory = value - parseOpts(tail) - - case ("--driver-memory") :: value :: tail => - driverMemory = value - parseOpts(tail) - - case ("--driver-cores") :: value :: tail => - driverCores = value - parseOpts(tail) - - case ("--driver-class-path") :: value :: tail => - driverExtraClassPath = value - parseOpts(tail) - - case ("--driver-java-options") :: value :: tail => - driverExtraJavaOptions = value - parseOpts(tail) - - case ("--driver-library-path") :: value :: tail => - driverExtraLibraryPath = value - parseOpts(tail) - - case ("--properties-file") :: value :: tail => - propertiesFile = value - parseOpts(tail) - - case ("--supervise") :: tail => - supervise = true - parseOpts(tail) - - case ("--queue") :: value :: tail => - queue = value - parseOpts(tail) - - case ("--files") :: value :: tail => - files = value - parseOpts(tail) - - case ("--archives") :: value :: tail => - archives = value - parseOpts(tail) - - case ("--arg") :: value :: tail => - childArgs += value - parseOpts(tail) - - case ("--jars") :: value :: tail => - jars = value - parseOpts(tail) - - case ("--help" | "-h") :: tail => - printUsageAndExit(0) - - case ("--verbose" | "-v") :: tail => - verbose = true - parseOpts(tail) - - case value :: tail => - if (value.startsWith("-")) { - val errMessage = s"Unrecognized option '$value'." - val suggestion: Option[String] = value match { - case v if v.startsWith("--") && v.contains("=") => - val parts = v.split("=") - Some(s"Perhaps you want '${parts(0)} ${parts(1)}'?") - case _ => - None + case ("--class") :: value :: tail => + mainClass = value + parse(tail) + + case ("--deploy-mode") :: value :: tail => + if (value != "client" && value != "cluster") { + SparkSubmit.printErrorAndExit("--deploy-mode must be either \"client\" or \"cluster\"") + } + deployMode = value + parse(tail) + + case ("--num-executors") :: value :: tail => + numExecutors = value + parse(tail) + + case ("--total-executor-cores") :: value :: tail => + totalExecutorCores = value + parse(tail) + + case ("--executor-cores") :: value :: tail => + executorCores = value + parse(tail) + + case ("--executor-memory") :: value :: tail => + executorMemory = value + parse(tail) + + case ("--driver-memory") :: value :: tail => + driverMemory = value + parse(tail) + + case ("--driver-cores") :: value :: tail => + driverCores = value + parse(tail) + + case ("--driver-class-path") :: value :: tail => + driverExtraClassPath = value + parse(tail) + + case ("--driver-java-options") :: value :: tail => + driverExtraJavaOptions = value + parse(tail) + + case ("--driver-library-path") :: value :: tail => + driverExtraLibraryPath = value + parse(tail) + + case ("--properties-file") :: value :: tail => + propertiesFile = value + parse(tail) + + case ("--supervise") :: tail => + supervise = true + parse(tail) + + case ("--queue") :: value :: tail => + queue = value + parse(tail) + + case ("--files") :: value :: tail => + files = value + parse(tail) + + case ("--archives") :: value :: tail => + archives = value + parse(tail) + + case ("--jars") :: value :: tail => + jars = value + parse(tail) + + case ("--help" | "-h") :: tail => + printUsageAndExit(0) + + case ("--verbose" | "-v") :: tail => + verbose = true + parse(tail) + + case value :: tail => + if (inSparkOpts) { + value match { + // convert --foo=bar to --foo bar + case v if v.startsWith("--") && v.contains("=") && v.split("=").size == 2 => + val parts = v.split("=") + parse(Seq(parts(0), parts(1)) ++ tail) + case v if v.startsWith("-") => + val errMessage = s"Unrecognized option '$value'." + SparkSubmit.printErrorAndExit(errMessage) + case v => + primaryResource = v + inSparkOpts = false + parse(tail) + } + } else { + childArgs += value + parse(tail) } - SparkSubmit.printErrorAndExit(errMessage + suggestion.map(" " + _).getOrElse("")) - } - if (primaryResource != null) { - val error = s"Found two conflicting resources, $value and $primaryResource." + - " Expecting only one resource." - SparkSubmit.printErrorAndExit(error) + case Nil => } - primaryResource = value - parseOpts(tail) - - case Nil => } private def printUsageAndExit(exitCode: Int, unknownParam: Any = null) { @@ -277,7 +279,7 @@ private[spark] class SparkSubmitArguments(args: Array[String]) { outStream.println("Unknown/unsupported param " + unknownParam) } outStream.println( - """Usage: spark-submit [options] + """Usage: spark-submit [options] [app options] |Options: | --master MASTER_URL spark://host:port, mesos://host:port, yarn, or local. | --deploy-mode DEPLOY_MODE Mode to deploy the app in, either 'client' or 'cluster'. diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 8351f7156a5e4..5a55e7df34832 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -1056,4 +1056,11 @@ private[spark] object Utils extends Logging { def getHadoopFileSystem(path: String): FileSystem = { getHadoopFileSystem(new URI(path)) } + + /** + * Indicates whether Spark is currently running unit tests. + */ + private[spark] def isTesting = { + sys.env.contains("SPARK_TESTING") || sys.props.contains("spark.testing") + } } diff --git a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala index 657b44668d385..10a65c75cc621 100644 --- a/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala +++ b/core/src/test/scala/org/apache/spark/deploy/SparkSubmitSuite.scala @@ -28,6 +28,9 @@ import org.scalatest.FunSuite import org.scalatest.matchers.ShouldMatchers class SparkSubmitSuite extends FunSuite with ShouldMatchers { + def beforeAll() { + System.setProperty("spark.testing", "true") + } val noOpOutputStream = new OutputStream { def write(b: Int) = {} @@ -74,33 +77,35 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { testPrematureExit(Array("--help"), "Usage: spark-submit") } - test("prints error with unrecognized option") { + test("prints error with unrecognized options") { testPrematureExit(Array("--blarg"), "Unrecognized option '--blarg'") testPrematureExit(Array("-bleg"), "Unrecognized option '-bleg'") - testPrematureExit(Array("--master=abc"), - "Unrecognized option '--master=abc'. Perhaps you want '--master abc'?") } - test("handles multiple binary definitions") { - val adjacentJars = Array("foo.jar", "bar.jar") - testPrematureExit(adjacentJars, "error: Found two conflicting resources") + test("handle binary specified but not class") { + testPrematureExit(Array("foo.jar"), "Must specify a main class") + } - val nonAdjacentJars = - Array("foo.jar", "--master", "123", "--class", "abc", "bar.jar") - testPrematureExit(nonAdjacentJars, "error: Found two conflicting resources") + test("handles arguments with --key=val") { + val clArgs = Seq("--jars=one.jar,two.jar,three.jar", "--name=myApp") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.jars should be ("one.jar,two.jar,three.jar") + appArgs.name should be ("myApp") } - test("handle binary specified but not class") { - testPrematureExit(Array("foo.jar"), "Must specify a main class") + test("handles arguments to user program") { + val clArgs = Seq("--name", "myApp", "userjar.jar", "some", "--random", "args", "here") + val appArgs = new SparkSubmitArguments(clArgs) + appArgs.childArgs should be (Seq("some", "--random", "args", "here")) } test("handles YARN cluster mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", + val clArgs = Seq("--deploy-mode", "cluster", "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", - "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", + "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -121,12 +126,12 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles YARN client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", + val clArgs = Seq("--deploy-mode", "client", "--master", "yarn", "--executor-memory", "5g", "--executor-cores", "5", "--class", "org.SomeClass", "--jars", "one.jar,two.jar,three.jar", - "--arg", "arg1", "--arg", "arg2", "--driver-memory", "4g", - "--queue", "thequeue", "--files", "file1.txt,file2.txt", - "--archives", "archive1.txt,archive2.txt", "--num-executors", "6") + "--driver-memory", "4g", "--queue", "thequeue", "--files", "file1.txt,file2.txt", + "--archives", "archive1.txt,archive2.txt", "--num-executors", "6", "thejar.jar", + "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -144,9 +149,9 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone cluster mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "cluster", - "--master", "spark://h:p", "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--supervise", "--driver-memory", "4g", "--driver-cores", "5") + val clArgs = Seq("--deploy-mode", "cluster", + "--master", "spark://h:p", "--class", "org.SomeClass", + "--supervise", "--driver-memory", "4g", "--driver-cores", "5", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) val childArgsStr = childArgs.mkString(" ") @@ -158,10 +163,9 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles standalone client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", + val clArgs = Seq("--deploy-mode", "client", "--master", "spark://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--driver-memory", "4g") + "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -172,10 +176,9 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { } test("handles mesos client mode") { - val clArgs = Array("thejar.jar", "--deploy-mode", "client", + val clArgs = Seq("--deploy-mode", "client", "--master", "mesos://h:p", "--executor-memory", "5g", "--total-executor-cores", "5", - "--class", "org.SomeClass", "--arg", "arg1", "--arg", "arg2", - "--driver-memory", "4g") + "--class", "org.SomeClass", "--driver-memory", "4g", "thejar.jar", "arg1", "arg2") val appArgs = new SparkSubmitArguments(clArgs) val (childArgs, classpath, sysProps, mainClass) = createLaunchEnv(appArgs) childArgs.mkString(" ") should be ("arg1 arg2") @@ -187,22 +190,24 @@ class SparkSubmitSuite extends FunSuite with ShouldMatchers { test("launch simple application with spark-submit") { runSparkSubmit( - Seq("unUsed.jar", + Seq( "--class", SimpleApplicationTest.getClass.getName.stripSuffix("$"), "--name", "testApp", - "--master", "local")) + "--master", "local", + "unUsed.jar")) } test("spark submit includes jars passed in through --jar") { val jar1 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassA")) val jar2 = TestUtils.createJarWithClasses(Seq("SparkSubmitClassB")) val jarsString = Seq(jar1, jar2).map(j => j.toString).mkString(",") - runSparkSubmit( - Seq("unUsed.jar", - "--class", JarCreationTest.getClass.getName.stripSuffix("$"), - "--name", "testApp", - "--master", "local-cluster[2,1,512]", - "--jars", jarsString)) + val args = Seq( + "--class", JarCreationTest.getClass.getName.stripSuffix("$"), + "--name", "testApp", + "--master", "local-cluster[2,1,512]", + "--jars", jarsString, + "unused.jar") + runSparkSubmit(args) } // NOTE: This is an expensive operation in terms of time (10 seconds+). Use sparingly. diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md index dcc063042628c..b011679fede2d 100644 --- a/docs/cluster-overview.md +++ b/docs/cluster-overview.md @@ -73,30 +73,34 @@ the bin directory. This script takes care of setting up the classpath with Spark dependencies, and can support different cluster managers and deploy modes that Spark supports. It's usage is - ./bin/spark-submit --class path.to.your.Class [other options..] + ./bin/spark-submit --class path.to.your.Class [options] [app options] -To enumerate all options available to `spark-submit` run it with the `--help` flag. -Here are a few examples of common options: +When calling `spark-submit`, `[app options]` will be passed along to your application's +main class. To enumerate all options available to `spark-submit` run it with +the `--help` flag. Here are a few examples of common options: {% highlight bash %} # Run application locally -./bin/spark-submit my-app.jar \ +./bin/spark-submit \ --class my.main.ClassName - --master local[8] + --master local[8] \ + my-app.jar # Run on a Spark cluster -./bin/spark-submit my-app.jar \ +./bin/spark-submit \ --class my.main.ClassName --master spark://mycluster:7077 \ --executor-memory 20G \ - --total-executor-cores 100 + --total-executor-cores 100 \ + my-app.jar # Run on a YARN cluster -HADOOP_CONF_DIR=XX /bin/spark-submit my-app.jar \ +HADOOP_CONF_DIR=XX /bin/spark-submit \ --class my.main.ClassName --master yarn-cluster \ # can also be `yarn-client` for client mode --executor-memory 20G \ - --num-executors 50 + --num-executors 50 \ + my-app.jar {% endhighlight %} ### Loading Configurations from a File diff --git a/docs/quick-start.md b/docs/quick-start.md index 68afa6e1bff95..64996b52e0404 100644 --- a/docs/quick-start.md +++ b/docs/quick-start.md @@ -179,9 +179,10 @@ $ sbt package [info] Packaging {..}/{..}/target/scala-2.10/simple-project_2.10-1.0.jar # Use spark-submit to run your application -$ YOUR_SPARK_HOME/bin/spark-submit target/scala-2.10/simple-project_2.10-1.0.jar \ +$ YOUR_SPARK_HOME/bin/spark-submit \ --class "SimpleApp" \ - --master local[4] + --master local[4] \ + target/scala-2.10/simple-project_2.10-1.0.jar ... Lines with a: 46, Lines with b: 23 {% endhighlight %} @@ -272,9 +273,10 @@ $ mvn package [INFO] Building jar: {..}/{..}/target/simple-project-1.0.jar # Use spark-submit to run your application -$ YOUR_SPARK_HOME/bin/spark-submit target/simple-project-1.0.jar \ +$ YOUR_SPARK_HOME/bin/spark-submit \ --class "SimpleApp" \ - --master local[4] + --master local[4] \ + target/simple-project-1.0.jar ... Lines with a: 46, Lines with b: 23 {% endhighlight %} From 7b2527d74deac9512f8ee9ad6d4b060f05e1ab26 Mon Sep 17 00:00:00 2001 From: Rahul Singhal Date: Sun, 27 Apr 2014 15:17:06 -0700 Subject: [PATCH 396/397] SPARK-1650: Correctly identify maven project version Better account for various side-effect outputs while executing "mvn help:evaluate -Dexpression=project.version" Author: Rahul Singhal Closes #572 from rahulsinghaliitd/SPARK-1650 and squashes the following commits: fd6a611 [Rahul Singhal] SPARK-1650: Correctly identify maven project version --- make-distribution.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/make-distribution.sh b/make-distribution.sh index 55fe6c09d0d66..661d1ff5e54c9 100755 --- a/make-distribution.sh +++ b/make-distribution.sh @@ -43,7 +43,7 @@ FWDIR="$(cd `dirname $0`; pwd)" DISTDIR="$FWDIR/dist" -VERSION=$(mvn help:evaluate -Dexpression=project.version |grep -v "INFO") +VERSION=$(mvn help:evaluate -Dexpression=project.version | grep -v "INFO" | tail -n 1) if [ $? == -1 ] ;then echo -e "You need Maven installed to build Spark." echo -e "Download Maven from https://maven.apache.org." From fe65beeaab737a1c9922b84452ca661b39075cfa Mon Sep 17 00:00:00 2001 From: Patrick Wendell Date: Sun, 27 Apr 2014 15:41:57 -0700 Subject: [PATCH 397/397] SPARK-1648 Support closing JIRA's as part of merge script. Adds an automated hook in the merge script that can close the JIRA, set the fix versions, and leave a comment on the JIRA indicating the PR in which it was resolved. This ensures that (a) we always close JIRA's when issues are merged and (b) there is a link to the pull request in every JIRA. This requires a python library called `jira-client`. We could look at embedding this library in our project, but it seemed simple enough to just gracefully disable this feature if it is not installed. It can be installed with `pip install jira-client`. Author: Patrick Wendell Closes #570 from pwendell/jira-pr-merge and squashes the following commits: 3022b96 [Patrick Wendell] SPARK-1648 Support closing JIRA's as part of merge script. --- dev/merge_spark_pr.py | 114 ++++++++++++++++++++++++++++++++++++++---- 1 file changed, 105 insertions(+), 9 deletions(-) diff --git a/dev/merge_spark_pr.py b/dev/merge_spark_pr.py index 7a61943e94814..8d15c05d50adc 100755 --- a/dev/merge_spark_pr.py +++ b/dev/merge_spark_pr.py @@ -26,19 +26,33 @@ import json import os +import re import subprocess import sys import tempfile import urllib2 +try: + import jira.client + JIRA_IMPORTED=True +except ImportError: + JIRA_IMPORTED=False + # Location of your Spark git development area SPARK_HOME = os.environ.get("SPARK_HOME", "/home/patrick/Documents/spark") # Remote name which points to the Gihub site PR_REMOTE_NAME = os.environ.get("PR_REMOTE_NAME", "apache-github") # Remote name which points to Apache git PUSH_REMOTE_NAME = os.environ.get("PUSH_REMOTE_NAME", "apache") - -GIT_API_BASE = "https://api.github.com/repos/apache/spark" +# ASF JIRA username +JIRA_USERNAME = os.environ.get("JIRA_USERNAME", "pwendell") +# ASF JIRA password +JIRA_PASSWORD = os.environ.get("JIRA_PASSWORD", "1234") + +GITHUB_BASE = "https://github.com/apache/spark/pull" +GITHUB_API_BASE = "https://api.github.com/repos/apache/spark" +JIRA_BASE = "https://issues.apache.org/jira/browse" +JIRA_API_BASE = "https://issues.apache.org/jira" # Prefix added to temporary branches BRANCH_PREFIX = "PR_TOOL" @@ -145,8 +159,7 @@ def merge_pr(pr_num, target_ref): return merge_hash -def maybe_cherry_pick(pr_num, merge_hash, default_branch): - continue_maybe("Would you like to pick %s into another branch?" % merge_hash) +def cherry_pick(pr_num, merge_hash, default_branch): pick_ref = raw_input("Enter a branch name [%s]: " % default_branch) if pick_ref == "": pick_ref = default_branch @@ -171,14 +184,86 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): print("Pull request #%s picked into %s!" % (pr_num, pick_ref)) print("Pick hash: %s" % pick_hash) + return pick_ref + +def fix_version_from_branch(branch, versions): + # Note: Assumes this is a sorted (newest->oldest) list of un-released versions + if branch == "master": + return versions[0] + else: + branch_ver = branch.replace("branch-", "") + return filter(lambda x: x.name.startswith(branch_ver), versions)[-1] + +def resolve_jira(title, merge_branches, comment): + asf_jira = jira.client.JIRA({'server': JIRA_API_BASE}, + basic_auth=(JIRA_USERNAME, JIRA_PASSWORD)) + + default_jira_id = "" + search = re.findall("SPARK-[0-9]{4,5}", title) + if len(search) > 0: + default_jira_id = search[0] + + jira_id = raw_input("Enter a JIRA id [%s]: " % default_jira_id) + if jira_id == "": + jira_id = default_jira_id -branches = get_json("%s/branches" % GIT_API_BASE) + try: + issue = asf_jira.issue(jira_id) + except Exception as e: + fail("ASF JIRA could not find %s\n%s" % (jira_id, e)) + + cur_status = issue.fields.status.name + cur_summary = issue.fields.summary + cur_assignee = issue.fields.assignee + if cur_assignee == None: + cur_assignee = "NOT ASSIGNED!!!" + else: + cur_assignee = cur_assignee.displayName + + if cur_status == "Resolved" or cur_status == "Closed": + fail("JIRA issue %s already has status '%s'" % (jira_id, cur_status)) + print ("=== JIRA %s ===" % jira_id) + print ("summary\t\t%s\nassignee\t%s\nstatus\t\t%s\nurl\t\t%s/%s\n" % ( + cur_summary, cur_assignee, cur_status, JIRA_BASE, jira_id)) + + versions = asf_jira.project_versions("SPARK") + versions = sorted(versions, key = lambda x: x.name, reverse=True) + versions = filter(lambda x: x.raw['released'] == False, versions) + + default_fix_versions = map(lambda x: fix_version_from_branch(x, versions).name, merge_branches) + for v in default_fix_versions: + # Handles the case where we have forked a release branch but not yet made the release. + # In this case, if the PR is committed to the master branch and the release branch, we + # only consider the release branch to be the fix version. E.g. it is not valid to have + # both 1.1.0 and 1.0.0 as fix versions. + (major, minor, patch) = v.split(".") + if patch == 0: + previous = "%s.%s.%s" % (major, int(minor) - 1, 0) + if previous in default_fix_versions: + default_fix_versions = filter(lambda x: x != v, default_fix_versions) + default_fix_versions = ",".join(default_fix_versions) + + fix_versions = raw_input("Enter comma-separated fix version(s) [%s]: " % default_fix_versions) + if fix_versions == "": + fix_versions = default_fix_versions + fix_versions = fix_versions.replace(" ", "").split(",") + + def get_version_json(version_str): + return filter(lambda v: v.name == version_str, versions)[0].raw + jira_fix_versions = map(lambda v: get_version_json(v), fix_versions) + + resolve = filter(lambda a: a['name'] == "Resolve Issue", asf_jira.transitions(jira_id))[0] + asf_jira.transition_issue(jira_id, resolve["id"], fixVersions=jira_fix_versions, comment=comment) + + print "Succesfully resolved %s with fixVersions=%s!" % (jira_id, fix_versions) + +branches = get_json("%s/branches" % GITHUB_API_BASE) branch_names = filter(lambda x: x.startswith("branch-"), [x['name'] for x in branches]) # Assumes branch names can be sorted lexicographically latest_branch = sorted(branch_names, reverse=True)[0] pr_num = raw_input("Which pull request would you like to merge? (e.g. 34): ") -pr = get_json("%s/pulls/%s" % (GIT_API_BASE, pr_num)) +pr = get_json("%s/pulls/%s" % (GITHUB_API_BASE, pr_num)) url = pr["url"] title = pr["title"] @@ -208,11 +293,22 @@ def maybe_cherry_pick(pr_num, merge_hash, default_branch): continue_maybe(msg) print ("\n=== Pull Request #%s ===" % pr_num) -print("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( +print ("title\t%s\nsource\t%s\ntarget\t%s\nurl\t%s" % ( title, pr_repo_desc, target_ref, url)) continue_maybe("Proceed with merging pull request #%s?" % pr_num) +merged_refs = [target_ref] + merge_hash = merge_pr(pr_num, target_ref) -while True: - maybe_cherry_pick(pr_num, merge_hash, latest_branch) +pick_prompt = "Would you like to pick %s into another branch?" % merge_hash +while raw_input("\n%s (y/n): " % pick_prompt).lower() == "y": + merged_refs = merged_refs + [cherry_pick(pr_num, merge_hash, latest_branch)] + +if JIRA_IMPORTED: + continue_maybe("Would you like to update an associated JIRA?") + jira_comment = "Issue resolved by pull request %s\n[%s/%s]" % (pr_num, GITHUB_BASE, pr_num) + resolve_jira(title, merged_refs, jira_comment) +else: + print "Could not find jira-python library. Run 'sudo pip install jira-python' to install." + print "Exiting without trying to close the associated JIRA."